Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hudi.aws.sync;

import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.hive.HiveSyncConfig;
Expand Down Expand Up @@ -380,16 +381,22 @@ public void close() {
}

@Override
public void updateLastCommitTimeSynced(String tableName) {
if (!getActiveTimeline().lastInstant().isPresent()) {
LOG.warn("No commit in active timeline.");
return;
public void updateLastCommitTimeSynced(String tableName, Option<HoodieInstant> hoodieInstantOption) {

Option<HoodieInstant> lastCommitSynced;
if (hoodieInstantOption.isPresent()) {
lastCommitSynced = hoodieInstantOption;
} else {
lastCommitSynced = getActiveTimeline().lastInstant();
}
final String lastCommitTimestamp = getActiveTimeline().lastInstant().get().getTimestamp();
try {
updateTableParameters(awsGlue, databaseName, tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitTimestamp), false);
} catch (Exception e) {
throw new HoodieGlueSyncException("Fail to update last sync commit time for " + tableId(databaseName, tableName), e);
if (lastCommitSynced.isPresent()) {
try {
updateTableParameters(awsGlue, databaseName, tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced.get().getTimestamp()), false);
} catch (Exception e) {
throw new HoodieGlueSyncException("Fail to update last sync commit time for " + tableId(databaseName, tableName), e);
}
} else {
LOG.warn("No commit in active timeline.");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@

import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import org.apache.hudi.hive.SchemaDifference;
import org.apache.hudi.hive.util.HiveSchemaUtil;
Expand Down Expand Up @@ -190,21 +192,21 @@ private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, b
LOG.info("Last commit time synced was found:{}", lastCommitTimeSynced.orElse("null"));

// Scan synced partitions
List<String> writtenPartitionsSince;
Pair<List<String>, Option<HoodieInstant>> writtenPartitionsSincePair;
if (config.getSplitStrings(META_SYNC_PARTITION_FIELDS).isEmpty()) {
writtenPartitionsSince = new ArrayList<>();
writtenPartitionsSincePair = Pair.of(new ArrayList<>(), Option.empty());
} else {
writtenPartitionsSince = syncClient.getPartitionsWrittenToSince(lastCommitTimeSynced);
writtenPartitionsSincePair = syncClient.getPartitionsWrittenToSince(lastCommitTimeSynced);
}
LOG.info("Scan partitions complete, partitionNum:{}", writtenPartitionsSince.size());
LOG.info("Scan partitions complete, partitionNum:{}", writtenPartitionsSincePair.getLeft().size());

// Sync the partitions if needed
syncPartitions(tableName, writtenPartitionsSince);
syncPartitions(tableName, writtenPartitionsSincePair.getLeft());

// Update sync commit time
// whether to skip syncing commit time stored in tbl properties, since it is time consuming.
if (!config.getBoolean(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC)) {
syncClient.updateLastCommitTimeSynced(tableName);
syncClient.updateLastCommitTimeSynced(tableName, writtenPartitionsSincePair.getRight());
}
LOG.info("Sync complete for table:{}", tableName);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hudi.sync.adb;

import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
Expand Down Expand Up @@ -259,14 +260,23 @@ public Option<String> getLastCommitTimeSynced(String tableName) {
}

@Override
public void updateLastCommitTimeSynced(String tableName) {
public void updateLastCommitTimeSynced(String tableName, Option<HoodieInstant> hoodieInstantOption) {
// Set the last commit time from the TBLProperties
String lastCommitSynced = getActiveTimeline().lastInstant().get().getTimestamp();
try {
String sql = constructUpdateTblPropertiesSql(tableName, lastCommitSynced);
executeAdbSql(sql);
} catch (Exception e) {
throw new HoodieHiveSyncException("Fail to get update last commit time synced:" + lastCommitSynced, e);
Option<HoodieInstant> lastCommitSynced;
if (hoodieInstantOption.isPresent()) {
lastCommitSynced = hoodieInstantOption;
} else {
lastCommitSynced = getActiveTimeline().lastInstant();
}
if (lastCommitSynced.isPresent()) {
try {
String sql = constructUpdateTblPropertiesSql(tableName, lastCommitSynced.get().getTimestamp());
executeAdbSql(sql);
} catch (Exception e) {
throw new HoodieHiveSyncException("Fail to get update last commit time synced:" + hoodieInstantOption.get(), e);
}
} else {
LOG.warn("No commit in active timeline.");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.sync.common.HoodieSyncClient;
import org.apache.hudi.sync.common.HoodieSyncException;
Expand Down Expand Up @@ -74,8 +75,8 @@ public Option<String> getLastCommitTimeSynced(String tableName) {
}

@Override
public void updateLastCommitTimeSynced(String tableName) {
updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, getActiveTimeline().lastInstant().get().getTimestamp()));
public void updateLastCommitTimeSynced(String tableName, Option<HoodieInstant> hoodieInstantOption) {
updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, hoodieInstantOption.get().getTimestamp()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.apache.hudi.sync.datahub;

import org.apache.hudi.common.util.Option;
import org.apache.hudi.sync.common.HoodieSyncTool;
import org.apache.hudi.sync.datahub.config.DataHubSyncConfig;

Expand Down Expand Up @@ -53,7 +54,7 @@ public DataHubSyncTool(Properties props) {
public void syncHoodieTable() {
try (DataHubSyncClient syncClient = new DataHubSyncClient(config)) {
syncClient.updateTableSchema(config.getString(META_SYNC_TABLE_NAME), null);
syncClient.updateLastCommitTimeSynced(config.getString(META_SYNC_TABLE_NAME));
syncClient.updateLastCommitTimeSynced(config.getString(META_SYNC_TABLE_NAME), Option.empty());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@

import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
Expand Down Expand Up @@ -224,14 +226,14 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat,
lastCommitTimeSynced = syncClient.getLastCommitTimeSynced(tableName);
}
LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null"));
List<String> writtenPartitionsSince = syncClient.getPartitionsWrittenToSince(lastCommitTimeSynced);
LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size());
Pair<List<String>, Option<HoodieInstant>> writtenPartitionsSincePair = syncClient.getPartitionsWrittenToSince(lastCommitTimeSynced);
LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSincePair.getLeft().size());

// Sync the partitions if needed
boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSince, isDropPartition);
boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSincePair.getLeft(), isDropPartition);
boolean meetSyncConditions = schemaChanged || partitionsChanged;
if (!config.getBoolean(META_SYNC_CONDITIONAL_SYNC) || meetSyncConditions) {
syncClient.updateLastCommitTimeSynced(tableName);
syncClient.updateLastCommitTimeSynced(tableName, writtenPartitionsSincePair.getRight());
}
LOG.info("Sync complete for " + tableName);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -257,17 +257,24 @@ public void close() {
}

@Override
public void updateLastCommitTimeSynced(String tableName) {
public void updateLastCommitTimeSynced(String tableName, Option<HoodieInstant> hoodieInstantOption) {
// Set the last commit time from the TBLproperties
Option<String> lastCommitSynced = getActiveTimeline().lastInstant().map(HoodieInstant::getTimestamp);
Option<HoodieInstant> lastCommitSynced;
if (hoodieInstantOption.isPresent()) {
lastCommitSynced = hoodieInstantOption;
} else {
lastCommitSynced = getActiveTimeline().lastInstant();
}
if (lastCommitSynced.isPresent()) {
try {
Table table = client.getTable(databaseName, tableName);
table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced.get());
table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced.get().getTimestamp());
client.alter_table(databaseName, tableName, table);
} catch (Exception e) {
throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e);
throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + hoodieInstantOption.get(), e);
}
} else {
LOG.warn("No commit in active timeline.");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.testutils.NetworkTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.common.util.Option;
Expand Down Expand Up @@ -188,8 +189,8 @@ public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode)
+ "` PARTITION (`datestr`='2050-01-01') SET LOCATION '/some/new/location'");

List<Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty());
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
Pair<List<String>, Option<HoodieInstant>> writtenPartitionsSincePair = hiveClient.getPartitionsWrittenToSince(Option.empty());
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSincePair.getLeft(), false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
assertEquals(PartitionEventType.UPDATE, partitionEvents.iterator().next().eventType,
"The one partition event must of type UPDATE");
Expand Down Expand Up @@ -466,10 +467,10 @@ public void testSyncIncremental(String syncMode) throws Exception {

// Lets do the sync
reSyncHiveTable();
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1));
assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit");
Pair<List<String>, Option<HoodieInstant>> writtenPartitionsSincePair = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1));
assertEquals(1, writtenPartitionsSincePair.getLeft().size(), "We should have one partition written after 100 commit");
List<org.apache.hudi.sync.common.model.Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSincePair.getLeft(), false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD");

