1
0

[HUDI-153] Use com.uber.hoodie.common.util.Option instead of Java and Guava Optional

This commit is contained in:
yanghua
2019-08-06 14:20:42 +08:00
committed by Balaji Varadarajan
parent d288e32833
commit 722b6be04a
128 changed files with 769 additions and 769 deletions

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.config.HoodieCompactionConfig;
@@ -39,7 +40,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.GenericRecord;
@@ -191,7 +191,7 @@ public class DataSourceUtils {
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
JavaRDD<HoodieRecord> incomingHoodieRecords,
HoodieWriteConfig writeConfig, Optional<EmbeddedTimelineService> timelineService) throws Exception {
HoodieWriteConfig writeConfig, Option<EmbeddedTimelineService> timelineService) throws Exception {
HoodieReadClient client = null;
try {
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
@@ -212,7 +212,7 @@ public class DataSourceUtils {
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
JavaRDD<HoodieRecord> incomingHoodieRecords,
Map<String, String> parameters,
Optional<EmbeddedTimelineService> timelineService)
Option<EmbeddedTimelineService> timelineService)
throws Exception {
HoodieWriteConfig writeConfig = HoodieWriteConfig
.newBuilder()

View File

@@ -19,7 +19,7 @@
package com.uber.hoodie;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.util.Optional;
import com.uber.hoodie.common.util.Option;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -37,12 +37,12 @@ public class EmptyHoodieRecordPayload implements HoodieRecordPayload<EmptyHoodie
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
return Optional.empty();
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
return Option.empty();
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) {
return Optional.empty();
public Option<IndexedRecord> getInsertValue(Schema schema) {
return Option.empty();
}
}

View File

@@ -20,8 +20,8 @@ package com.uber.hoodie;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.Option;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -43,7 +43,7 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
super(record, orderingVal);
}
public OverwriteWithLatestAvroPayload(Optional<GenericRecord> record) {
public OverwriteWithLatestAvroPayload(Option<GenericRecord> record) {
this(record.get(), (record1) -> 0); // natural order
}
@@ -58,14 +58,14 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
// combining strategy here trivially ignores currentValue on disk and writes this record
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
}
}

View File

@@ -18,7 +18,6 @@
package com.uber.hoodie
import java.util
import java.util.Optional
import com.uber.hoodie.DataSourceWriteOptions._
import com.uber.hoodie.common.table.HoodieTableMetaClient
@@ -26,6 +25,7 @@ import com.uber.hoodie.common.util.{FSUtils, TypedProperties}
import com.uber.hoodie.config.HoodieWriteConfig
import com.uber.hoodie.exception.HoodieException
import com.uber.hoodie.hive.{HiveSyncConfig, HiveSyncTool}
import com.uber.hoodie.common.util.Option
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.conf.HiveConf
@@ -105,7 +105,7 @@ private[hoodie] object HoodieSparkSqlWriter {
}
if (mode == SaveMode.Ignore && exists) {
log.warn(s"hoodie dataset at $basePath already exists. Ignoring & not performing actual writes.")
return (true, None)
return (true, Option.empty())
}
if (mode == SaveMode.Overwrite && exists) {
log.warn(s"hoodie dataset at $basePath already exists. Deleting existing data & overwriting with new data.")
@@ -136,7 +136,7 @@ private[hoodie] object HoodieSparkSqlWriter {
if (hoodieRecords.isEmpty()) {
log.info("new batch has no new records, skipping...")
return (true, None)
return (true, Option.empty())
}
val commitTime = client.startCommit()
@@ -153,7 +153,7 @@ private[hoodie] object HoodieSparkSqlWriter {
client.commit(commitTime, writeStatuses)
} else {
client.commit(commitTime, writeStatuses,
Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
}
if (commitSuccess) {
@@ -189,7 +189,7 @@ private[hoodie] object HoodieSparkSqlWriter {
}
false
}
(writeSuccessful, Some(commitTime))
(writeSuccessful, Option.ofNullable(commitTime))
}
/**

View File

@@ -56,7 +56,10 @@ class HoodieStreamingSink(sqlContext: SQLContext,
) match {
case Success((true, commitOps)) =>
log.info(s"Micro batch id=$batchId succeeded"
+ commitOps.map(commit => s" for commit=$commit").getOrElse(" with no new commits"))
+ (commitOps.isPresent match {
case true => s" for commit=${commitOps.get()}"
case _ => s" with no new commits"
}))
Success((true, commitOps))
case Failure(e) =>
// clean up persist rdds in the write process
@@ -76,7 +79,10 @@ class HoodieStreamingSink(sqlContext: SQLContext,
}
case Success((false, commitOps)) =>
log.error(s"Micro batch id=$batchId ended up with errors"
+ commitOps.map(commit => s" for commit=$commit").getOrElse(""))
+ (commitOps.isPresent match {
case true => s" for commit=${commitOps.get()}"
case _ => s""
}))
if (ignoreFailedBatch) {
log.info(s"Ignore the errors and move on streaming as per " +
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration")

View File

@@ -18,9 +18,9 @@
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.util.Option;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
/**
@@ -28,14 +28,14 @@ import java.util.stream.Collectors;
*/
public class DataSourceTestUtils {
public static Optional<String> convertToString(HoodieRecord record) {
public static Option<String> convertToString(HoodieRecord record) {
try {
String str = ((TestRawTripPayload) record.getData()).getJsonData();
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
return Optional.of(str.replaceAll("}",
return Option.of(str.replaceAll("}",
", \"partition\": \"" + record.getPartitionPath() + "\"}"));
} catch (IOException e) {
return Optional.empty();
return Option.empty();
}
}

View File

@@ -15,9 +15,7 @@
* limitations under the License.
*/
import java.util.Optional
import com.uber.hoodie.common.util.{SchemaTestUtil, TypedProperties}
import com.uber.hoodie.common.util.{Option, SchemaTestUtil, TypedProperties}
import com.uber.hoodie.exception.HoodieException
import com.uber.hoodie.{DataSourceWriteOptions, EmptyHoodieRecordPayload, OverwriteWithLatestAvroPayload, SimpleKeyGenerator, ComplexKeyGenerator}
import org.apache.avro.generic.GenericRecord
@@ -178,6 +176,6 @@ class DataSourceDefaultsTest extends AssertionsForJUnit {
// it will provide an empty record
val combinedPayload12 = emptyPayload1.preCombine(emptyPayload2)
val combined12 = combinedPayload12.getInsertValue(schema)
assertEquals(Optional.empty(), combined12)
assertEquals(Option.empty(), combined12)
}
}