Adding hoodie-spark to support Spark Datasource for Hoodie
- Write with COW/MOR paths work fully - Read with RO view works on both storages* - Incremental view supported on COW - Refactored out HoodieReadClient methods, to just contain key based access - HoodieDataSourceHelpers class can be now used to construct inputs to datasource - Tests in hoodie-client using new helpers and mechanisms - Basic tests around save modes & insert/upserts (more to follow) - Bumped up scala to 2.11, since 2.10 is deprecated & complains with scalatest - Updated documentation to describe usage - New sample app written using the DataSource API
This commit is contained in:
committed by
vinoth chandar
parent
c98ee057fc
commit
64e0573aca
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Base class for all AVRO record based payloads, that can be ordered based on a field
|
||||
*/
|
||||
public abstract class BaseAvroPayload implements Serializable {
|
||||
|
||||
|
||||
/**
|
||||
* Avro data extracted from the source
|
||||
*/
|
||||
protected final GenericRecord record;
|
||||
|
||||
/**
|
||||
* For purposes of preCombining
|
||||
*/
|
||||
protected final Comparable orderingVal;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param record
|
||||
* @param orderingVal
|
||||
*/
|
||||
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
this.record = record;
|
||||
this.orderingVal = orderingVal;
|
||||
}
|
||||
}
|
||||
148
hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java
Normal file
148
hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java
Normal file
@@ -0,0 +1,148 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
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.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.commons.lang3.reflect.ConstructorUtils;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Utilities used throughout the data source
|
||||
*/
|
||||
public class DataSourceUtils {
|
||||
|
||||
/**
|
||||
* Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c
|
||||
*/
|
||||
public static String getNestedFieldValAsString(GenericRecord record, String fieldName) {
|
||||
String[] parts = fieldName.split("\\.");
|
||||
GenericRecord valueNode = record;
|
||||
for (int i = 0; i < parts.length; i++) {
|
||||
String part = parts[i];
|
||||
Object val = valueNode.get(part);
|
||||
if (val == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
// return, if last part of name
|
||||
if (i == parts.length - 1) {
|
||||
return val.toString();
|
||||
} else {
|
||||
if (val instanceof GenericRecord) {
|
||||
throw new HoodieException("Cannot find a record at part value :" + part);
|
||||
}
|
||||
valueNode = (GenericRecord) val;
|
||||
}
|
||||
}
|
||||
throw new HoodieException(fieldName + " field not found in record");
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a key generator class via reflection, passing in any configs needed
|
||||
*/
|
||||
public static KeyGenerator createKeyGenerator(String keyGeneratorClass, PropertiesConfiguration cfg) throws IOException {
|
||||
try {
|
||||
return (KeyGenerator) ConstructorUtils.invokeConstructor(Class.forName(keyGeneratorClass), (Object) cfg);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a payload class via reflection, passing in an ordering/precombine value value.
|
||||
*/
|
||||
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) throws IOException {
|
||||
try {
|
||||
return (HoodieRecordPayload) ConstructorUtils.invokeConstructor(Class.forName(payloadClass), (Object) record, (Object) orderingVal);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not create payload for class: " + payloadClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void checkRequiredProperties(PropertiesConfiguration configuration, List<String> checkPropNames) {
|
||||
checkPropNames.stream().forEach(prop -> {
|
||||
if (!configuration.containsKey(prop)) {
|
||||
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc,
|
||||
String schemaStr,
|
||||
String basePath,
|
||||
String tblName,
|
||||
Map<String, String> parameters) throws Exception {
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
|
||||
.combineInput(true, true)
|
||||
.withPath(basePath)
|
||||
.withAutoCommit(false)
|
||||
.withSchema(schemaStr)
|
||||
.forTable(tblName)
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder()
|
||||
.withIndexType(HoodieIndex.IndexType.BLOOM)
|
||||
.build())
|
||||
// override above with Hoodie configs specified as options.
|
||||
.withProps(parameters)
|
||||
.build();
|
||||
|
||||
return new HoodieWriteClient<>(jssc, writeConfig);
|
||||
}
|
||||
|
||||
|
||||
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client,
|
||||
JavaRDD<HoodieRecord> hoodieRecords,
|
||||
String commitTime,
|
||||
String operation) {
|
||||
if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) {
|
||||
return client.bulkInsert(hoodieRecords, commitTime);
|
||||
} else if (operation.equals(DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())) {
|
||||
return client.insert(hoodieRecords, commitTime);
|
||||
} else {
|
||||
//default is upsert
|
||||
return client.upsert(hoodieRecords, commitTime);
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieRecord createHoodieRecord(GenericRecord gr,
|
||||
Comparable orderingVal,
|
||||
HoodieKey hKey,
|
||||
String payloadClass) throws IOException {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(
|
||||
payloadClass,
|
||||
gr,
|
||||
orderingVal);
|
||||
return new HoodieRecord<>(hKey, payload);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,88 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* List of helpers to aid, construction of instanttime for read and write operations using datasource
|
||||
*/
|
||||
public class HoodieDataSourceHelpers {
|
||||
|
||||
/**
|
||||
* Checks if the Hoodie dataset has new data since given timestamp. This can be subsequently
|
||||
* fed to an incremental view read, to perform incremental processing.
|
||||
*/
|
||||
public static boolean hasNewCommits(FileSystem fs, String basePath, String commitTimestamp) {
|
||||
return listCommitsSince(fs, basePath, commitTimestamp).size() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last successful write operation's instant time
|
||||
*/
|
||||
public static String latestCommit(FileSystem fs, String basePath) {
|
||||
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
|
||||
return timeline.lastInstant().get().getTimestamp();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all the commits, compactions that have occurred on the timeline, whose
|
||||
* instant times could be fed into the datasource options.
|
||||
*
|
||||
* @param fs
|
||||
* @param basePath
|
||||
* @return
|
||||
*/
|
||||
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||
if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
return table.getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION)
|
||||
);
|
||||
} else {
|
||||
return table.getCompletedCompactionCommitTimeline();
|
||||
}
|
||||
}
|
||||
}
|
||||
47
hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java
Normal file
47
hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java
Normal file
@@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Abstract class to extend for plugging in extraction of {@link com.uber.hoodie.common.model.HoodieKey}
|
||||
* from an Avro record
|
||||
*/
|
||||
public abstract class KeyGenerator implements Serializable {
|
||||
|
||||
protected transient PropertiesConfiguration config;
|
||||
|
||||
protected KeyGenerator(PropertiesConfiguration config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a Hoodie Key out of provided generic record.
|
||||
*
|
||||
* @param record
|
||||
* @return
|
||||
*/
|
||||
public abstract HoodieKey getKey(GenericRecord record);
|
||||
}
|
||||
@@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Default payload used for delta streamer.
|
||||
*
|
||||
* 1. preCombine - Picks the latest delta record for a key, based on an ordering field
|
||||
* 2. combineAndGetUpdateValue/getInsertValue - Simply overwrites storage with latest delta record
|
||||
*/
|
||||
public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
|
||||
|
||||
/**
|
||||
*
|
||||
* @param record
|
||||
* @param orderingVal
|
||||
*/
|
||||
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
super(record, orderingVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
|
||||
// pick the payload with greatest ordering value
|
||||
if (another.orderingVal.compareTo(orderingVal) > 0) {
|
||||
return another;
|
||||
} else {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<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.rewriteRecord(record, schema));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
|
||||
/**
|
||||
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath
|
||||
* as configs.
|
||||
*/
|
||||
public class SimpleKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected final String recordKeyField;
|
||||
|
||||
protected final String partitionPathField;
|
||||
|
||||
public SimpleKeyGenerator(PropertiesConfiguration config) {
|
||||
super(config);
|
||||
this.recordKeyField = config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
this.partitionPathField = config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyField == null || partitionPathField == null) {
|
||||
throw new HoodieException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
|
||||
DataSourceUtils.getNestedFieldValAsString(record, partitionPathField));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
package com.uber.hoodie
|
||||
|
||||
import java.nio.ByteBuffer
|
||||
import java.sql.{Date, Timestamp}
|
||||
import java.util
|
||||
|
||||
import com.databricks.spark.avro.SchemaConverters
|
||||
import org.apache.avro.generic.GenericData.Record
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.avro.{Schema, SchemaBuilder}
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.sql.{DataFrame, Row}
|
||||
|
||||
|
||||
object AvroConversionUtils {
|
||||
|
||||
def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = {
|
||||
val dataType = df.schema
|
||||
df.rdd.mapPartitions { records =>
|
||||
if (records.isEmpty) Iterator.empty
|
||||
else {
|
||||
val convertor = createConverterToAvro(dataType, structName, recordNamespace)
|
||||
records.map { x => convertor(x).asInstanceOf[GenericRecord] }
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def createConverterToAvro(dataType: DataType,
|
||||
structName: String,
|
||||
recordNamespace: String): (Any) => Any = {
|
||||
dataType match {
|
||||
case BinaryType => (item: Any) => item match {
|
||||
case null => null
|
||||
case bytes: Array[Byte] => ByteBuffer.wrap(bytes)
|
||||
}
|
||||
case ByteType | ShortType | IntegerType | LongType |
|
||||
FloatType | DoubleType | StringType | BooleanType => identity
|
||||
case _: DecimalType => (item: Any) => if (item == null) null else item.toString
|
||||
case TimestampType => (item: Any) =>
|
||||
if (item == null) null else item.asInstanceOf[Timestamp].getTime
|
||||
case DateType => (item: Any) =>
|
||||
if (item == null) null else item.asInstanceOf[Date].getTime
|
||||
case ArrayType(elementType, _) =>
|
||||
val elementConverter = createConverterToAvro(elementType, structName, recordNamespace)
|
||||
(item: Any) => {
|
||||
if (item == null) {
|
||||
null
|
||||
} else {
|
||||
val sourceArray = item.asInstanceOf[Seq[Any]]
|
||||
val sourceArraySize = sourceArray.size
|
||||
val targetList = new util.ArrayList[Any](sourceArraySize)
|
||||
var idx = 0
|
||||
while (idx < sourceArraySize) {
|
||||
targetList.add(elementConverter(sourceArray(idx)))
|
||||
idx += 1
|
||||
}
|
||||
targetList
|
||||
}
|
||||
}
|
||||
case MapType(StringType, valueType, _) =>
|
||||
val valueConverter = createConverterToAvro(valueType, structName, recordNamespace)
|
||||
(item: Any) => {
|
||||
if (item == null) {
|
||||
null
|
||||
} else {
|
||||
val javaMap = new util.HashMap[String, Any]()
|
||||
item.asInstanceOf[Map[String, Any]].foreach { case (key, value) =>
|
||||
javaMap.put(key, valueConverter(value))
|
||||
}
|
||||
javaMap
|
||||
}
|
||||
}
|
||||
case structType: StructType =>
|
||||
val builder = SchemaBuilder.record(structName).namespace(recordNamespace)
|
||||
val schema: Schema = SchemaConverters.convertStructToAvro(
|
||||
structType, builder, recordNamespace)
|
||||
val fieldConverters = structType.fields.map(field =>
|
||||
createConverterToAvro(field.dataType, field.name, recordNamespace))
|
||||
(item: Any) => {
|
||||
if (item == null) {
|
||||
null
|
||||
} else {
|
||||
val record = new Record(schema)
|
||||
val convertersIterator = fieldConverters.iterator
|
||||
val fieldNamesIterator = dataType.asInstanceOf[StructType].fieldNames.iterator
|
||||
val rowIterator = item.asInstanceOf[Row].toSeq.iterator
|
||||
|
||||
while (convertersIterator.hasNext) {
|
||||
val converter = convertersIterator.next()
|
||||
record.put(fieldNamesIterator.next(), converter(rowIterator.next()))
|
||||
}
|
||||
record
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def convertStructTypeToAvroSchema(structType: StructType,
|
||||
structName: String,
|
||||
recordNamespace: String) : Schema = {
|
||||
val builder = SchemaBuilder.record(structName).namespace(recordNamespace)
|
||||
SchemaConverters.convertStructToAvro(structType, builder, recordNamespace)
|
||||
}
|
||||
|
||||
def convertAvroSchemaToStructType(avroSchema: Schema) : StructType = {
|
||||
SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType];
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,145 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableType
|
||||
|
||||
/**
|
||||
* List of options that can be passed to the Hoodie datasource,
|
||||
* in addition to the hoodie client configs
|
||||
*/
|
||||
|
||||
/**
|
||||
* Options supported for reading hoodie datasets.
|
||||
*/
|
||||
object DataSourceReadOptions {
|
||||
/**
|
||||
* Whether data needs to be read, in
|
||||
* incremental mode (new data since an instantTime)
|
||||
* (or) Read Optimized mode (obtain latest view, based on columnar data)
|
||||
* (or) Real time mode (obtain latest view, based on row & columnar data)
|
||||
*
|
||||
* 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 DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
|
||||
|
||||
|
||||
/**
|
||||
* Instant time to start incrementally pulling data from. The instanttime here need not
|
||||
* necessarily correspond to an instant on the timeline. New data written with an
|
||||
* `instant_time > BEGIN_INSTANTTIME` are fetched out. For e.g: '20170901080000' will get
|
||||
* all new data written after Sep 1, 2017 08:00AM.
|
||||
*
|
||||
* Default: None (Mandatory in incremental mode)
|
||||
*/
|
||||
val BEGIN_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.begin.instanttime"
|
||||
|
||||
|
||||
/**
|
||||
* Instant time to limit incrementally fetched data to. New data written with an
|
||||
* `instant_time <= END_INSTANTTIME` are fetched out.
|
||||
*
|
||||
* Default: latest instant (i.e fetches all new data since begin instant time)
|
||||
*
|
||||
*/
|
||||
val END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.end.instanttime"
|
||||
}
|
||||
|
||||
/**
|
||||
* Options supported for writing hoodie datasets.
|
||||
*/
|
||||
object DataSourceWriteOptions {
|
||||
/**
|
||||
* The client operation, that this write should do
|
||||
*
|
||||
* Default: upsert()
|
||||
*/
|
||||
val OPERATION_OPT_KEY = "hoodie.datasource.write.operation"
|
||||
val BULK_INSERT_OPERATION_OPT_VAL = "bulk_insert"
|
||||
val INSERT_OPERATION_OPT_VAL = "insert"
|
||||
val UPSERT_OPERATION_OPT_VAL = "upsert"
|
||||
val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL;
|
||||
|
||||
/**
|
||||
* The storage type for the underlying data, for this write.
|
||||
*
|
||||
* Default: COPY_ON_WRITE
|
||||
*/
|
||||
val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type"
|
||||
val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
|
||||
val MOR_STORAGE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
|
||||
val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
|
||||
|
||||
/**
|
||||
* Hive table name, to register the dataset into.
|
||||
*
|
||||
* Default: None (mandatory)
|
||||
*/
|
||||
val TABLE_NAME_OPT_KEY = "hoodie.datasource.write.table.name"
|
||||
|
||||
/**
|
||||
* Field used in preCombining before actual write. When two records have the same
|
||||
* key value, we will pick the one with the largest value for the precombine field,
|
||||
* determined by Object.compareTo(..)
|
||||
*/
|
||||
val PRECOMBINE_FIELD_OPT_KEY = "hoodie.datasource.write.precombine.field"
|
||||
val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = "ts"
|
||||
|
||||
|
||||
/**
|
||||
* Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.
|
||||
* This will render any value set for `PRECOMBINE_FIELD_OPT_VAL` in-effective
|
||||
*/
|
||||
val PAYLOAD_CLASS_OPT_KEY = "hoodie.datasource.write.payload.class"
|
||||
val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName
|
||||
|
||||
/**
|
||||
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
|
||||
* will be obtained by invoking .toString() on the field value. Nested fields can be specified using
|
||||
* the dot notation eg: `a.b.c`
|
||||
*
|
||||
*/
|
||||
val RECORDKEY_FIELD_OPT_KEY = "hoodie.datasource.write.recordkey.field"
|
||||
val DEFAULT_RECORDKEY_FIELD_OPT_VAL = "uuid"
|
||||
|
||||
/**
|
||||
* Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`. Actual
|
||||
* value ontained by invoking .toString()
|
||||
*/
|
||||
val PARTITIONPATH_FIELD_OPT_KEY = "hoodie.datasource.write.partitionpath.field"
|
||||
val DEFAULT_PARTITIONPATH_FIELD_OPT_VAL = "partitionpath"
|
||||
|
||||
/**
|
||||
* Key generator class, that implements will extract the key out of incoming record
|
||||
*
|
||||
*/
|
||||
val KEYGENERATOR_CLASS_OPT_KEY = "hoodie.datasource.write.keygenerator.class"
|
||||
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName
|
||||
|
||||
/**
|
||||
* Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata.
|
||||
* This is useful to store checkpointing information, in a consistent way with the hoodie timeline
|
||||
*/
|
||||
val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix"
|
||||
val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_"
|
||||
}
|
||||
244
hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala
Normal file
244
hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala
Normal file
@@ -0,0 +1,244 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie
|
||||
|
||||
import java.util
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import java.util.{Optional, Properties}
|
||||
|
||||
import com.uber.hoodie.DataSourceReadOptions._
|
||||
import com.uber.hoodie.DataSourceWriteOptions._
|
||||
import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import com.uber.hoodie.config.HoodieWriteConfig
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.commons.configuration.PropertiesConfiguration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.execution.datasources.DataSource
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Hoodie Spark Datasource, for reading and writing hoodie datasets
|
||||
*
|
||||
*/
|
||||
class DefaultSource extends RelationProvider
|
||||
with SchemaRelationProvider
|
||||
with CreatableRelationProvider
|
||||
with DataSourceRegister
|
||||
with Serializable {
|
||||
|
||||
private val log = LogManager.getLogger(classOf[DefaultSource])
|
||||
|
||||
override def createRelation(sqlContext: SQLContext,
|
||||
parameters: Map[String, String]): BaseRelation = {
|
||||
createRelation(sqlContext, parameters, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* Add default options for unspecified read options keys.
|
||||
*
|
||||
* @param parameters
|
||||
* @return
|
||||
*/
|
||||
def parametersWithReadDefaults(parameters: Map[String, String]) = {
|
||||
val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters))
|
||||
defaultsMap.putIfAbsent(VIEW_TYPE_OPT_KEY, DEFAULT_VIEW_TYPE_OPT_VAL)
|
||||
mapAsScalaMap(defaultsMap)
|
||||
}
|
||||
|
||||
override def createRelation(sqlContext: SQLContext,
|
||||
optParams: Map[String, String],
|
||||
schema: StructType): BaseRelation = {
|
||||
val parameters = parametersWithReadDefaults(optParams)
|
||||
val path = parameters.get("path")
|
||||
if (path.isEmpty) {
|
||||
throw new HoodieException("'path' must be specified.")
|
||||
}
|
||||
|
||||
if (parameters(VIEW_TYPE_OPT_KEY).equals(VIEW_TYPE_REALTIME_OPT_VAL)) {
|
||||
throw new HoodieException("Realtime view not supported yet via data source. Please use HiveContext route.")
|
||||
}
|
||||
|
||||
if (parameters(VIEW_TYPE_OPT_KEY).equals(VIEW_TYPE_INCREMENTAL_OPT_VAL)) {
|
||||
new IncrementalRelation(sqlContext, path.get, optParams, schema)
|
||||
} else {
|
||||
// this is just effectively RO view only, where `path` can contain a mix of
|
||||
// non-hoodie/hoodie path files. set the path filter up
|
||||
sqlContext.sparkContext.hadoopConfiguration.setClass(
|
||||
"mapreduce.input.pathFilter.class",
|
||||
classOf[com.uber.hoodie.hadoop.HoodieROTablePathFilter],
|
||||
classOf[org.apache.hadoop.fs.PathFilter]);
|
||||
|
||||
// simply return as a regular parquet relation
|
||||
DataSource.apply(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
userSpecifiedSchema = Option(schema),
|
||||
className = "parquet",
|
||||
options = parameters.toMap)
|
||||
.resolveRelation()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add default options for unspecified write options keys.
|
||||
*
|
||||
* @param parameters
|
||||
* @return
|
||||
*/
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]) = {
|
||||
val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters))
|
||||
defaultsMap.putIfAbsent(OPERATION_OPT_KEY, DEFAULT_OPERATION_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(STORAGE_TYPE_OPT_KEY, DEFAULT_STORAGE_TYPE_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(PRECOMBINE_FIELD_OPT_KEY, DEFAULT_PRECOMBINE_FIELD_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(PAYLOAD_CLASS_OPT_KEY, DEFAULT_PAYLOAD_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(RECORDKEY_FIELD_OPT_KEY, DEFAULT_RECORDKEY_FIELD_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL)
|
||||
mapAsScalaMap(defaultsMap)
|
||||
}
|
||||
|
||||
def toPropertiesConfiguration(params: Map[String, String]): PropertiesConfiguration = {
|
||||
val propCfg = new PropertiesConfiguration()
|
||||
params.foreach(kv => propCfg.addProperty(kv._1, kv._2))
|
||||
propCfg
|
||||
}
|
||||
|
||||
|
||||
override def createRelation(sqlContext: SQLContext,
|
||||
mode: SaveMode,
|
||||
optParams: Map[String, String],
|
||||
df: DataFrame): BaseRelation = {
|
||||
|
||||
val parameters = parametersWithWriteDefaults(optParams).toMap
|
||||
val path = parameters.get("path")
|
||||
val tblName = parameters.get(HoodieWriteConfig.TABLE_NAME)
|
||||
if (path.isEmpty || tblName.isEmpty) {
|
||||
throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.")
|
||||
}
|
||||
|
||||
val storageType = parameters(STORAGE_TYPE_OPT_KEY)
|
||||
val operation = parameters(OPERATION_OPT_KEY)
|
||||
|
||||
// register classes & schemas
|
||||
val structName = s"${tblName.get}_record"
|
||||
val nameSpace = s"hoodie.${tblName.get}"
|
||||
sqlContext.sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
sqlContext.sparkContext.getConf.registerAvroSchemas(schema)
|
||||
|
||||
// Convert to RDD[HoodieRecord]
|
||||
val keyGenerator = DataSourceUtils.createKeyGenerator(
|
||||
parameters(KEYGENERATOR_CLASS_OPT_KEY),
|
||||
toPropertiesConfiguration(parameters)
|
||||
)
|
||||
val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace)
|
||||
val hoodieRecords = genericRecords.map(gr => {
|
||||
val orderingVal = DataSourceUtils.getNestedFieldValAsString(
|
||||
gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]]
|
||||
DataSourceUtils.createHoodieRecord(gr,
|
||||
orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
}).toJavaRDD();
|
||||
|
||||
|
||||
val basePath = new Path(parameters.get("path").get)
|
||||
val fs = basePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
|
||||
var exists = fs.exists(basePath)
|
||||
|
||||
// Handle various save modes
|
||||
if (mode == SaveMode.ErrorIfExists && exists) {
|
||||
throw new HoodieException(s"basePath ${basePath} already exists.")
|
||||
}
|
||||
if (mode == SaveMode.Ignore && exists) {
|
||||
log.warn(s" basePath ${basePath} already exists. Ignoring & not performing actual writes.")
|
||||
return createRelation(sqlContext, parameters, df.schema)
|
||||
}
|
||||
if (mode == SaveMode.Overwrite && exists) {
|
||||
log.warn(s" basePath ${basePath} already exists. Deleting existing data & overwriting with new data.")
|
||||
fs.delete(basePath, true)
|
||||
exists = false
|
||||
}
|
||||
|
||||
// Create the dataset if not present (APPEND mode)
|
||||
if (!exists) {
|
||||
val properties = new Properties();
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tblName.get);
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, storageType);
|
||||
HoodieTableMetaClient.initializePathAsHoodieDataset(fs, path.get, properties);
|
||||
}
|
||||
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = DataSourceUtils.createHoodieClient(new JavaSparkContext(sqlContext.sparkContext),
|
||||
schema.toString,
|
||||
path.get,
|
||||
tblName.get,
|
||||
mapAsJavaMap(parameters)
|
||||
)
|
||||
val commitTime = client.startCommit();
|
||||
|
||||
val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, commitTime, operation)
|
||||
// Check for errors and commit the write.
|
||||
val errorCount = writeStatuses.rdd.filter(ws => ws.hasErrors).count()
|
||||
if (errorCount == 0) {
|
||||
log.info("No errors. Proceeding to commit the write.");
|
||||
val metaMap = parameters.filter(kv =>
|
||||
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY)))
|
||||
val success = if (metaMap.isEmpty) {
|
||||
client.commit(commitTime, writeStatuses)
|
||||
} else {
|
||||
client.commit(commitTime, writeStatuses,
|
||||
Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
}
|
||||
|
||||
if (success) {
|
||||
log.info("Commit " + commitTime + " successful!")
|
||||
}
|
||||
else {
|
||||
log.info("Commit " + commitTime + " failed!")
|
||||
}
|
||||
client.close
|
||||
} else {
|
||||
log.error(s"Upsert failed with ${errorCount} errors :");
|
||||
if (log.isTraceEnabled) {
|
||||
log.trace("Printing out the top 100 errors")
|
||||
writeStatuses.rdd.filter(ws => ws.hasErrors)
|
||||
.take(100)
|
||||
.foreach(ws => {
|
||||
log.trace("Global error :", ws.getGlobalError)
|
||||
if (ws.getErrors.size() > 0) {
|
||||
ws.getErrors.foreach(kt =>
|
||||
log.trace(s"Error for key: ${kt._1}", kt._2))
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
createRelation(sqlContext, parameters, df.schema)
|
||||
}
|
||||
|
||||
override def shortName(): String = "hoodie"
|
||||
}
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie
|
||||
|
||||
import com.uber.hoodie.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieTableType}
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient
|
||||
import com.uber.hoodie.common.util.ParquetUtils
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import com.uber.hoodie.table.HoodieTable
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.sources.{BaseRelation, TableScan}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{Row, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
* Relation, that implements the Hoodie incremental view.
|
||||
*
|
||||
* Implemented for Copy_on_write storage.
|
||||
*
|
||||
*/
|
||||
class IncrementalRelation(val sqlContext: SQLContext,
|
||||
val basePath: String,
|
||||
val optParams: Map[String, String],
|
||||
val userSchema: StructType) extends BaseRelation with TableScan {
|
||||
|
||||
private val log = LogManager.getLogger(classOf[IncrementalRelation])
|
||||
|
||||
val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
|
||||
val metaClient = new HoodieTableMetaClient(fs, basePath, true)
|
||||
// MOR datasets not supported yet
|
||||
if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets")
|
||||
}
|
||||
val hoodieTable = HoodieTable.getHoodieTable(metaClient, null)
|
||||
val commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
|
||||
if (commitTimeline.empty()) {
|
||||
throw new HoodieException("No instants to incrementally pull")
|
||||
}
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
|
||||
throw new HoodieException(s"Specify the begin instant time to pull from using " +
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
|
||||
}
|
||||
val commitsToReturn = commitTimeline.findInstantsInRange(
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY,
|
||||
commitTimeline.lastInstant().get().getTimestamp))
|
||||
.getInstants.iterator().toList
|
||||
|
||||
// use schema from a file produced in the latest instant
|
||||
val latestSchema = {
|
||||
val latestMeta = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get)
|
||||
val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next()
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema(new Path(metaFilePath)))
|
||||
}
|
||||
|
||||
override def schema: StructType = latestSchema
|
||||
|
||||
override def buildScan(): RDD[Row] = {
|
||||
val fileIdToFullPath = mutable.HashMap[String, String]()
|
||||
for (commit <- commitsToReturn) {
|
||||
val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get)
|
||||
fileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap
|
||||
}
|
||||
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
|
||||
sqlContext.read.options(sOpts)
|
||||
.parquet(fileIdToFullPath.values.toList: _*)
|
||||
.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp))
|
||||
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp))
|
||||
.toDF().rdd
|
||||
|
||||
}
|
||||
}
|
||||
36
hoodie-spark/src/main/scala/com/uber/hoodie/package.scala
Normal file
36
hoodie-spark/src/main/scala/com/uber/hoodie/package.scala
Normal file
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*
|
||||
*/
|
||||
package com.uber.hoodie
|
||||
|
||||
import org.apache.spark.sql.{DataFrame, DataFrameReader, DataFrameWriter}
|
||||
|
||||
package object hoodie {
|
||||
/**
|
||||
* Adds a method, `hoodie`, to DataFrameWriter
|
||||
*/
|
||||
implicit class AvroDataFrameWriter[T](writer: DataFrameWriter[T]) {
|
||||
def avro: String => Unit = writer.format("com.uber.hoodie").save
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a method, `hoodie`, to DataFrameReader
|
||||
*/
|
||||
implicit class AvroDataFrameReader(reader: DataFrameReader) {
|
||||
def avro: String => DataFrame = reader.format("com.uber.hoodie").load
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user