1
0

Incorporating code review feedback for DataSource

This commit is contained in:
Vinoth Chandar
2017-10-02 20:29:07 -07:00
committed by vinoth chandar
parent 64e0573aca
commit 274aaf49fe
11 changed files with 141 additions and 43 deletions

View File

@@ -59,7 +59,8 @@ public class DataSourceUtils {
if (i == parts.length - 1) {
return val.toString();
} else {
if (val instanceof GenericRecord) {
// VC: Need a test here
if (!(val instanceof GenericRecord)) {
throw new HoodieException("Cannot find a record at part value :" + part);
}
valueNode = (GenericRecord) val;
@@ -80,7 +81,7 @@ public class DataSourceUtils {
}
/**
* Create a payload class via reflection, passing in an ordering/precombine value value.
* Create a payload class via reflection, passing in an ordering/precombine value.
*/
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) throws IOException {
try {

View File

@@ -49,7 +49,6 @@ public class HoodieDataSourceHelpers {
* Get a list of instant times that have occurred, from the given instant timestamp.
*
* @param instantTimestamp
* @return
*/
public static List<String> listCommitsSince(FileSystem fs, String basePath, String instantTimestamp) {
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
@@ -71,7 +70,6 @@ public class HoodieDataSourceHelpers {
*
* @param fs
* @param basePath
* @return
*/
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
HoodieTable table = HoodieTable

View File

@@ -38,9 +38,9 @@ object DataSourceReadOptions {
* Default: READ_OPTIMIZED
*/
val VIEW_TYPE_OPT_KEY = "hoodie.datasource.view.type"
val VIEW_TYPE_READ_OPTIMIZED_OPT_VAL = "READ_OPTIMIZED"
val VIEW_TYPE_INCREMENTAL_OPT_VAL = "INCREMENTAL"
val VIEW_TYPE_REALTIME_OPT_VAL = "REALTIME"
val VIEW_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental"
val VIEW_TYPE_REALTIME_OPT_VAL = "realtime"
val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
@@ -82,6 +82,7 @@ object DataSourceWriteOptions {
/**
* The storage type for the underlying data, for this write.
* Note that this can't change across writes.
*
* Default: COPY_ON_WRITE
*/

View File

@@ -66,9 +66,6 @@ public class HoodieJavaApp {
cli.run();
}
public void run() throws Exception {
// Spark session setup..
@@ -103,6 +100,7 @@ public class HoodieJavaApp {
.mode(SaveMode.Overwrite) // This will remove any existing data at path below, and create a new dataset if needed
.save(tablePath); // ultimately where the dataset will be placed
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
logger.info("First commit at instant time :" + commitInstantTime1);
/**
* Commit that updates records
@@ -120,6 +118,7 @@ public class HoodieJavaApp {
.mode(SaveMode.Append)
.save(tablePath);
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
logger.info("Second commit at instant time :" + commitInstantTime1);
/**
* Read & do some queries
@@ -142,7 +141,7 @@ public class HoodieJavaApp {
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1) // Only changes in write 2 above
.load(tablePath); // For incremental view, pass in the root/base path of dataset
System.out.println("You will only see records from : " + commitInstantTime2);
logger.info("You will only see records from : " + commitInstantTime2);
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
}
}

View File

@@ -0,0 +1,110 @@
/*
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*
*/
import com.uber.hoodie.{DataSourceWriteOptions, OverwriteWithLatestAvroPayload, SimpleKeyGenerator}
import com.uber.hoodie.common.util.SchemaTestUtil
import com.uber.hoodie.exception.HoodieException
import org.apache.avro.generic.GenericRecord
import org.apache.commons.configuration.PropertiesConfiguration
import org.junit.Assert._
import org.junit.{Before, Test}
import org.scalatest.junit.AssertionsForJUnit
/**
* Tests on the default key generator, payload classes.
*/
class DataSourceDefaultsTest extends AssertionsForJUnit {
val schema = SchemaTestUtil.getComplexEvolvedSchema
var baseRecord : GenericRecord = null
@Before def initialize(): Unit = {
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1")
}
private def getKeyConfig(recordKeyFieldName: String, paritionPathField: String): PropertiesConfiguration = {
val props = new PropertiesConfiguration()
props.addProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
props.addProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, paritionPathField)
props
}
@Test def testSimpleKeyGenerator() = {
// top level, valid fields
val hk1 = new SimpleKeyGenerator(getKeyConfig("field1", "name")).getKey(baseRecord)
assertEquals("field1", hk1.getRecordKey)
assertEquals("name1", hk1.getPartitionPath)
// recordKey field not specified
try {
val props = new PropertiesConfiguration()
props.addProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
new SimpleKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieException => {
// do nothing
}
};
// partitionPath field is null
try {
new SimpleKeyGenerator(getKeyConfig("field1", null)).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieException => {
// do nothing
}
};
// nested field as record key and partition path
val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin"))
.getKey(baseRecord)
assertEquals("UserId1@001", hk2.getRecordKey)
assertEquals("false", hk2.getPartitionPath)
// Nested record key not found
try {
new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin"))
.getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieException => {
// do nothing
}
};
}
@Test def testOverwriteWithLatestAvroPayload() = {
val overWritePayload1 = new OverwriteWithLatestAvroPayload(baseRecord, 1)
val laterRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 2, "001", "f1")
val overWritePayload2 = new OverwriteWithLatestAvroPayload(laterRecord, 2)
// it will provide the record with greatest combine value
val combinedPayload12 = overWritePayload1.preCombine(overWritePayload2)
val combinedGR12 = combinedPayload12.getInsertValue(schema).get().asInstanceOf[GenericRecord]
assertEquals("field2", combinedGR12.get("field1"))
// and it will be deterministic, to order of processing.
val combinedPayload21 = overWritePayload2.preCombine(overWritePayload1)
val combinedGR21 = combinedPayload21.getInsertValue(schema).get().asInstanceOf[GenericRecord]
assertEquals("field2", combinedGR21.get("field1"))
}
}

View File

@@ -21,7 +21,7 @@ import com.uber.hoodie.common.util.FSUtils
import com.uber.hoodie.config.HoodieWriteConfig
import com.uber.hoodie.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.junit.Assert._
import org.junit.{Before, Test}
import org.junit.rules.TemporaryFolder
@@ -127,9 +127,9 @@ class DataSourceTest extends AssertionsForJUnit {
try {
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie")
.load(basePath + "/*/*/*/*")
fail() // we would error out, since no compaction has yet occurred.
fail("we should error out, since no compaction has yet occurred.")
} catch {
case e: Exception => {
case e: AnalysisException => {
// do nothing
}
};