1
0

Moving depedencies off cdh to apache + Hive2 support

- Tests redone in the process
 - Main changes are to RealtimeRecordReader and how it treats maps/arrays
 - Make hive sync work with Hive 1/2 and CDH environments
 - Fixes to make corner cases for Hive queries
 - Spark Hive integration - Working version across Apache and CDH versions
 - Known Issue - https://github.com/uber/hudi/issues/439
This commit is contained in:
Vinoth Chandar
2018-07-15 22:34:02 -07:00
committed by vinoth chandar
parent 2b1af18941
commit a5359662be
32 changed files with 1983 additions and 407 deletions

View File

@@ -32,6 +32,8 @@ import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -47,7 +49,8 @@ public class DataSourceUtils {
public static String getNestedFieldValAsString(GenericRecord record, String fieldName) {
String[] parts = fieldName.split("\\.");
GenericRecord valueNode = record;
for (int i = 0; i < parts.length; i++) {
int i = 0;
for (;i < parts.length; i++) {
String part = parts[i];
Object val = valueNode.get(part);
if (val == null) {
@@ -65,7 +68,9 @@ public class DataSourceUtils {
valueNode = (GenericRecord) val;
}
}
throw new HoodieException(fieldName + " field not found in record");
throw new HoodieException(fieldName + "(Part -" + parts[i] + ") field not found in record. "
+ "Acceptable fields were :" + valueNode.getSchema().getFields()
.stream().map(Field::name).collect(Collectors.toList()));
}
/**