You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by "homatthew (via GitHub)" <gi...@apache.org> on 2023/02/01 05:04:49 UTC

[GitHub] [gobblin] homatthew opened a new pull request, #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

homatthew opened a new pull request, #3633:
URL: https://github.com/apache/gobblin/pull/3633

   - drop_file GMCE should not try to create a table
   - issues creating the table will not throw an exception (i.e. schema issues should be skipped over)
   
   Dear Gobblin maintainers,
   
   Please accept this PR. I understand that it will not be reviewed until I have checked off all the steps below!
   
   
   ### JIRA
   - [ ] My PR addresses the following [Gobblin JIRA](https://issues.apache.org/jira/browse/GOBBLIN/) issues and references them in the PR title. For example, "[GOBBLIN-XXX] My Gobblin PR"
       - https://issues.apache.org/jira/browse/GOBBLIN-XXX
   
   
   ### Description
   - [ ] Here are some details about my PR, including screenshots (if applicable):
   
   
   ### Tests
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   
   ### Commits
   - [ ] My commits all reference JIRA issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
       1. Subject is separated from body by a blank line
       2. Subject is limited to 50 characters
       3. Subject does not end with a period
       4. Subject uses the imperative mood ("add", not "adding")
       5. Body wraps at 72 characters
       6. Body explains "what" and "why", not "how"
   
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] Will-Lo commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "Will-Lo (via GitHub)" <gi...@apache.org>.
Will-Lo commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1105141078


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -281,15 +281,24 @@ private boolean inHiveSpecCache(String tableKey) {
   }
 
   private void updateLatestSchemaMapWithExistingSchema(String dbName, String tableName, String tableKey) throws IOException {
+    updateLatestSchemaMapWithExistingSchema(dbName, tableName, tableKey, useExistingTableSchemaAllowDenyList, hiveRegister, latestSchemaMap);
+  }
+
+  // returns if latest schema map was updated with the existing schema in Hive

Review Comment:
   This should be a javadoc



