[HUDI-583] Code Cleanup, remove redundant code, and other changes (#1237)
This commit is contained in:
@@ -43,7 +43,7 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -127,7 +127,7 @@ public class DataSourceUtils {
|
||||
}
|
||||
|
||||
public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
|
||||
checkPropNames.stream().forEach(prop -> {
|
||||
checkPropNames.forEach(prop -> {
|
||||
if (!props.containsKey(prop)) {
|
||||
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
|
||||
}
|
||||
@@ -182,19 +182,13 @@ public class DataSourceUtils {
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig, Option<EmbeddedTimelineService> timelineService) {
|
||||
HoodieReadClient client = null;
|
||||
try {
|
||||
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
|
||||
try (HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig, timelineService)) {
|
||||
return client.tagLocation(incomingHoodieRecords)
|
||||
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
|
||||
} catch (TableNotFoundException e) {
|
||||
// this will be executed when there is no hoodie table yet
|
||||
// so no dups to drop
|
||||
return incomingHoodieRecords;
|
||||
} finally {
|
||||
if (null != client) {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -207,12 +201,12 @@ public class DataSourceUtils {
|
||||
}
|
||||
|
||||
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath) {
|
||||
checkRequiredProperties(props, Arrays.asList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
|
||||
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.basePath = basePath;
|
||||
hiveSyncConfig.usePreApacheInputFormat =
|
||||
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(),
|
||||
Boolean.valueOf(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
|
||||
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(),
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL());
|
||||
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
|
||||
|
||||
@@ -52,7 +52,7 @@ object AvroConversionUtils {
|
||||
}
|
||||
|
||||
def createRddForDeletes(df: DataFrame, rowField: String, partitionField: String): RDD[HoodieKey] = {
|
||||
df.rdd.map(row => (new HoodieKey(row.getAs[String](rowField), row.getAs[String](partitionField))))
|
||||
df.rdd.map(row => new HoodieKey(row.getAs[String](rowField), row.getAs[String](partitionField)))
|
||||
}
|
||||
|
||||
def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = {
|
||||
@@ -67,7 +67,7 @@ object AvroConversionUtils {
|
||||
val convertor = AvroConversionHelper.createConverterToRow(schema, dataType)
|
||||
records.map { x => convertor(x).asInstanceOf[Row] }
|
||||
}
|
||||
}, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr))).asInstanceOf[Dataset[Row]]
|
||||
}, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr)))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -63,7 +63,7 @@ class DefaultSource extends RelationProvider
|
||||
sqlContext.sparkContext.hadoopConfiguration.setClass(
|
||||
"mapreduce.input.pathFilter.class",
|
||||
classOf[HoodieROTablePathFilter],
|
||||
classOf[org.apache.hadoop.fs.PathFilter]);
|
||||
classOf[org.apache.hadoop.fs.PathFilter])
|
||||
|
||||
log.info("Constructing hoodie (as parquet) data source with options :" + parameters)
|
||||
log.warn("Snapshot view not supported yet via data source, for MERGE_ON_READ tables. " +
|
||||
|
||||
@@ -77,7 +77,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
|
||||
val jsc = new JavaSparkContext(sparkContext)
|
||||
val basePath = new Path(parameters("path"))
|
||||
val commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
val commitTime = HoodieActiveTimeline.createNewInstantTime()
|
||||
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
|
||||
var exists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
|
||||
|
||||
@@ -282,7 +282,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
client.close()
|
||||
commitSuccess && syncHiveSucess
|
||||
} else {
|
||||
log.error(s"$operation failed with ${errorCount} errors :");
|
||||
log.error(s"$operation failed with $errorCount errors :")
|
||||
if (log.isTraceEnabled) {
|
||||
log.trace("Printing out the top 100 errors")
|
||||
writeStatuses.rdd.filter(ws => ws.hasErrors)
|
||||
|
||||
@@ -31,7 +31,7 @@ import org.scalatest.junit.AssertionsForJUnit
|
||||
class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
|
||||
val schema = SchemaTestUtil.getComplexEvolvedSchema
|
||||
var baseRecord: GenericRecord = null
|
||||
var baseRecord: GenericRecord = _
|
||||
|
||||
@Before def initialize(): Unit = {
|
||||
baseRecord = SchemaTestUtil
|
||||
@@ -60,10 +60,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException => {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
@@ -72,10 +71,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException => {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// nested field as record key and partition path
|
||||
val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin", "false"))
|
||||
@@ -89,14 +87,13 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException => {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// if partition path can't be found, return default partition path
|
||||
val hk3 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
.getKey(baseRecord);
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
|
||||
// if enable hive style partitioning
|
||||
@@ -155,10 +152,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException => {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
@@ -167,10 +163,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException => {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// nested field as record key and partition path
|
||||
val hk2 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
|
||||
@@ -184,14 +179,13 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException => {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// if partition path can't be found, return default partition path
|
||||
val hk3 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
.getKey(baseRecord);
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
|
||||
// if enable hive style partitioning
|
||||
@@ -269,10 +263,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException => {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
@@ -280,10 +273,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException => {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
|
||||
Reference in New Issue
Block a user