-
Notifications
You must be signed in to change notification settings - Fork 748
[GOBBLIN-1960] Emit audit count after commit in IcebergMetadataWriter #3833
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -740,7 +740,7 @@ private Set<DataFile> getIcebergDataFilesToBeAdded(Table table, TableMetadata ta | |
| } | ||
| dataFiles.add(IcebergUtils.getIcebergDataFileWithMetric(file, table.spec(), partition, conf, schemaIdMap)); | ||
| } catch (Exception e) { | ||
| log.warn("Cannot get DataFile for {} dur to {}", file.getFilePath(), e); | ||
| log.warn("Cannot get DataFile for {} due to {}", file.getFilePath(), e); | ||
| } | ||
| } | ||
| return dataFiles; | ||
|
|
@@ -834,86 +834,122 @@ protected String getTopicName(TableIdentifier tid, TableMetadata tableMetadata) | |
| public void flush(String dbName, String tableName) throws IOException { | ||
| Lock writeLock = readWriteLock.writeLock(); | ||
| writeLock.lock(); | ||
| boolean transactionCommitted = false; | ||
| try { | ||
| TableIdentifier tid = TableIdentifier.of(dbName, tableName); | ||
| TableMetadata tableMetadata = tableMetadataMap.getOrDefault(tid, new TableMetadata(this.conf)); | ||
| if (tableMetadata.transaction.isPresent()) { | ||
| Transaction transaction = tableMetadata.transaction.get(); | ||
| Map<String, String> props = tableMetadata.newProperties.or( | ||
| Maps.newHashMap(tableMetadata.lastProperties.or(getIcebergTable(tid).properties()))); | ||
| //Set data offset range | ||
| setDatasetOffsetRange(tableMetadata, props); | ||
| String topicName = getTopicName(tid, tableMetadata); | ||
| if (tableMetadata.appendFiles.isPresent()) { | ||
| tableMetadata.appendFiles.get().commit(); | ||
| try (Timer.Context context = new Timer().time()) { | ||
| sendAuditCounts(topicName, tableMetadata.serializedAuditCountMaps); | ||
| log.info("Sending audit counts for {} took {} ms", topicName, TimeUnit.NANOSECONDS.toMillis(context.stop())); | ||
| } | ||
| if (tableMetadata.completenessEnabled) { | ||
| updateWatermarkWithFilesRegistered(topicName, tableMetadata, props, | ||
| tableMetadata.totalCountCompletenessEnabled); | ||
| } | ||
| } | ||
| if (tableMetadata.deleteFiles.isPresent()) { | ||
| tableMetadata.deleteFiles.get().commit(); | ||
| } | ||
| // Check and update completion watermark when there are no files to be registered, typically for quiet topics | ||
| // The logic is to check the window [currentHour-1,currentHour] and update the watermark if there are no audit counts | ||
| if(!tableMetadata.appendFiles.isPresent() && !tableMetadata.deleteFiles.isPresent() | ||
| && tableMetadata.completenessEnabled) { | ||
| updateWatermarkWithNoFilesRegistered(topicName, tableMetadata, props, | ||
| tableMetadata.totalCountCompletenessEnabled); | ||
| } | ||
| if (!tableMetadata.transaction.isPresent()) { | ||
| log.info("There's no transaction initiated for the table {}", tid); | ||
| return; | ||
| } | ||
|
|
||
| //Set high waterMark | ||
| Long highWatermark = tableCurrentWatermarkMap.get(tid); | ||
| props.put(String.format(GMCE_HIGH_WATERMARK_KEY, tableTopicPartitionMap.get(tid)), highWatermark.toString()); | ||
| //Set low waterMark | ||
| props.put(String.format(GMCE_LOW_WATERMARK_KEY, tableTopicPartitionMap.get(tid)), | ||
| tableMetadata.lowWatermark.get().toString()); | ||
| //Set whether to delete metadata files after commit | ||
| if (conf.getBoolean(ICEBERG_ENABLE_CUSTOM_METADATA_RETENTION_POLICY, DEFAULT_ICEBERG_ENABLE_CUSTOM_METADATA_RETENTION_POLICY)) { | ||
| props.put(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, Boolean.toString( | ||
| conf.getBoolean(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT))); | ||
| props.put(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, Integer.toString( | ||
| conf.getInt(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, TableProperties.METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT))); | ||
| } | ||
| //Update schema(commit) | ||
| updateSchema(tableMetadata, props, topicName); | ||
| //Update properties | ||
| UpdateProperties updateProperties = transaction.updateProperties(); | ||
| props.forEach(updateProperties::set); | ||
| updateProperties.commit(); | ||
| try (AutoCloseableHiveLock lock = this.locks.getTableLock(dbName, tableName); | ||
| Timer.Context context = new Timer().time()) { | ||
| transaction.commitTransaction(); | ||
| log.info("Committing transaction for table {} took {} ms", tid, TimeUnit.NANOSECONDS.toMillis(context.stop())); | ||
| Transaction transaction = tableMetadata.transaction.get(); | ||
| Map<String, String> props = tableMetadata.newProperties.or( | ||
| Maps.newHashMap(tableMetadata.lastProperties.or(getIcebergTable(tid).properties()))); | ||
| //Set data offset range | ||
| setDatasetOffsetRange(tableMetadata, props); | ||
| String topicName = getTopicName(tid, tableMetadata); | ||
| if (tableMetadata.appendFiles.isPresent()) { | ||
| tableMetadata.appendFiles.get().commit(); | ||
| if (tableMetadata.completenessEnabled) { | ||
| updateWatermarkWithFilesRegistered(topicName, tableMetadata, props, | ||
| tableMetadata.totalCountCompletenessEnabled); | ||
| } | ||
| } | ||
|
|
||
| // Emit GTE for snapshot commits | ||
| Snapshot snapshot = tableMetadata.table.get().currentSnapshot(); | ||
| Map<String, String> currentProps = tableMetadata.table.get().properties(); | ||
| try (Timer.Context context = new Timer().time()) { | ||
| submitSnapshotCommitEvent(snapshot, tableMetadata, dbName, tableName, currentProps, highWatermark); | ||
| log.info("Sending snapshot commit event for {} took {} ms", topicName, TimeUnit.NANOSECONDS.toMillis(context.stop())); | ||
| } | ||
| if (tableMetadata.deleteFiles.isPresent()) { | ||
| tableMetadata.deleteFiles.get().commit(); | ||
| } | ||
| // Check and update completion watermark when there are no files to be registered, typically for quiet topics | ||
| // The logic is to check the window [currentHour-1,currentHour] and update the watermark if there are no audit counts | ||
| if(!tableMetadata.appendFiles.isPresent() && !tableMetadata.deleteFiles.isPresent() | ||
| && tableMetadata.completenessEnabled) { | ||
| updateWatermarkWithNoFilesRegistered(topicName, tableMetadata, props, | ||
| tableMetadata.totalCountCompletenessEnabled); | ||
| } | ||
|
|
||
| //Reset the table metadata for next accumulation period | ||
| tableMetadata.reset(currentProps, highWatermark); | ||
| log.info(String.format("Finish commit of new snapshot %s for table %s", snapshot.snapshotId(), tid)); | ||
| } else { | ||
| log.info("There's no transaction initiated for the table {}", tid); | ||
| //Set high waterMark | ||
| Long highWatermark = tableCurrentWatermarkMap.get(tid); | ||
| props.put(String.format(GMCE_HIGH_WATERMARK_KEY, tableTopicPartitionMap.get(tid)), highWatermark.toString()); | ||
| //Set low waterMark | ||
| props.put(String.format(GMCE_LOW_WATERMARK_KEY, tableTopicPartitionMap.get(tid)), | ||
| tableMetadata.lowWatermark.get().toString()); | ||
| //Set whether to delete metadata files after commit | ||
| if (conf.getBoolean(ICEBERG_ENABLE_CUSTOM_METADATA_RETENTION_POLICY, DEFAULT_ICEBERG_ENABLE_CUSTOM_METADATA_RETENTION_POLICY)) { | ||
| props.put(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, Boolean.toString( | ||
| conf.getBoolean(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT))); | ||
| props.put(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, Integer.toString( | ||
| conf.getInt(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, TableProperties.METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT))); | ||
| } | ||
| //Update schema(commit) | ||
| updateSchema(tableMetadata, props, topicName); | ||
| //Update properties | ||
| UpdateProperties updateProperties = transaction.updateProperties(); | ||
| props.forEach(updateProperties::set); | ||
| updateProperties.commit(); | ||
| try (AutoCloseableHiveLock lock = this.locks.getTableLock(dbName, tableName); | ||
| Timer.Context context = new Timer().time()) { | ||
| transaction.commitTransaction(); | ||
| log.info("Committing transaction for table {} took {} ms", tid, TimeUnit.NANOSECONDS.toMillis(context.stop())); | ||
| transactionCommitted = true; | ||
| } | ||
|
|
||
| postCommit(tableMetadata, dbName, tableName, topicName, highWatermark); | ||
| //Reset the table metadata for next accumulation period | ||
| Map<String, String> currentProps = tableMetadata.table.get().properties(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why no put these line in postCommit ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The way I think of postCommit is that we want to execute to have any custom logic for emission of metrics here. But resetting the table metadata we always want to do. If we ever need to override the above postCommit in an inherited class, they need to remember to invoke this boiler plate too. |
||
| Snapshot snapshot = tableMetadata.table.get().currentSnapshot(); | ||
| tableMetadata.reset(currentProps, highWatermark); | ||
| log.info(String.format("Finish commit of new snapshot %s for table %s", snapshot.snapshotId(), tid)); | ||
| } catch (RuntimeException e) { | ||
| throw new IOException(String.format("Fail to flush table %s %s", dbName, tableName), e); | ||
| throw new IOException(String.format("Failed to flush table %s %s. transactionCommitted=%s", | ||
| dbName, tableName, transactionCommitted), e); | ||
| } catch (Exception e) { | ||
| throw new IOException(String.format("Fail to flush table %s %s", dbName, tableName), e); | ||
| throw new IOException(String.format("Failed to flush table %s %s. transactionCommitted=%s", | ||
| dbName, tableName, transactionCommitted), e); | ||
| } finally { | ||
| writeLock.unlock(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * PostCommit operation that executes after the transaction is committed to the Iceberg table. Operations in this | ||
| * method are considered non-critical to the transaction and will not cause the transaction to fail if they fail, | ||
| * but should ideally still be executed for observability. | ||
| * | ||
| * One example of this is observability events / metrics like {@link org.apache.gobblin.metrics.GobblinTrackingEvent}. | ||
| * The default behavior is to emit a GTE for the commit event and a kafka audit event | ||
| * | ||
| * @param tableMetadata | ||
| * @param dbName | ||
| * @param tableName | ||
| * @param topicName | ||
| * @param highWatermark | ||
| * @throws IOException | ||
| */ | ||
| protected void postCommit( | ||
| TableMetadata tableMetadata, | ||
| String dbName, | ||
| String tableName, | ||
| String topicName, | ||
| long highWatermark) throws IOException { | ||
| // Emit GTE for snapshot commits | ||
| Snapshot snapshot = tableMetadata.table.get().currentSnapshot(); | ||
| Map<String, String> currentProps = tableMetadata.table.get().properties(); | ||
|
|
||
| // Sending the audit count before the snapshot commit event because downstream users are more likely | ||
| // to consume this audit count API for determining completion since it is agnostic to the system (e.g. Kafka, Brooklin) | ||
| // The snapshot commit event is more for internal monitoring. | ||
| try (Timer.Context context = new Timer().time()) { | ||
| sendAuditCounts(topicName, tableMetadata.serializedAuditCountMaps); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. let's try send audit count first to avoid lower counting as much as possible?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There are 2 ways to think about it:
Practically speaking, the most common use case is the first (as you suggest), so I think it's a good idea to emit the audit count first
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1, and in addition to that, missing one GTE won't cause alert in our monitoring system, but missing sending audit count will cause us to have less count permanently |
||
| log.info("Sending audit counts for {} took {} ms", topicName, TimeUnit.NANOSECONDS.toMillis(context.stop())); | ||
| } | ||
|
|
||
| try (Timer.Context context = new Timer().time()) { | ||
| submitSnapshotCommitEvent(snapshot, tableMetadata, dbName, tableName, currentProps, highWatermark); | ||
| log.info("Sending snapshot commit event for {} took {} ms", topicName, TimeUnit.NANOSECONDS.toMillis(context.stop())); | ||
| } | ||
| } | ||
|
|
||
| private CompletenessWatermarkUpdater getWatermarkUpdater(String topicName, TableMetadata tableMetadata, | ||
| Map<String, String> propsToUpdate) { | ||
| return new CompletenessWatermarkUpdater(topicName, this.auditCheckGranularity, this.timeZone, | ||
|
|
@@ -946,7 +982,7 @@ public void reset(String dbName, String tableName) throws IOException { | |
| this.tableMetadataMap.remove(TableIdentifier.of(dbName, tableName)); | ||
| } | ||
|
|
||
| private void submitSnapshotCommitEvent(Snapshot snapshot, TableMetadata tableMetadata, String dbName, | ||
| protected void submitSnapshotCommitEvent(Snapshot snapshot, TableMetadata tableMetadata, String dbName, | ||
| String tableName, Map<String, String> props, Long highWaterMark) { | ||
| GobblinEventBuilder gobblinTrackingEvent = | ||
| new GobblinEventBuilder(MetadataWriterKeys.ICEBERG_COMMIT_EVENT_NAME); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Review this PR with
hide whitespaceenabled. There is quite a bit of whitespace change