##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -281,15 +281,24 @@ private boolean inHiveSpecCache(String tableKey) {
   }
 
   private void updateLatestSchemaMapWithExistingSchema(String dbName, String tableName, String tableKey) throws IOException {
+    updateLatestSchemaMapWithExistingSchema(dbName, tableName, tableKey, useExistingTableSchemaAllowDenyList, hiveRegister, latestSchemaMap);
+  }
+
+  // returns if latest schema map was updated with the existing schema in Hive
+  @VisibleForTesting
+  protected static boolean updateLatestSchemaMapWithExistingSchema(String dbName, String tableName, String tableKey,

Review Comment:
   Is there a reason why this function returns a boolean if we just ignore the return value in the wrapper function? Is it just for ease of testing purposes?



##########
gobblin-iceberg/src/test/java/org/apache/gobblin/iceberg/writer/HiveMetadataWriterTest.java:
##########
@@ -80,6 +82,10 @@
 import org.apache.gobblin.stream.RecordEnvelope;
 import org.apache.gobblin.util.ClustersNames;
 import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.function.CheckedExceptionFunction;
+import org.mockito.Mockito;
+
+import static org.mockito.Matchers.eq;
 
 
 public class HiveMetadataWriterTest extends HiveMetastoreTest {

Review Comment:
   Is there a reason why we can't move this class to the hive module? Is it a dependency issue? To me it seems like one of these classes should ultimately live in the other's module but I might be overlooking something 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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] jack-moseley commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "jack-moseley (via GitHub)" <gi...@apache.org>.
jack-moseley commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1105139677


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -281,15 +281,24 @@ private boolean inHiveSpecCache(String tableKey) {
   }
 
   private void updateLatestSchemaMapWithExistingSchema(String dbName, String tableName, String tableKey) throws IOException {
+    updateLatestSchemaMapWithExistingSchema(dbName, tableName, tableKey, useExistingTableSchemaAllowDenyList, hiveRegister, latestSchemaMap);
+  }
+
+  // returns if latest schema map was updated with the existing schema in Hive

Review Comment:
   Meant to say "returns true if"?



##########
gobblin-iceberg/src/test/java/org/apache/gobblin/iceberg/writer/HiveMetadataWriterTest.java:
##########
@@ -350,6 +403,34 @@ private String writeRecord(File file) throws IOException {
     return path;
   }
 
+  /**
+   * Test class for exposing methods for exposing internal HiveMetaWriter functions without making them public.

Review Comment:
   Typo
   
   "Test class ~~for exposing methods~~ for exposing internal HiveMetaWriter functions"



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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] ZihanLi58 commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "ZihanLi58 (via GitHub)" <gi...@apache.org>.
ZihanLi58 commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1100604690


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +229,69 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. i.e. IOExceptions when creating the table
+   * are swallowed and logged to error
+   * @param tableSpec
+   * @param tableKey table key used to check if table is in spec cache
+   * @return if the table the table was created. If the table existed beforehand, it still returns true.
+   */
+  private boolean createTable(HiveSpec tableSpec, String tableKey) {
+    try {
+      // no-op if it's in spec cache (spec cache contains tablekey for all db / tables created since last flush)
+      if (inHiveSpecCache(tableKey)) {
+        return true;
+      }
+
+      this.hiveRegister.createTableIfNotExists(tableSpec.getTable());
+      return true;
+    } catch (IOException e) {
+      log.error("Failed to create table. Skipping this event", e);
+      return false;
+    }
+  }
+
+  @Nullable
+  private String getTopicName(GobblinMetadataChangeEvent gmce) {
+    //Calculate the topic name from gmce, fall back to topic.name in hive spec which can also be null
+    //todo: make topicName fall back to topic.name in hive spec so that we can also get schema for re-write operation
+    String topicName = null;
+    if (gmce.getTopicPartitionOffsetsRange() != null && !gmce.getTopicPartitionOffsetsRange().isEmpty()) {
+      String topicPartitionString = gmce.getTopicPartitionOffsetsRange().keySet().iterator().next();
+      //In case the topic name is not the table name or the topic name contains '-'
+      topicName = topicPartitionString.substring(0, topicPartitionString.lastIndexOf('-'));
+    }
+    return topicName;
+  }
+
+  /**
+   * We care about if a table key is in the spec cache because it means that we have already created this table before
+   * since the last flush. Therefore, we can use this method to check whether we need to create a table
+   * @param tableKey
+   * @return
+   */
+  private boolean inHiveSpecCache(String tableKey) {
+    return specMaps.containsKey(tableKey) && specMaps.get(tableKey).size() > 0;
+  }
+
+  private void updateLatestSchemaMap(String dbName, String tableName, String tableKey) throws IOException {
+    //ToDo: after making sure all spec has topic.name set, we should use topicName as key for schema
+    boolean alwaysUseLatestSchema = useLatestTableSchemaAllowDenyList.acceptTable(dbName, tableName);

Review Comment:
   I think to make it more clear, we should set the name to be alwaysUseExistingSchema, and I would suggest you to change the allowDenyList accordingly as you touch it now



##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +229,69 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. i.e. IOExceptions when creating the table
+   * are swallowed and logged to error
+   * @param tableSpec
+   * @param tableKey table key used to check if table is in spec cache
+   * @return if the table the table was created. If the table existed beforehand, it still returns true.
+   */
+  private boolean createTable(HiveSpec tableSpec, String tableKey) {
+    try {
+      // no-op if it's in spec cache (spec cache contains tablekey for all db / tables created since last flush)
+      if (inHiveSpecCache(tableKey)) {
+        return true;
+      }
+
+      this.hiveRegister.createTableIfNotExists(tableSpec.getTable());
+      return true;
+    } catch (IOException e) {
+      log.error("Failed to create table. Skipping this event", e);
+      return false;
+    }
+  }
+
+  @Nullable
+  private String getTopicName(GobblinMetadataChangeEvent gmce) {
+    //Calculate the topic name from gmce, fall back to topic.name in hive spec which can also be null
+    //todo: make topicName fall back to topic.name in hive spec so that we can also get schema for re-write operation
+    String topicName = null;
+    if (gmce.getTopicPartitionOffsetsRange() != null && !gmce.getTopicPartitionOffsetsRange().isEmpty()) {
+      String topicPartitionString = gmce.getTopicPartitionOffsetsRange().keySet().iterator().next();
+      //In case the topic name is not the table name or the topic name contains '-'
+      topicName = topicPartitionString.substring(0, topicPartitionString.lastIndexOf('-'));
+    }
+    return topicName;
+  }
+
+  /**
+   * We care about if a table key is in the spec cache because it means that we have already created this table before
+   * since the last flush. Therefore, we can use this method to check whether we need to create a table
+   * @param tableKey
+   * @return
+   */
+  private boolean inHiveSpecCache(String tableKey) {
+    return specMaps.containsKey(tableKey) && specMaps.get(tableKey).size() > 0;
+  }
+
+  private void updateLatestSchemaMap(String dbName, String tableName, String tableKey) throws IOException {

Review Comment:
   Change this to be updateLatestSchemaMapWithExistingSchema



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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] homatthew commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "homatthew (via GitHub)" <gi...@apache.org>.
homatthew commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1105154240


##########
gobblin-iceberg/src/test/java/org/apache/gobblin/iceberg/writer/HiveMetadataWriterTest.java:
##########
@@ -80,6 +82,10 @@
 import org.apache.gobblin.stream.RecordEnvelope;
 import org.apache.gobblin.util.ClustersNames;
 import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.function.CheckedExceptionFunction;
+import org.mockito.Mockito;
+
+import static org.mockito.Matchers.eq;
 
 
 public class HiveMetadataWriterTest extends HiveMetastoreTest {

Review Comment:
   The writer lives in in gobblin-hive-registration. And the test lives in gobblin iceberg.
   
   Your suspicion is correct. The problem is that this test relies on some hive testing utilities in the iceberg module. It's a bit of a mess to move it there and the benefit is not worth it



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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] homatthew commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "homatthew (via GitHub)" <gi...@apache.org>.
homatthew commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1093517280


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +227,63 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. Existence is not an error.
+   * IOExceptions when creating the table are swallowed and logged to error
+   * @param tableSpec
+   * @return true if the table the table was created without error. Existence of the table is not an error
+   */
+  private boolean createTable(HiveSpec tableSpec) {
+    try {
+      this.hiveRegister.createTableIfNotExists(tableSpec.getTable());
+      return true;
+    } catch (IOException e) {
+      log.error("Failed to create table. Skipping this event", e);
+      return false;
+    }
+  }
+
+  @Nullable
+  private String getTopicName(GobblinMetadataChangeEvent gmce) {
+    //Calculate the topic name from gmce, fall back to topic.name in hive spec which can also be null
+    //todo: make topicName fall back to topic.name in hive spec so that we can also get schema for re-write operation

Review Comment:
   pre existing comments / TODOS



##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +227,63 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. Existence is not an error.
+   * IOExceptions when creating the table are swallowed and logged to error
+   * @param tableSpec
+   * @return true if the table the table was created without error. Existence of the table is not an error
+   */
+  private boolean createTable(HiveSpec tableSpec) {
+    try {
+      this.hiveRegister.createTableIfNotExists(tableSpec.getTable());
+      return true;
+    } catch (IOException e) {
+      log.error("Failed to create table. Skipping this event", e);
+      return false;
+    }
+  }
+
+  @Nullable
+  private String getTopicName(GobblinMetadataChangeEvent gmce) {
+    //Calculate the topic name from gmce, fall back to topic.name in hive spec which can also be null
+    //todo: make topicName fall back to topic.name in hive spec so that we can also get schema for re-write operation
+    String topicName = null;
+    if (gmce.getTopicPartitionOffsetsRange() != null && !gmce.getTopicPartitionOffsetsRange().isEmpty()) {
+      String topicPartitionString = gmce.getTopicPartitionOffsetsRange().keySet().iterator().next();
+      //In case the topic name is not the table name or the topic name contains '-'
+      topicName = topicPartitionString.substring(0, topicPartitionString.lastIndexOf('-'));
+    }
+    return topicName;
+  }
+
+  /**
+   * We care about if a table key is in the spec cache because it means that we have already created this table before
+   * since the last flush. Therefore, we can use this method to check whether we need to create a table
+   * @param tableKey
+   * @return
+   */
+  private boolean inHiveSpecCache(String tableKey) {
+    return specMaps.containsKey(tableKey) && specMaps.get(tableKey).size() > 0;
+  }
+
+  private void updateLatestSchemaMap(String dbName, String tableName, String tableKey) throws IOException {
+    //ToDo: after making sure all spec has topic.name set, we should use topicName as key for schema

Review Comment:
   pre existing comments / TODOS



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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] jack-moseley commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "jack-moseley (via GitHub)" <gi...@apache.org>.
jack-moseley commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1099158421


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +222,69 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. Existence is not an error.

Review Comment:
   Ah okay, yeah I think that wording is clear, 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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] jack-moseley commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "jack-moseley (via GitHub)" <gi...@apache.org>.
jack-moseley commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1098074596


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +222,69 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. Existence is not an error.

Review Comment:
   Maybe "Non-existence is not an error" is a bit more accurate.



##########
gobblin-iceberg/src/test/java/org/apache/gobblin/iceberg/writer/HiveMetadataWriterTest.java:
##########
@@ -256,6 +258,54 @@ public void testHiveWriteRewriteFileGMCE() throws IOException {
     });
   }
 
