[HUDI-153] Use com.uber.hoodie.common.util.Option instead of Java and Guava Optional
This commit is contained in:
committed by
Balaji Varadarajan
parent
d288e32833
commit
722b6be04a
@@ -20,6 +20,7 @@ package com.uber.hoodie.hive;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import com.uber.hoodie.hadoop.HoodieInputFormat;
|
||||
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||
@@ -28,7 +29,6 @@ import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
||||
import com.uber.hoodie.hive.util.SchemaUtil;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -97,7 +97,7 @@ public class HiveSyncTool {
|
||||
|
||||
LOG.info("Schema sync complete. Syncing partitions for " + cfg.tableName);
|
||||
// Get the last time we successfully synced partitions
|
||||
Optional<String> lastCommitTimeSynced = Optional.empty();
|
||||
Option<String> lastCommitTimeSynced = Option.empty();
|
||||
if (tableExists) {
|
||||
lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced();
|
||||
}
|
||||
|
||||
@@ -29,6 +29,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
@@ -44,7 +45,6 @@ import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.commons.dbcp.ConnectionFactory;
|
||||
@@ -329,13 +329,13 @@ public class HoodieHiveClient {
|
||||
// If this is MOR, depending on whether the latest commit is a delta commit or
|
||||
// compaction commit
|
||||
// Get a datafile written and get the schema from that file
|
||||
Optional<HoodieInstant> lastCompactionCommit = metaClient.getActiveTimeline()
|
||||
Option<HoodieInstant> lastCompactionCommit = metaClient.getActiveTimeline()
|
||||
.getCommitTimeline()
|
||||
.filterCompletedInstants()
|
||||
.lastInstant();
|
||||
LOG.info("Found the last compaction commit as " + lastCompactionCommit);
|
||||
|
||||
Optional<HoodieInstant> lastDeltaCommit;
|
||||
Option<HoodieInstant> lastDeltaCommit;
|
||||
if (lastCompactionCommit.isPresent()) {
|
||||
lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline()
|
||||
.filterCompletedInstants()
|
||||
@@ -394,7 +394,7 @@ public class HoodieHiveClient {
|
||||
* Read schema from a data file from the last compaction commit done.
|
||||
*/
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
private MessageType readSchemaFromLastCompaction(Optional<HoodieInstant> lastCompactionCommitOpt)
|
||||
private MessageType readSchemaFromLastCompaction(Option<HoodieInstant> lastCompactionCommitOpt)
|
||||
throws IOException {
|
||||
HoodieInstant lastCompactionCommit = lastCompactionCommitOpt.orElseThrow(
|
||||
() -> new HoodieHiveSyncException(
|
||||
@@ -415,7 +415,7 @@ public class HoodieHiveClient {
|
||||
* Read the schema from the log file on path
|
||||
*/
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
private MessageType readSchemaFromLogFile(Optional<HoodieInstant> lastCompactionCommitOpt,
|
||||
private MessageType readSchemaFromLogFile(Option<HoodieInstant> lastCompactionCommitOpt,
|
||||
Path path) throws IOException {
|
||||
MessageType messageType = SchemaUtil.readSchemaFromLogFile(fs, path);
|
||||
// Fall back to read the schema from last compaction
|
||||
@@ -530,11 +530,11 @@ public class HoodieHiveClient {
|
||||
return fs;
|
||||
}
|
||||
|
||||
public Optional<String> getLastCommitTimeSynced() {
|
||||
public Option<String> getLastCommitTimeSynced() {
|
||||
// Get the last commit time from the TBLproperties
|
||||
try {
|
||||
Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName);
|
||||
return Optional.ofNullable(
|
||||
return Option.ofNullable(
|
||||
database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
@@ -556,7 +556,7 @@ public class HoodieHiveClient {
|
||||
}
|
||||
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
List<String> getPartitionsWrittenToSince(Optional<String> lastCommitTimeSynced) {
|
||||
List<String> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
|
||||
if (!lastCommitTimeSynced.isPresent()) {
|
||||
LOG.info("Last commit time synced is not known, listing all partitions in " + syncConfig.basePath + ",FS :" + fs);
|
||||
try {
|
||||
|
||||
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent;
|
||||
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
||||
@@ -30,7 +31,6 @@ import com.uber.hoodie.hive.util.SchemaUtil;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.apache.thrift.TException;
|
||||
import org.joda.time.DateTime;
|
||||
@@ -195,7 +195,7 @@ public class HiveSyncToolTest {
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
|
||||
TestUtil.fileSystem);
|
||||
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(
|
||||
Optional.of(commitTime1));
|
||||
Option.of(commitTime1));
|
||||
assertEquals("We should have one partition written after 100 commit", 1,
|
||||
writtenPartitionsSince.size());
|
||||
List<Partition> hivePartitions = hiveClient.scanTablePartitions();
|
||||
|
||||
Reference in New Issue
Block a user