deniskuzZ commented on code in PR #6039:
URL: https://github.com/apache/hive/pull/6039#discussion_r2291015220


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergTableOptimizer.java:
##########
@@ -85,52 +85,53 @@ public IcebergTableOptimizer(HiveConf conf, TxnStore 
txnHandler, MetadataCache m
    */
   @Override
   public Set<CompactionInfo> findPotentialCompactions(long lastChecked, 
ShowCompactResponse currentCompactions,
-      Set<String> skipDBs, Set<String> skipTables) {
+                                                      Set<String> skipDBs, 
Set<String> skipTables) {
     Set<CompactionInfo> compactionTargets = Sets.newHashSet();
 
-    getTableNames().stream()
-        .filter(table -> !skipDBs.contains(table.getDb()))
-        .filter(table -> !skipTables.contains(table.getNotEmptyDbTable()))
-        .map(table -> {
-          org.apache.hadoop.hive.ql.metadata.Table hiveTable = 
getHiveTable(table.getDb(), table.getTable());
-          org.apache.iceberg.Table icebergTable = 
IcebergTableUtil.getTable(conf, hiveTable.getTTable());
-          return Pair.of(hiveTable, icebergTable);
-        })
-        .filter(t -> hasNewCommits(t.getRight(),
-            snapshotTimeMilCache.get(t.getLeft().getFullyQualifiedName())))
-        .forEach(t -> {
-          String qualifiedTableName = t.getLeft().getFullyQualifiedName();
-          org.apache.hadoop.hive.ql.metadata.Table hiveTable = t.getLeft();
-          org.apache.iceberg.Table icebergTable = t.getRight();
-
-          if (icebergTable.spec().isPartitioned()) {
-            List<org.apache.hadoop.hive.ql.metadata.Partition> partitions = 
findModifiedPartitions(hiveTable,
-                icebergTable, snapshotTimeMilCache.get(qualifiedTableName), 
true);
-
-            partitions.forEach(partition -> 
addCompactionTargetIfEligible(hiveTable.getTTable(), icebergTable,
-                partition.getName(), compactionTargets, currentCompactions, 
skipDBs, skipTables));
-
-            if (IcebergTableUtil.hasUndergonePartitionEvolution(icebergTable) 
&& !findModifiedPartitions(hiveTable,
-                icebergTable, snapshotTimeMilCache.get(qualifiedTableName), 
false).isEmpty()) {
-              addCompactionTargetIfEligible(hiveTable.getTTable(), 
icebergTable,
-                  null, compactionTargets, currentCompactions, skipDBs, 
skipTables);
-            }
-          } else {
-            addCompactionTargetIfEligible(hiveTable.getTTable(), icebergTable, 
null, compactionTargets,
-                currentCompactions, skipDBs, skipTables);
-          }
-
-          snapshotTimeMilCache.put(qualifiedTableName, 
icebergTable.currentSnapshot().timestampMillis());
-        });
+    Iterable<Table> tables = getTables();

Review Comment:
   `getHiveTable` is no longer used
   
   `skipDBs` and `skipTables` add an extra filter post select, maybe we can 
optimize, by propagating it to TableFetcher
   
   ````
   List<String> databases = client.getDatabases(catalogName, dbPattern)
     .filter(dbName -> !skipDBs.contains(dbName)
     .toList()
   ````
   and 
   ````
    List<String> tablesNames = getTableNamesForDatabase(catalogName, db);
     .filter(tblName -> !skipTables.contains(tblName)
     .toList()
   ````



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org
For additional commands, e-mail: gitbox-h...@hive.apache.org

Reply via email to