+  @Test(dependsOnMethods = {"testHiveWriteRewriteFileGMCE"}, groups={"hiveMetadataWriterTest"})
+  public void testHiveWriteDeleteFileGMCE() throws IOException, TException {
+    gmce.setTopicPartitionOffsetsRange(null);
+    Map<String, String> registrationState = gmce.getRegistrationProperties();
+    registrationState.put("additional.hive.database.names", dedupedDbName);
+    registrationState.put(HiveMetaStoreBasedRegister.SCHEMA_SOURCE_DB, dbName);
+    gmce.setRegistrationProperties(registrationState);
+    gmce.setSchemaSource(SchemaSource.NONE);
+    gmce.setOldFilePrefixes(Lists.newArrayList(dailyDataFile.toString()));
+    gmce.setOperationType(OperationType.drop_files);
+    gobblinMCEWriter.writeEnvelope(new RecordEnvelope<>(gmce,

Review Comment:
   Could you add a couple more comments in the tests just to make it easy to understand what is being tested, here are we testing dropping a partition that doesn't 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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] homatthew commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "homatthew (via GitHub)" <gi...@apache.org>.
homatthew commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1105147057


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -281,15 +281,24 @@ private boolean inHiveSpecCache(String tableKey) {
   }
 
   private void updateLatestSchemaMapWithExistingSchema(String dbName, String tableName, String tableKey) throws IOException {
+    updateLatestSchemaMapWithExistingSchema(dbName, tableName, tableKey, useExistingTableSchemaAllowDenyList, hiveRegister, latestSchemaMap);
+  }
+
+  // returns if latest schema map was updated with the existing schema in Hive
+  @VisibleForTesting
+  protected static boolean updateLatestSchemaMapWithExistingSchema(String dbName, String tableName, String tableKey,

Review Comment:
   Yeah just for testing purposes.



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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] Will-Lo merged pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "Will-Lo (via GitHub)" <gi...@apache.org>.
Will-Lo merged PR #3633:
URL: https://github.com/apache/gobblin/pull/3633


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] homatthew commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "homatthew (via GitHub)" <gi...@apache.org>.
homatthew commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1098109140


##########
gobblin-hive-registration/src/main/java/org/apache/gobblin/hive/writer/HiveMetadataWriter.java:
##########
@@ -232,12 +222,69 @@ public void write(GobblinMetadataChangeEvent gmce, Map<String, Collection<HiveSp
         break;
       }
       default: {
-        log.error("unsupported operation {}", gmce.getOperationType().toString());
+        log.error("unsupported operation {}", opType);
         return;
       }
     }
   }
 