Expand Down Expand Up @@ -745,10 +746,10 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception {
HiveTestUtil.addCOWPartition("2010/01/02", true, true, commitTime2);

reinitHiveSyncClient();
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(instantTime));
assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit");
Pair<List<String>, Option<HoodieInstant>> writtenPartitionsSincePair = hiveClient.getPartitionsWrittenToSince(Option.of(instantTime));
assertEquals(1, writtenPartitionsSincePair.getLeft().size(), "We should have one partition written after 100 commit");
List<org.apache.hudi.sync.common.model.Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSincePair.getLeft(), false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD");

Expand All @@ -775,7 +776,7 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception {
"Table partitions should match the number of partitions we wrote");
assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(),
"The last commit that was synced should be updated in the TBLPROPERTIES");
assertEquals(1, hiveClient.getPartitionsWrittenToSince(Option.of(commitTime2)).size());
assertEquals(1, hiveClient.getPartitionsWrittenToSince(Option.of(commitTime2)).getLeft().size());
}

@ParameterizedTest
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.apache.hudi.sync.common;

import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.sync.common.model.FieldSchema;
import org.apache.hudi.sync.common.model.Partition;
Expand Down Expand Up @@ -162,7 +163,7 @@ default Option<String> getLastCommitTimeSynced(String tableName) {
/**
* Update the timestamp of last sync.
*/
default void updateLastCommitTimeSynced(String tableName) {
default void updateLastCommitTimeSynced(String tableName, Option<HoodieInstant> hoodieInstantOption) {

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,12 @@
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.metadata.HoodieTableMetadataUtil;
import org.apache.hudi.sync.common.model.Partition;
import org.apache.hudi.sync.common.model.PartitionEvent;
Expand Down Expand Up @@ -106,20 +108,22 @@ public MessageType getStorageSchema() {
}
}

public List<String> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
public Pair<List<String>, Option<HoodieInstant>> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
if (!lastCommitTimeSynced.isPresent()) {
LOG.info("Last commit time synced is not known, listing all partitions in "
+ config.getString(META_SYNC_BASE_PATH)
+ ",FS :" + config.getHadoopFileSystem());
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
return FSUtils.getAllPartitionPaths(engineContext,
return Pair.of(FSUtils.getAllPartitionPaths(engineContext,
config.getString(META_SYNC_BASE_PATH),
config.getBoolean(META_SYNC_USE_FILE_LISTING_FROM_METADATA),
config.getBoolean(META_SYNC_ASSUME_DATE_PARTITION));
config.getBoolean(META_SYNC_ASSUME_DATE_PARTITION)), Option.empty());
} else {
Option<HoodieInstant> hoodieInstantOption = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant();
HoodieTimeline instantsInRange = metaClient.getActiveTimeline().getCommitsTimeline()
.findInstantsInRange(lastCommitTimeSynced.get(), hoodieInstantOption.get().getTimestamp());
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then");
return TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().getCommitsTimeline()
.findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE));
return Pair.of(TimelineUtils.getPartitionsWritten(instantsInRange), hoodieInstantOption);
}
}

Expand Down