You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/27 12:51:32 UTC

[GitHub] [iceberg] pvary opened a new pull request #1394: Hive: Add logging for Hive related classes

pvary opened a new pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394


   As suggested on the dev list added new logs so it will be easier to follow what is happening in case there is a problem using Iceberg/Hive integration.
   
   Modified the existing logging in one place to collapse a multiline log to a single line. Mulitline logs are not very useful with high concurrency since the different log messages are prone to interfere with each other. 


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478757514



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +141,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Table is dropped [{}]", identifier);

Review comment:
       How about "Dropped table: {}" instead of using a passive form? We also tend to use `:` instead of brackets in messages.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479287144



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -65,13 +68,17 @@ public static Table loadTable(Configuration conf, Properties props) {
   private static Table loadTable(Configuration conf, String tableIdentifier, String tableLocation) {
     Optional<Catalog> catalog = loadCatalog(conf);
 
+    Table table;
     if (catalog.isPresent()) {
       Preconditions.checkArgument(tableIdentifier != null, "Table identifier not set");
-      return catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+      table = catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+    } else {
+      Preconditions.checkArgument(tableLocation != null, "Table location not set");
+      table = new HadoopTables(conf).load(tableLocation);
     }
 
-    Preconditions.checkArgument(tableLocation != null, "Table location not set");
-    return new HadoopTables(conf).load(tableLocation);
+    LOG.info("Table loaded by catalog: [{}]", table);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479490435



##########
File path: bundled-guava/src/main/java/org/apache/iceberg/GuavaClasses.java
##########
@@ -81,6 +82,7 @@
     Multimaps.class.getName();
     Ordering.class.getName();
     Sets.class.getName();
+    Stopwatch.class.getName();

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479270527



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -130,6 +130,7 @@ protected void doRefresh() {
     }
 
     refreshFromMetadataLocation(metadataLocation);
+    LOG.debug("Refreshed [{}]", fullName);

Review comment:
       removed




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478430717



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +182,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table form: [{}], to: [{}]", from, to);

Review comment:
       Fixed. Thanks!




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478760991



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -65,13 +68,17 @@ public static Table loadTable(Configuration conf, Properties props) {
   private static Table loadTable(Configuration conf, String tableIdentifier, String tableLocation) {
     Optional<Catalog> catalog = loadCatalog(conf);
 
+    Table table;
     if (catalog.isPresent()) {
       Preconditions.checkArgument(tableIdentifier != null, "Table identifier not set");
-      return catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+      table = catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+    } else {
+      Preconditions.checkArgument(tableLocation != null, "Table location not set");
+      table = new HadoopTables(conf).load(tableLocation);
     }
 
-    Preconditions.checkArgument(tableLocation != null, "Table location not set");
-    return new HadoopTables(conf).load(tableLocation);
+    LOG.info("Table loaded by catalog: [{}]", table);

Review comment:
       I think this should be done in the catalog or `HadoopTables`, not in `Catalogs`. Otherwise, we don't get the log message consistently when table loads go directly through the catalog (as in Spark 3). And if we were to later add a message to the catalog, then this would be a duplicate.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479202069



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +141,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Table is dropped [{}]", identifier);
       return true;
 
     } catch (NoSuchTableException | NoSuchObjectException e) {
+      LOG.debug("Unable to drop missing table: [{}]", identifier, e);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r480924004



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -300,17 +300,11 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    boolean result = alterHiveDataBase(namespace, database);
+    alterHiveDataBase(namespace, database);
+    LOG.debug("Successfully set properties for {}", namespace);

Review comment:
       > > This way we will at least know how the SparkCatalog.alterNamespace/FlinkCatalog.alterNamespace classified the properties, and which one was removed which one was set instead.
   > 
   > I don't think I follow your logic here. All this says is that `setProperties` succeeded. But that's usually obvious from the job because the SQL that's running is `ALTER TABLE ... SET TBLPROPERTIES` or `ALTER TABLE ... UNSET TBLPROPERTIES`. So it seems to me that this is already clear from context.
   
   I understand your concerns about logging where the only added info we have is that the code line is reached, and I agree that is rarely useful (I occasionally find them handy when debugging performance issues, but that is a rare case). This is why I have removed the logs in the first commit.
   
   After pushing the commit I checked again how the catalog calls are called.
   In the fear of stating the obvious:
   - SparkCatalog.alterNamespace/FlinkCatalog.alterNamespace gets an array of NamespaceChange and splits them to 2 lists updates/removals.
   - Updates ends in setProperties call, removals ends in a removeProperties call.
   
   That is why I have added back and modified the setProperties logging, so it will log not only the call but also logs the *properties.keySet()* too. This way we could know which properties were removed, and which were updated in this call. I think this could be beneficial to have in debug level, unless this info is also available somewhere else in the logs.
   
   > > I kinda feel that I have dragged on too much on your patience, so one last request and I will remove both log lines from the code.
   > 
   > Don't worry about it! I had a similar thought because I'm asking for so many logs to be removed or changed. Sounds like we're both happy to come to an agreement and improve logging, even if that takes a lot of discussion. Thanks for your patience here.
   
   Good to meet a likeminded person, and thanks for all the useful help here!




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r480977428



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +141,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Table is dropped [{}]", identifier);

Review comment:
       I too have endured this pain of whitespace at the start/end of a string like this causing all kinds of problems and being really hard to spot as the cause!




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479500812



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -83,28 +87,33 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
               .impl(catalogLoaderClass)
               .build()
               .newInstance();
-      return Optional.of(loader.load(conf));
+      Catalog catalog = loader.load(conf);
+      LOG.info("Loaded catalog {} using {}", catalog, catalogLoaderClass);
+      return Optional.of(catalog);
     }
 
     String catalogName = conf.get(InputFormatConfig.CATALOG);
 
     if (catalogName != null) {
+      Catalog catalog;
       switch (catalogName.toLowerCase()) {
         case HADOOP:
           String warehouseLocation = conf.get(InputFormatConfig.HADOOP_CATALOG_WAREHOUSE_LOCATION);
 
-          if (warehouseLocation != null) {
-            return Optional.of(new HadoopCatalog(conf, warehouseLocation));
-          }
-
-          return Optional.of(new HadoopCatalog(conf));
+          catalog = (warehouseLocation != null) ? new HadoopCatalog(conf, warehouseLocation) : new HadoopCatalog(conf);
+          break;
         case HIVE:
-          return Optional.of(HiveCatalogs.loadCatalog(conf));
+          catalog = HiveCatalogs.loadCatalog(conf);
+          break;
         default:
           throw new NoSuchNamespaceException("Catalog " + catalogName + " is not supported.");
       }
+
+      LOG.info("Catalog is used: {}", catalog);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479492410



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -101,12 +109,25 @@ public void commit(TableMetadata base, TableMetadata metadata) {
       return;
     }
 
-    doCommit(base, metadata);
+    Stopwatch stopwatch = Stopwatch.createStarted();
+    String newMetadataLocation = doCommit(base, metadata);
     deleteRemovedMetadataFiles(base, metadata);
     requestRefresh();
+    stopwatch.stop();
+
+    LOG.info("Successfully committed to table: {} in: {} ms: {}",
+        fullName,
+        newMetadataLocation,
+        stopwatch.elapsed(TimeUnit.MILLISECONDS));
   }
 
-  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+  /**
+   * Executes the commit in the Metastore
+   * @param base The base table metadata
+   * @param metadata The new table metadata
+   * @return The new metadata location
+   */
+  protected String doCommit(TableMetadata base, TableMetadata metadata) {

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478758463



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(
           Arrays.copyOfRange(createStack, 1, createStack.length));
-      LOG.warn("Unclosed input stream created by:\n\t{}", trace);
+      LOG.warn("Unclosed input stream created by:\\n\t{}", trace);
     }
   }
+
+  @Override
+  public String toString() {
+    return "HiveCatalog [name=" + name + ", location=" + this.conf.get("hive.metastore.uris") + "]";

Review comment:
       In configuration, we use `uri` instead of `location`. Can you use that here?
   
   Also, other `toString` methods use the helpers in `MoreObjects`. To have a common format, could you use that instead?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481232797



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -54,6 +54,10 @@
   protected BaseMetastoreTableOperations() {
   }
 
+  protected String fullName() {

Review comment:
       Could we rename this to `tableName` so it is more clear for implementations? It would also be good to add Javadoc.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478757173



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -89,10 +89,13 @@ public HiveCatalog(String name, String uri, int clientPoolSize, Configuration co
 
     try {
       List<String> tables = clients.run(client -> client.getAllTables(database));
-      return tables.stream()
+      List<TableIdentifier> tableIdentifiers = tables.stream()
           .map(t -> TableIdentifier.of(namespace, t))
           .collect(Collectors.toList());
 
+      LOG.debug("Listing of namespace [{}] resulted in the following tables: [{}]", namespace, tableIdentifiers);

Review comment:
       What is the purpose of this debug message? I'm not sure how it would help.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479268647



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -143,9 +144,11 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       // TODO add lock heart beating for cases where default lock timeout is too low.
       Table tbl;
       if (base != null) {
+        LOG.info("Committing existing table [{}]", fullName);

Review comment:
       This was a bit more involved. Please check the resulting method changes:
   - BaseMetastoreTableOperations did not know about the tableName - added fullName
   - BaseMetastoreTableOperations#commit did not know about the newMetadataLocation - added newMetadataLocation as a return value for doCommit
   - Added Stopwatch to the used guava classes 




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479606129



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -63,7 +66,10 @@ private TableIdentifier canonicalizeIdentifier(TableIdentifier tableIdentifier)
 
   @Override
   public Table loadTable(TableIdentifier ident) {
-    return tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    Table result = tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    LOG.info("Table loaded by catalog: {}", result);

Review comment:
       Ok, then cache hit will be marked by the absence of the log message.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481234064



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -83,28 +86,32 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
               .impl(catalogLoaderClass)
               .build()
               .newInstance();
-      return Optional.of(loader.load(conf));
+      Catalog catalog = loader.load(conf);
+      LOG.info("Loaded catalog {} using {}", catalog, catalogLoaderClass);
+      return Optional.of(catalog);
     }
 
     String catalogName = conf.get(InputFormatConfig.CATALOG);
 
     if (catalogName != null) {
+      Catalog catalog;
       switch (catalogName.toLowerCase()) {
         case HADOOP:
           String warehouseLocation = conf.get(InputFormatConfig.HADOOP_CATALOG_WAREHOUSE_LOCATION);
 
-          if (warehouseLocation != null) {
-            return Optional.of(new HadoopCatalog(conf, warehouseLocation));
-          }
-
-          return Optional.of(new HadoopCatalog(conf));
+          catalog = (warehouseLocation != null) ? new HadoopCatalog(conf, warehouseLocation) : new HadoopCatalog(conf);
+          LOG.info("Loaded Hadoop catalog {}", catalog);
+          return Optional.of(catalog);
         case HIVE:
-          return Optional.of(HiveCatalogs.loadCatalog(conf));
+          catalog = HiveCatalogs.loadCatalog(conf);
+          LOG.info("Loaded Hive Metastore catalog {}", catalog);
+          return Optional.of(catalog);
         default:
           throw new NoSuchNamespaceException("Catalog " + catalogName + " is not supported.");
       }
     }
 
+    LOG.info("No catalog is used");

Review comment:
       How about `Catalog is not configured` instead?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479494971



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -63,7 +66,10 @@ private TableIdentifier canonicalizeIdentifier(TableIdentifier tableIdentifier)
 
   @Override
   public Table loadTable(TableIdentifier ident) {
-    return tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    Table result = tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    LOG.info("Table loaded by catalog: {}", result);

Review comment:
       I was thinking around the same lines, but I did not find a way to differentiate between the cache hit, and the cache miss. 
   Shall we just remove the log, or you have some idea how to check for cache hit?
   Removed the log for the time being.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479420789



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java
##########
@@ -71,20 +74,24 @@ public HadoopTables(Configuration conf) {
    */
   @Override
   public Table load(String location) {
+    Table result;
     Pair<String, MetadataTableType> parsedMetadataType = parseMetadataType(location);
 
     if (parsedMetadataType != null) {
       // Load a metadata table
-      return loadMetadataTable(parsedMetadataType.first(), parsedMetadataType.second());
+      result = loadMetadataTable(parsedMetadataType.first(), parsedMetadataType.second());
     } else {
       // Load a normal table
       TableOperations ops = newTableOps(location);
       if (ops.current() != null) {
-        return new BaseTable(ops, location);
+        result = new BaseTable(ops, location);
       } else {
         throw new NoSuchTableException("Table does not exist at location: " + location);
       }
     }
+
+    LOG.info("Table loaded by HadoopTables: {}", result);

Review comment:
       `HadoopTables` is logging this message, so there is no need to add it here.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-684941667


   Thanks, @pvary! Just a couple minor things to fix and I'll merge this.


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479328595



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       Reverted in the meantime to proceed with the patch




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481351128



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -111,7 +116,7 @@ protected void doRefresh() {
     String metadataLocation = null;
     try {
       Table table = metaClients.run(client -> client.getTable(database, tableName));
-      validateTableIsIceberg(table, fullName);
+      validateTableIsIceberg(table, fullName());

Review comment:
       Reverted




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481379786



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -329,7 +350,9 @@ private boolean alterHiveDataBase(Namespace namespace,  Database database) {
 
     try {
       Database database = clients.run(client -> client.getDatabase(namespace.level(0)));
-      return convertToMetadata(database);
+      Map<String, String> metadata = convertToMetadata(database);
+      LOG.debug("Loaded metadata for namespace {}", namespace);

Review comment:
       Thanks for the quick turn-around!




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479498443



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -296,7 +319,10 @@ public boolean removeProperties(Namespace namespace,  Set<String> properties) {
     properties.forEach(key -> parameter.put(key, null));
     Database database = convertToDatabase(namespace, parameter);
 
-    return alterHiveDataBase(namespace, database);
+    boolean result = alterHiveDataBase(namespace, database);
+    LOG.debug("Remove properties: {} from: {} resulted: {}", properties, namespace, result);

Review comment:
       We do not log the actual values, just the properties we are removing.
   Could it also contain sensitive information?
   If not I would keep the log message, so we can be able to trace things in case of trouble




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479423556



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -285,7 +300,15 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    return alterHiveDataBase(namespace, database);
+    boolean result = alterHiveDataBase(namespace, database);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Set properties for: {} to: {} resulted: {}",
+          namespace,
+          Joiner.on(",").withKeyValueSeparator("=").join(properties),
+          result);

Review comment:
       I'm really skeptical that this is a good idea. Properties can contain sensitive information that we don't want to leak into logs. If the operation succeeds, we can get the properties directly from the table. If it fails, we will get an exception and the properties that were attempted aren't very important (the user should also be able to tell us what they were).




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478761972



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java
##########
@@ -227,6 +227,7 @@ public void close() throws IOException {
 
     private CloseableIterable<T> open(FileScanTask currentTask, Schema readSchema) {
       DataFile file = currentTask.file();
+      LOG.debug("Opening [{}] for read", file);

Review comment:
       Aren't there already logs for this from the underlying file system implementation?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479508968



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -296,7 +319,10 @@ public boolean removeProperties(Namespace namespace,  Set<String> properties) {
     properties.forEach(key -> parameter.put(key, null));
     Database database = convertToDatabase(namespace, parameter);
 
-    return alterHiveDataBase(namespace, database);
+    boolean result = alterHiveDataBase(namespace, database);
+    LOG.debug("Remove properties: {} from: {} resulted: {}", properties, namespace, result);

Review comment:
       If not logging the properties, then what is the value of this log? If the query fails, it will throw an exception so the only time this will run is when the result is `true`. That doesn't seem useful to me, considering this is called from an operation that probably logs itself, like `updateProperties`.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479420342



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -63,7 +66,10 @@ private TableIdentifier canonicalizeIdentifier(TableIdentifier tableIdentifier)
 
   @Override
   public Table loadTable(TableIdentifier ident) {
-    return tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    Table result = tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    LOG.info("Table loaded by catalog: {}", result);

Review comment:
       I think this is misleading. What we want to know is whether a cached version of the table was used or whether the catalog loaded the table. The catalog should log when it loads a table, so this just needs to log when there was a cache hit.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479295929



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java
##########
@@ -227,6 +227,7 @@ public void close() throws IOException {
 
     private CloseableIterable<T> open(FileScanTask currentTask, Schema readSchema) {
       DataFile file = currentTask.file();
+      LOG.debug("Opening [{}] for read", file);

Review comment:
       You are right. Removed.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-685118055


   Thanks @pvary! Merged.


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481380033



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -329,7 +350,9 @@ private boolean alterHiveDataBase(Namespace namespace,  Database database) {
 
     try {
       Database database = clients.run(client -> client.getDatabase(namespace.level(0)));
-      return convertToMetadata(database);
+      Map<String, String> metadata = convertToMetadata(database);
+      LOG.debug("Loaded metadata for namespace {}", namespace);

Review comment:
       I did not log the metadata loaded, as I should.
   Added the log of the loaded metadata keys, like this:
   ```
   Loaded metadata for namespace dbname_load found [owner, location, comment, group]
   ```




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479419375



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -101,12 +109,25 @@ public void commit(TableMetadata base, TableMetadata metadata) {
       return;
     }
 
-    doCommit(base, metadata);
+    Stopwatch stopwatch = Stopwatch.createStarted();
+    String newMetadataLocation = doCommit(base, metadata);
     deleteRemovedMetadataFiles(base, metadata);
     requestRefresh();
+    stopwatch.stop();
+
+    LOG.info("Successfully committed to table: {} in: {} ms: {}",
+        fullName,
+        newMetadataLocation,
+        stopwatch.elapsed(TimeUnit.MILLISECONDS));
   }
 
-  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+  /**
+   * Executes the commit in the Metastore
+   * @param base The base table metadata
+   * @param metadata The new table metadata
+   * @return The new metadata location
+   */
+  protected String doCommit(TableMetadata base, TableMetadata metadata) {

Review comment:
       I don't think we should change the signature of this method, since there are external implementations. This also highlights that there is not even a requirement to store metadata in a file (this could be implemented with a backing database). So we probably shouldn't log the metadata location from this class.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478400542



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +182,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table form: [{}], to: [{}]", from, to);

Review comment:
       ```suggestion
         LOG.info("Renamed table from: [{}], to: [{}]", from, to);
   ```

##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       Is the additional `\` here intentional? If so, what behaviour changes?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479492598



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -101,12 +109,25 @@ public void commit(TableMetadata base, TableMetadata metadata) {
       return;
     }
 
-    doCommit(base, metadata);
+    Stopwatch stopwatch = Stopwatch.createStarted();
+    String newMetadataLocation = doCommit(base, metadata);
     deleteRemovedMetadataFiles(base, metadata);
     requestRefresh();
+    stopwatch.stop();
+
+    LOG.info("Successfully committed to table: {} in: {} ms: {}",

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-683347588


   Looks like the current issue is a checkstyle failure:
   
   ```
   > Task :iceberg-hive:checkstyleMain FAILED
   [ant:checkstyle] [ERROR] /home/travis/build/apache/iceberg/hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:249: Trailing whitespace is not allowed. [RegexpSinglelineJava]
   [ant:checkstyle] [ERROR] /home/travis/build/apache/iceberg/hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:249: Whitespace at end-of-line [RegexpSingleline]
   ```


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479201683



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +141,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Table is dropped [{}]", identifier);

Review comment:
       Done, I will also remove every '[]' - I tend to use them because once spent 2 days trying to figure out why my code is not able to find table 'sneakytable', where the log cleanly shows that 'sneakytable ' is in the listing (notice the missing ' ', which is hardly noticeable in the logs if the beginning/end of the variable is not marked clearly, and it is the end of the line :))




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479495739



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -89,10 +91,13 @@ public HiveCatalog(String name, String uri, int clientPoolSize, Configuration co
 
     try {
       List<String> tables = clients.run(client -> client.getAllTables(database));
-      return tables.stream()
+      List<TableIdentifier> tableIdentifiers = tables.stream()
           .map(t -> TableIdentifier.of(namespace, t))
           .collect(Collectors.toList());
 
+      LOG.debug("Listing of namespace: {} resulted in the following tables: {}", namespace, tableIdentifiers);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479497774



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -285,7 +300,15 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    return alterHiveDataBase(namespace, database);
+    boolean result = alterHiveDataBase(namespace, database);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Set properties for: {} to: {} resulted: {}",
+          namespace,
+          Joiner.on(",").withKeyValueSeparator("=").join(properties),
+          result);

Review comment:
       Good point!
   Removed the printing of the parameters.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478405334



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       Collapsing the multiline log to a single line - was not brave enough to remove the \n-s altogether.
   This should create logs like:
   ```
   Unclosed input stream created by:\n {Stacktrace.line1\n  Stractrace.line2\n  Stractrace.line3}
   ```




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479496041



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +184,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table from: {}, to: {}", from, to);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479728173



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -300,17 +300,11 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    boolean result = alterHiveDataBase(namespace, database);
+    alterHiveDataBase(namespace, database);
+    LOG.debug("Successfully set properties for {}", namespace);

Review comment:
       First I removed the logs, but then thought better of it.
   Added it back with the list of the name of the properties we decided to set.
   This way we will at least know how the SparkCatalog.alterNamespace/FlinkCatalog.alterNamespace classified the properties, and which one was removed which one was set instead.
   Do you think it is worthwhile to log this way?
   I kinda feel that I have dragged on too much on your patience, so one last request and I will remove both log lines from the code.
   Thanks again!
   Peter




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481233624



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -111,7 +116,7 @@ protected void doRefresh() {
     String metadataLocation = null;
     try {
       Table table = metaClients.run(client -> client.getTable(database, tableName));
-      validateTableIsIceberg(table, fullName);
+      validateTableIsIceberg(table, fullName());

Review comment:
       Is this change needed? This is an instance variable so there should be no need to call the getter.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479509408



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -63,7 +66,10 @@ private TableIdentifier canonicalizeIdentifier(TableIdentifier tableIdentifier)
 
   @Override
   public Table loadTable(TableIdentifier ident) {
-    return tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    Table result = tableCache.get(canonicalizeIdentifier(ident), catalog::loadTable);
+    LOG.info("Table loaded by catalog: {}", result);

Review comment:
       Well, a cache hit could just be the absence of a log message. But you could also inject an atomic boolean in the load table call that is passed to the cache. If it gets called, it sets the boolean to true. Then the log only triggers if it was not called.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479196916



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -89,10 +89,13 @@ public HiveCatalog(String name, String uri, int clientPoolSize, Configuration co
 
     try {
       List<String> tables = clients.run(client -> client.getAllTables(database));
-      return tables.stream()
+      List<TableIdentifier> tableIdentifiers = tables.stream()
           .map(t -> TableIdentifier.of(namespace, t))
           .collect(Collectors.toList());
 
+      LOG.debug("Listing of namespace [{}] resulted in the following tables: [{}]", namespace, tableIdentifiers);

Review comment:
       In Hive we often find that it could be important to know what is the exact information we got from downstream systems, so we can decide if we have to check downstream code to find the problem, or we have to check if we had a bug in Hive code.
   For the specific situation (made up example), it could be that HMS returned wrong list of tables because the namespace contained '%' character and HMS used as a wildcard instead of returning exact matches. This debug log could help find the root cause of the issue.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479495477



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java
##########
@@ -71,20 +74,24 @@ public HadoopTables(Configuration conf) {
    */
   @Override
   public Table load(String location) {
+    Table result;
     Pair<String, MetadataTableType> parsedMetadataType = parseMetadataType(location);
 
     if (parsedMetadataType != null) {
       // Load a metadata table
-      return loadMetadataTable(parsedMetadataType.first(), parsedMetadataType.second());
+      result = loadMetadataTable(parsedMetadataType.first(), parsedMetadataType.second());
     } else {
       // Load a normal table
       TableOperations ops = newTableOps(location);
       if (ops.current() != null) {
-        return new BaseTable(ops, location);
+        result = new BaseTable(ops, location);
       } else {
         throw new NoSuchTableException("Table does not exist at location: " + location);
       }
     }
+
+    LOG.info("Table loaded by HadoopTables: {}", result);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481377034



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +141,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Table is dropped [{}]", identifier);

Review comment:
       For concerns like this, I prefer to have a `quote` method that handles special characters. That way you get a readable error message when everything is normal, but the space problem is highlighted when it needs to be. So `ident.quote()` would return `db.table` normally and `db."table "` if there is a trailing space.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481351246



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -83,28 +86,32 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
               .impl(catalogLoaderClass)
               .build()
               .newInstance();
-      return Optional.of(loader.load(conf));
+      Catalog catalog = loader.load(conf);
+      LOG.info("Loaded catalog {} using {}", catalog, catalogLoaderClass);
+      return Optional.of(catalog);
     }
 
     String catalogName = conf.get(InputFormatConfig.CATALOG);
 
     if (catalogName != null) {
+      Catalog catalog;
       switch (catalogName.toLowerCase()) {
         case HADOOP:
           String warehouseLocation = conf.get(InputFormatConfig.HADOOP_CATALOG_WAREHOUSE_LOCATION);
 
-          if (warehouseLocation != null) {
-            return Optional.of(new HadoopCatalog(conf, warehouseLocation));
-          }
-
-          return Optional.of(new HadoopCatalog(conf));
+          catalog = (warehouseLocation != null) ? new HadoopCatalog(conf, warehouseLocation) : new HadoopCatalog(conf);
+          LOG.info("Loaded Hadoop catalog {}", catalog);
+          return Optional.of(catalog);
         case HIVE:
-          return Optional.of(HiveCatalogs.loadCatalog(conf));
+          catalog = HiveCatalogs.loadCatalog(conf);
+          LOG.info("Loaded Hive Metastore catalog {}", catalog);
+          return Optional.of(catalog);
         default:
           throw new NoSuchNamespaceException("Catalog " + catalogName + " is not supported.");
       }
     }
 
+    LOG.info("No catalog is used");

Review comment:
       Changed the text




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478760991



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -65,13 +68,17 @@ public static Table loadTable(Configuration conf, Properties props) {
   private static Table loadTable(Configuration conf, String tableIdentifier, String tableLocation) {
     Optional<Catalog> catalog = loadCatalog(conf);
 
+    Table table;
     if (catalog.isPresent()) {
       Preconditions.checkArgument(tableIdentifier != null, "Table identifier not set");
-      return catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+      table = catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+    } else {
+      Preconditions.checkArgument(tableLocation != null, "Table location not set");
+      table = new HadoopTables(conf).load(tableLocation);
     }
 
-    Preconditions.checkArgument(tableLocation != null, "Table location not set");
-    return new HadoopTables(conf).load(tableLocation);
+    LOG.info("Table loaded by catalog: [{}]", table);

Review comment:
       I think this should be done in the catalog or `HadoopTables`, not in `Catalogs`. Otherwise, we don't get the log message consistently when table loads go directly through the catalog (as in Spark 3) or through a different class like IcebergSource (Spark 2). And if we were to later add a message to the catalog, then this would be a duplicate.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -83,28 +90,33 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
               .impl(catalogLoaderClass)
               .build()
               .newInstance();
-      return Optional.of(loader.load(conf));
+      Catalog catalog = loader.load(conf);
+      LOG.info("Dynamic catalog is used: [{}]", catalog);

Review comment:
       I think it would be better to give more specific information:
   
   ```java
   LOG.info("Loaded catalog {} using {}", catalog, catalogLoaderClass);
   ```




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479214772



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(
           Arrays.copyOfRange(createStack, 1, createStack.length));
-      LOG.warn("Unclosed input stream created by:\n\t{}", trace);
+      LOG.warn("Unclosed input stream created by:\\n\t{}", trace);
     }
   }
+
+  @Override
+  public String toString() {
+    return "HiveCatalog [name=" + name + ", location=" + this.conf.get("hive.metastore.uris") + "]";

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479414455



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       @pvary, I'm not sure why, but I've never seen mixed log lines in our environment. So I don't think it is a good idea to solve the problem for one environment by making logs harder to read everywhere.
   
   What about using a custom appender or SLF4J binding in your environment that converts newlines to `\\n` so that you can configure it locally? I think that is a fairly easy solution that doesn't require changing logs everywhere.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479422415



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +184,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table from: {}, to: {}", from, to);

Review comment:
       No need for colons after from and to.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479416085



##########
File path: bundled-guava/src/main/java/org/apache/iceberg/GuavaClasses.java
##########
@@ -81,6 +82,7 @@
     Multimaps.class.getName();
     Ordering.class.getName();
     Sets.class.getName();
+    Stopwatch.class.getName();

Review comment:
       We are trying to avoid adding more Guava classes. Could you use `System.currentTimeMilis()` instead?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479495917



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +143,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Dropped table: {}", identifier);
       return true;
 
     } catch (NoSuchTableException | NoSuchObjectException e) {
+      LOG.debug("Skipping drop, table does not exist: {}", identifier, e);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479425205



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -83,28 +87,33 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
               .impl(catalogLoaderClass)
               .build()
               .newInstance();
-      return Optional.of(loader.load(conf));
+      Catalog catalog = loader.load(conf);
+      LOG.info("Loaded catalog {} using {}", catalog, catalogLoaderClass);
+      return Optional.of(catalog);
     }
 
     String catalogName = conf.get(InputFormatConfig.CATALOG);
 
     if (catalogName != null) {
+      Catalog catalog;
       switch (catalogName.toLowerCase()) {
         case HADOOP:
           String warehouseLocation = conf.get(InputFormatConfig.HADOOP_CATALOG_WAREHOUSE_LOCATION);
 
-          if (warehouseLocation != null) {
-            return Optional.of(new HadoopCatalog(conf, warehouseLocation));
-          }
-
-          return Optional.of(new HadoopCatalog(conf));
+          catalog = (warehouseLocation != null) ? new HadoopCatalog(conf, warehouseLocation) : new HadoopCatalog(conf);
+          break;
         case HIVE:
-          return Optional.of(HiveCatalogs.loadCatalog(conf));
+          catalog = HiveCatalogs.loadCatalog(conf);
+          break;
         default:
           throw new NoSuchNamespaceException("Catalog " + catalogName + " is not supported.");
       }
+
+      LOG.info("Catalog is used: {}", catalog);

Review comment:
       How about logging a similar message in the HADOOP and HIVE cases instead of this?
   
   ```
   LOG.info("Loaded Hive Metastore catalog {}", catalog);
   ```
   
   That gives more information about the path the code took.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479423732



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -296,7 +319,10 @@ public boolean removeProperties(Namespace namespace,  Set<String> properties) {
     properties.forEach(key -> parameter.put(key, null));
     Database database = convertToDatabase(namespace, parameter);
 
-    return alterHiveDataBase(namespace, database);
+    boolean result = alterHiveDataBase(namespace, database);
+    LOG.debug("Remove properties: {} from: {} resulted: {}", properties, namespace, result);

Review comment:
       Same here. I'd rather not add debug logs for properties.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479695372



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -300,17 +300,11 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    boolean result = alterHiveDataBase(namespace, database);
+    alterHiveDataBase(namespace, database);
+    LOG.debug("Successfully set properties for {}", namespace);

Review comment:
       This log doesn't seem useful to me since it doesn't have any information other than the method reached this point. It should be clear from other places (e.g., the SQL statement that ran) what was called. And we know it is successful or else an error is logged.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478760172



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -143,9 +144,11 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
       // TODO add lock heart beating for cases where default lock timeout is too low.
       Table tbl;
       if (base != null) {
+        LOG.info("Committing existing table [{}]", fullName);

Review comment:
       I think these changes are too verbose for INFO, although I do think it would be good to have a log message for the commit.
   
   Could you add an INFO log in `BaseMetastoreTableOperations#commit` instead of these? Something like this:
   
   ```java
   LOG.info("Successfully committed to table {} in {} ms: {}", table.name(), end - start, newMetadataLocation);
   ```
   
   That way, logging is not too verbose, consistent across implementations, and adds useful info: the time it took to commit.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479498902



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -329,7 +355,13 @@ private boolean alterHiveDataBase(Namespace namespace,  Database database) {
 
     try {
       Database database = clients.run(client -> client.getDatabase(namespace.level(0)));
-      return convertToMetadata(database);
+      Map<String, String> metadata = convertToMetadata(database);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("For namespace: {} loaded metadata: {}",
+            namespace,
+            Joiner.on(",").withKeyValueSeparator("=").join(metadata));
+      }

Review comment:
       Removed




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479492001



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -46,12 +48,18 @@
 
   private static final String METADATA_FOLDER_NAME = "metadata";
 
+  private String fullName;
   private TableMetadata currentMetadata = null;
   private String currentMetadataLocation = null;
   private boolean shouldRefresh = true;
   private int version = -1;
 
-  protected BaseMetastoreTableOperations() {
+  protected BaseMetastoreTableOperations(String fullName) {

Review comment:
       Absolutely!
   Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479489626



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       Ok. Keeping the reverted stuff




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479287979



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -83,28 +90,33 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
               .impl(catalogLoaderClass)
               .build()
               .newInstance();
-      return Optional.of(loader.load(conf));
+      Catalog catalog = loader.load(conf);
+      LOG.info("Dynamic catalog is used: [{}]", catalog);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479417990



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -46,12 +48,18 @@
 
   private static final String METADATA_FOLDER_NAME = "metadata";
 
+  private String fullName;
   private TableMetadata currentMetadata = null;
   private String currentMetadataLocation = null;
   private boolean shouldRefresh = true;
   private int version = -1;
 
-  protected BaseMetastoreTableOperations() {
+  protected BaseMetastoreTableOperations(String fullName) {

Review comment:
       This base class is extended by external projects, so I think we should avoid breaking changes like this one.
   
   I propose adding a `tableName` method with a base implementation that returns `null`. Hive can override the implementation to provide the table name (as could external implementations) and the log message can include the table name if it is non-null. Does that sound reasonable?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481380404



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -329,7 +350,9 @@ private boolean alterHiveDataBase(Namespace namespace,  Database database) {
 
     try {
       Database database = clients.run(client -> client.getDatabase(namespace.level(0)));
-      return convertToMetadata(database);
+      Map<String, String> metadata = convertToMetadata(database);
+      LOG.debug("Loaded metadata for namespace {}", namespace);

Review comment:
       And feel free to ask changes if they are needed. Better now, than 1 year from now when some info is leaked!




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478760362



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -130,6 +130,7 @@ protected void doRefresh() {
     }
 
     refreshFromMetadataLocation(metadataLocation);
+    LOG.debug("Refreshed [{}]", fullName);

Review comment:
       There is already a log in `BaseMetastoreTableOperations` for this. It has the location, but not the table name. Maybe just add table name to that one.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479424103



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -329,7 +355,13 @@ private boolean alterHiveDataBase(Namespace namespace,  Database database) {
 
     try {
       Database database = clients.run(client -> client.getDatabase(namespace.level(0)));
-      return convertToMetadata(database);
+      Map<String, String> metadata = convertToMetadata(database);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("For namespace: {} loaded metadata: {}",
+            namespace,
+            Joiner.on(",").withKeyValueSeparator("=").join(metadata));
+      }

Review comment:
       Same here. We should not be logging potentially sensitive properties.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478757687



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +141,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Table is dropped [{}]", identifier);
       return true;
 
     } catch (NoSuchTableException | NoSuchObjectException e) {
+      LOG.debug("Unable to drop missing table: [{}]", identifier, e);

Review comment:
       How about "Skipping drop, table does not exist: {}"?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-685474655


   Thanks @rdblue for all the effort put into reviewing this!


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-685079780


   @pvary, I found one more thing. Sorry for not finding it sooner.


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478757772



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +182,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table from: [{}], to: [{}]", from, to);

Review comment:
       Can you remove the brackers?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-683119588


   Thanks @rdblue for the reviews!
   Closed the conversations where I feel that we finished, left only 3 open where you might not agree with my solution.
   Thanks,
   Peter


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479419634



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -101,12 +109,25 @@ public void commit(TableMetadata base, TableMetadata metadata) {
       return;
     }
 
-    doCommit(base, metadata);
+    Stopwatch stopwatch = Stopwatch.createStarted();
+    String newMetadataLocation = doCommit(base, metadata);
     deleteRemovedMetadataFiles(base, metadata);
     requestRefresh();
+    stopwatch.stop();
+
+    LOG.info("Successfully committed to table: {} in: {} ms: {}",

Review comment:
       I don't think we need so many colons. How about `Successfully committed to table {} in {} ms`?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479420789



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java
##########
@@ -71,20 +74,24 @@ public HadoopTables(Configuration conf) {
    */
   @Override
   public Table load(String location) {
+    Table result;
     Pair<String, MetadataTableType> parsedMetadataType = parseMetadataType(location);
 
     if (parsedMetadataType != null) {
       // Load a metadata table
-      return loadMetadataTable(parsedMetadataType.first(), parsedMetadataType.second());
+      result = loadMetadataTable(parsedMetadataType.first(), parsedMetadataType.second());
     } else {
       // Load a normal table
       TableOperations ops = newTableOps(location);
       if (ops.current() != null) {
-        return new BaseTable(ops, location);
+        result = new BaseTable(ops, location);
       } else {
         throw new NoSuchTableException("Table does not exist at location: " + location);
       }
     }
+
+    LOG.info("Table loaded by HadoopTables: {}", result);

Review comment:
       `HadoopTables` is logging this message, so there is no need to add it here. Also, can we log the location instead of the table instance?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479421721



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -89,10 +91,13 @@ public HiveCatalog(String name, String uri, int clientPoolSize, Configuration co
 
     try {
       List<String> tables = clients.run(client -> client.getAllTables(database));
-      return tables.stream()
+      List<TableIdentifier> tableIdentifiers = tables.stream()
           .map(t -> TableIdentifier.of(namespace, t))
           .collect(Collectors.toList());
 
+      LOG.debug("Listing of namespace: {} resulted in the following tables: {}", namespace, tableIdentifiers);

Review comment:
       Let's clean up the message a bit: "Listing namespace {} returned tables: {}"




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478758210



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       This makes logs harder to read. What is the intended purpose of this change?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479499416



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -68,10 +71,11 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
     if (catalog.isPresent()) {
       Preconditions.checkArgument(tableIdentifier != null, "Table identifier not set");
       return catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+    } else {
+      Preconditions.checkArgument(tableLocation != null, "Table location not set");
+      return new HadoopTables(conf).load(tableLocation);

Review comment:
       I missed this one. Sorry :(
   Reverted




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479422313



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -138,9 +143,11 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) {
         dropTableData(ops.io(), lastMetadata);
       }
 
+      LOG.info("Dropped table: {}", identifier);
       return true;
 
     } catch (NoSuchTableException | NoSuchObjectException e) {
+      LOG.debug("Skipping drop, table does not exist: {}", identifier, e);

Review comment:
       Should this be info?
   
   I doubt that this is much help if you have to turn debug on and re-run because whether the table exists is external and modified by this method.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479424419



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -68,10 +71,11 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
     if (catalog.isPresent()) {
       Preconditions.checkArgument(tableIdentifier != null, "Table identifier not set");
       return catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+    } else {
+      Preconditions.checkArgument(tableLocation != null, "Table location not set");
+      return new HadoopTables(conf).load(tableLocation);

Review comment:
       Looks like this change is not needed. Could you roll it back?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479607714



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -296,7 +319,10 @@ public boolean removeProperties(Namespace namespace,  Set<String> properties) {
     properties.forEach(key -> parameter.put(key, null));
     Database database = convertToDatabase(namespace, parameter);
 
-    return alterHiveDataBase(namespace, database);
+    boolean result = alterHiveDataBase(namespace, database);
+    LOG.debug("Remove properties: {} from: {} resulted: {}", properties, namespace, result);

Review comment:
       Removed the alterHiveDataBase method return value, since it is a private function and as you correctly pointed out never returns false.
   Tried to find the usages of the HiveCatalog.setProperties/removeProperties method, and followed the call stack a few steps, but did not find any logs around that. (I might missed something)




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#issuecomment-683380801


   > Looks like the current issue is a checkstyle failure:
   > 
   > ```
   > > Task :iceberg-hive:checkstyleMain FAILED
   > [ant:checkstyle] [ERROR] /home/travis/build/apache/iceberg/hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:249: Trailing whitespace is not allowed. [RegexpSinglelineJava]
   > [ant:checkstyle] [ERROR] /home/travis/build/apache/iceberg/hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:249: Whitespace at end-of-line [RegexpSingleline]
   > ```
   
   Yeah, introduced with the new line I put it to retrigger the tests (was so sure about putting in that line, but IntelliJ made fun of me :))
   .
   The failure before were these, and only on Java11:
   ```
   org.apache.iceberg.spark.actions.TestExpireSnapshotsAction3 > dataFilesCleanupWithParallelTasks FAILED
       java.lang.AssertionError: FILE_B should be deleted
   
   org.apache.iceberg.spark.sql.TestCreateTableAsSelect > testUnpartitionedCTAS[1] FAILED
       java.lang.AssertionError: Should have rows matching the source table: number of results should match expected:<12> but was:<15>
           at org.junit.Assert.fail(Assert.java:88)
           at org.junit.Assert.failNotEquals(Assert.java:834)
           at org.junit.Assert.assertEquals(Assert.java:645)
           at org.apache.iceberg.spark.SparkTestBase.assertEquals(SparkTestBase.java:100)
           at org.apache.iceberg.spark.sql.TestCreateTableAsSelect.testUnpartitionedCTAS(TestCreateTableAsSelect.java:70)
   ```


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394


   


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481351035



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
##########
@@ -54,6 +54,10 @@
   protected BaseMetastoreTableOperations() {
   }
 
+  protected String fullName() {

Review comment:
       Renamed and added javadoc.
   Could you please check them?
   
   Thanks, Peter




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r481375297



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -329,7 +350,9 @@ private boolean alterHiveDataBase(Namespace namespace,  Database database) {
 
     try {
       Database database = clients.run(client -> client.getDatabase(namespace.level(0)));
-      return convertToMetadata(database);
+      Map<String, String> metadata = convertToMetadata(database);
+      LOG.debug("Loaded metadata for namespace {}", namespace);

Review comment:
       Sorry, but I didn't catch this before. This is logging a property map that could contain sensitive items. Could you log just the keys instead?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479728173



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -300,17 +300,11 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    boolean result = alterHiveDataBase(namespace, database);
+    alterHiveDataBase(namespace, database);
+    LOG.debug("Successfully set properties for {}", namespace);

Review comment:
       First I removed the logs, but then thought better of it.
   Added it back with the list of the name of the properties we decided to set.
   This way we will at least know how the SparkCatalog.alterNamespace classified the properties, and which one was removed which one was set instead.
   Do you think it is worthwhile to log this way?
   I kinda feel that I have dragged on too much on your patience, so one last request and I will remove both log lines from the code.
   Thanks again!
   Peter




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479499416



##########
File path: mr/src/main/java/org/apache/iceberg/mr/Catalogs.java
##########
@@ -68,10 +71,11 @@ private static Table loadTable(Configuration conf, String tableIdentifier, Strin
     if (catalog.isPresent()) {
       Preconditions.checkArgument(tableIdentifier != null, "Table identifier not set");
       return catalog.get().loadTable(TableIdentifier.parse(tableIdentifier));
+    } else {
+      Preconditions.checkArgument(tableLocation != null, "Table location not set");
+      return new HadoopTables(conf).load(tableLocation);

Review comment:
       I missed this one. Sorry :(




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479202609



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +182,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table from: [{}], to: [{}]", from, to);

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r480483722



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -300,17 +300,11 @@ public boolean setProperties(Namespace namespace,  Map<String, String> propertie
     parameter.putAll(properties);
     Database database = convertToDatabase(namespace, parameter);
 
-    boolean result = alterHiveDataBase(namespace, database);
+    alterHiveDataBase(namespace, database);
+    LOG.debug("Successfully set properties for {}", namespace);

Review comment:
       > This way we will at least know how the SparkCatalog.alterNamespace/FlinkCatalog.alterNamespace classified the properties, and which one was removed which one was set instead.
   
   I don't think I follow your logic here. All this says is that `setProperties` succeeded. But that's usually obvious from the job because the SQL that's running is `ALTER TABLE ... SET TBLPROPERTIES` or `ALTER TABLE ... UNSET TBLPROPERTIES`. So it seems to me that this is already clear from context.
   
   > I kinda feel that I have dragged on too much on your patience, so one last request and I will remove both log lines from the code.
   
   Don't worry about it! I had a similar thought because I'm asking for so many logs to be removed or changed. Sounds like we're both happy to come to an agreement and improve logging, even if that takes a lot of discussion. Thanks for your patience here.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r479191540



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -465,9 +495,14 @@ protected void finalize() throws Throwable {
     super.finalize();
     if (!closed) {
       close(); // releasing resources is more important than printing the warning
-      String trace = Joiner.on("\n\t").join(
+      String trace = Joiner.on("\\n\t").join(

Review comment:
       When I have to read Hive logs where there are plenty of concurrent queries running parallel, extra log lines will appear inside if we have multiline logs which will make them hard to read (even harder than the one proposed above).
   Also only the first line of the log will get the timestamp and other log header information, which will make it hard to use any log analyzer tool to check them.
   
   These are the main reasons in Hive we usually try to avoid multiline logs, and accept the drawback of the "hardness" of reading them.
   I am open to suggestions, even reverting back to this change if you strongly feel that it should remain as it is.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1394: Hive: Add logging for Hive related classes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1394:
URL: https://github.com/apache/iceberg/pull/1394#discussion_r478757845



##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -177,6 +182,8 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
         return null;
       });
 
+      LOG.info("Renamed table from: [{}], to: [{}]", from, to);

Review comment:
       Here and in other messages




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org