+  /**
+   * Helper function to gracefully handle errors when creating a hive table. Existence is not an error.

Review Comment:
   Maybe the wording here is confusing. What I am trying to convey is that Existence of the table when making this call still returns true. What do you think of the following wording instead?
   
   > If the table existed before calling this method, the method will still return true
   
   The reason I even chose to add this wording is because the underlying call is `createTableIfNotExists`, which returns false if the table already existed.
   ```
     /**
      * Create a Hive table if not exists.
      *
      * @param table a {@link HiveTable} to be created.
      * @return true if the table is successfully created; false if the table already exists.
      * @throws IOException
      */
     public abstract boolean createTableIfNotExists(HiveTable table)
         throws IOException;
   ``` 



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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] codecov-commenter commented on pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#issuecomment-1423678058

   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3633](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e8716af) into [master](https://codecov.io/gh/apache/gobblin/commit/667d79787ae2f5ca2a22f5d023a6e89a00874835?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (667d797) will **increase** coverage by `0.01%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3633      +/-   ##
   ============================================
   + Coverage     46.56%   46.58%   +0.01%     
   - Complexity    10666    10680      +14     
   ============================================
     Files          2133     2133              
     Lines         83541    83589      +48     
     Branches       9288     9295       +7     
   ============================================
   + Hits          38905    38936      +31     
   - Misses        41074    41087      +13     
   - Partials       3562     3566       +4     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/gobblin/hive/writer/HiveMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1oaXZlLXJlZ2lzdHJhdGlvbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvZ29iYmxpbi9oaXZlL3dyaXRlci9IaXZlTWV0YWRhdGFXcml0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...che/gobblin/data/management/copy/CopyableFile.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1kYXRhLW1hbmFnZW1lbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vZGF0YS9tYW5hZ2VtZW50L2NvcHkvQ29weWFibGVGaWxlLmphdmE=) | `73.33% <0.00%> (-2.01%)` | :arrow_down: |
   | [.../apache/gobblin/runtime/api/JobExecutionState.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvYXBpL0pvYkV4ZWN1dGlvblN0YXRlLmphdmE=) | `79.43% <0.00%> (-0.94%)` | :arrow_down: |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `62.22% <0.00%> (-0.31%)` | :arrow_down: |
   | [...blin/service/monitoring/KafkaJobStatusMonitor.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3NlcnZpY2UvbW9uaXRvcmluZy9LYWZrYUpvYlN0YXR1c01vbml0b3IuamF2YQ==) | `45.98% <0.00%> (-0.29%)` | :arrow_down: |
   | [...lin/service/monitoring/SpecStoreChangeMonitor.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3NlcnZpY2UvbW9uaXRvcmluZy9TcGVjU3RvcmVDaGFuZ2VNb25pdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...ervice/monitoring/DagActionStoreChangeMonitor.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3NlcnZpY2UvbW9uaXRvcmluZy9EYWdBY3Rpb25TdG9yZUNoYW5nZU1vbml0b3IuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...lin/data/management/copy/ManifestBasedDataset.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1kYXRhLW1hbmFnZW1lbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vZGF0YS9tYW5hZ2VtZW50L2NvcHkvTWFuaWZlc3RCYXNlZERhdGFzZXQuamF2YQ==) | `51.78% <0.00%> (ø)` | |
   | [...bblin/data/management/copy/PreserveAttributes.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1kYXRhLW1hbmFnZW1lbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vZGF0YS9tYW5hZ2VtZW50L2NvcHkvUHJlc2VydmVBdHRyaWJ1dGVzLmphdmE=) | `97.36% <0.00%> (+0.07%)` | :arrow_up: |
   | [...rvice/modules/orchestration/DagManagerMetrics.java](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3NlcnZpY2UvbW9kdWxlcy9vcmNoZXN0cmF0aW9uL0RhZ01hbmFnZXJNZXRyaWNzLmphdmE=) | `87.25% <0.00%> (+0.12%)` | :arrow_up: |
   | ... and [7 more](https://codecov.io/gh/apache/gobblin/pull/3633?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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


[GitHub] [gobblin] homatthew commented on a diff in pull request #3633: [GOBBLIN-1775] GMIP Hive metadatawriter should gracefully fail

Posted by "homatthew (via GitHub)" <gi...@apache.org>.
homatthew commented on code in PR #3633:
URL: https://github.com/apache/gobblin/pull/3633#discussion_r1105162271


##########
gobblin-iceberg/src/test/java/org/apache/gobblin/iceberg/writer/HiveMetadataWriterTest.java:
##########
@@ -374,6 +374,7 @@ public void  testUpdateLatestSchemaWithExistingSchema() throws IOException {
 
 
     // Tables part of deny list, schema only fetched from hive on the first time and the all future calls will use the cache
+    Assert.assertTrue(updateLatestSchema.apply(tableNameDenied));

Review Comment:
   Fat finger removed this line on accident, causing the UT to fail in previous 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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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