1
0

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:
Vinoth Chandar
2017-08-28 01:28:08 -07:00
committed by vinoth chandar
parent c98ee057fc
commit 64e0573aca
44 changed files with 1830 additions and 331 deletions

View File

@@ -21,7 +21,7 @@ package com.uber.hoodie.utilities;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.utilities.keygen.KeyGenerator;
import com.uber.hoodie.KeyGenerator;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.Source;
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
@@ -60,22 +60,6 @@ public class UtilHelpers {
}
}
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);
}
}
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);
}
}
/**
*
* TODO: Support hierarchical config files (see CONFIGURATION-609 for sample)
@@ -98,11 +82,4 @@ public class UtilHelpers {
}
}
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");
}
});
}
}

View File

@@ -1,67 +0,0 @@
/*
* 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.utilities.deltastreamer;
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 DeltaStreamerAvroPayload extends DeltaStreamerPayload implements HoodieRecordPayload<DeltaStreamerAvroPayload> {
/**
*
* @param record
* @param orderingVal
*/
public DeltaStreamerAvroPayload(GenericRecord record, Comparable orderingVal) {
super(record, orderingVal);
}
@Override
public DeltaStreamerAvroPayload preCombine(DeltaStreamerAvroPayload another) {
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));
}
}

View File

@@ -1,49 +0,0 @@
/*
* 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.utilities.deltastreamer;
import org.apache.avro.generic.GenericRecord;
import java.io.Serializable;
/**
* Base class for all payload supported for the {@link HoodieDeltaStreamer}
*/
public abstract class DeltaStreamerPayload 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 DeltaStreamerPayload(GenericRecord record, Comparable orderingVal) {
this.record = record;
this.orderingVal = orderingVal;
}
}

View File

@@ -22,7 +22,9 @@ import com.beust.jcommander.IStringConverter;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.ParameterException;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.OverwriteWithLatestAvroPayload;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -36,10 +38,10 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.utilities.HiveIncrementalPuller;
import com.uber.hoodie.utilities.UtilHelpers;
import com.uber.hoodie.utilities.keygen.SimpleKeyGenerator;
import com.uber.hoodie.SimpleKeyGenerator;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
import com.uber.hoodie.utilities.sources.DFSSource;
import com.uber.hoodie.utilities.keygen.KeyGenerator;
import com.uber.hoodie.KeyGenerator;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.Source;
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
@@ -148,7 +150,7 @@ public class HoodieDeltaStreamer implements Serializable {
private void initKeyGenerator() throws IOException {
PropertiesConfiguration keygenCfg = UtilHelpers.readConfig(fs, new Path(cfg.keyGeneratorProps));
log.info("Creating key generator " + cfg.keyGeneratorClass + " with configs : " + keygenCfg.toString());
this.keyGenerator = UtilHelpers.createKeyGenerator(cfg.keyGeneratorClass, keygenCfg);
this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, keygenCfg);
}
@@ -174,8 +176,6 @@ public class HoodieDeltaStreamer implements Serializable {
}
private void sync() throws Exception {
// Retrieve the previous round checkpoints, if any
Optional<String> resumeCheckpointStr = Optional.empty();
if (commitTimelineOpt.isPresent()) {
@@ -209,7 +209,7 @@ public class HoodieDeltaStreamer implements Serializable {
JavaRDD<GenericRecord> avroRDD = dataAndCheckpoint.getKey().get();
JavaRDD<HoodieRecord> records = avroRDD
.map(gr -> {
HoodieRecordPayload payload = UtilHelpers.createPayload(
HoodieRecordPayload payload = DataSourceUtils.createPayload(
cfg.payloadClassName,
gr,
(Comparable) gr.get(cfg.sourceOrderingField));
@@ -248,7 +248,6 @@ public class HoodieDeltaStreamer implements Serializable {
}
private HoodieWriteConfig getHoodieClientConfig(String hoodieClientCfgPath) throws Exception {
// TODO(vc): Double check all the options can be passed in like this. CompactionConfig, IndexConfig everything.
return HoodieWriteConfig.newBuilder()
.combineInput(true, true)
.withPath(cfg.targetBasePath)
@@ -322,7 +321,7 @@ public class HoodieDeltaStreamer implements Serializable {
@Parameter(names = {"--payload-class"}, description = "subclass of HoodieRecordPayload, that works off a GenericRecord. " +
"Default: SourceWrapperPayload. Implement your own, if you want to do something other than overwriting existing value")
public String payloadClassName = DeltaStreamerAvroPayload.class.getName();
public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
@Parameter(names = {"--schemaprovider-class"}, description = "subclass of com.uber.hoodie.utilities.schema.SchemaProvider " +
"to attach schemas to input & target table data, built in options: FilebasedSchemaProvider")
@@ -349,7 +348,6 @@ public class HoodieDeltaStreamer implements Serializable {
public static void main(String[] args) throws Exception {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, args);
// TODO(vc): Do proper validation
if (cfg.help || args.length == 0) {
cmd.usage();
System.exit(1);

View File

@@ -1,47 +0,0 @@
/*
* 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.utilities.keygen;
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);
}

View File

@@ -1,58 +0,0 @@
/*
* 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.utilities.keygen;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.utilities.UtilHelpers;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.configuration.PropertiesConfiguration;
import java.util.Arrays;
/**
* 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;
/**
* Supported configs
*/
static class Config {
private static final String RECORD_KEY_FIELD_PROP = "hoodie.deltastreamer.keygen.simple.recordkey.field";
private static final String PARTITION_PATH_FIELD_PROP = "hoodie.deltastreamer.keygen.simple.partitionpath.field";
}
public SimpleKeyGenerator(PropertiesConfiguration config) {
super(config);
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.PARTITION_PATH_FIELD_PROP, Config.RECORD_KEY_FIELD_PROP));
this.recordKeyField = config.getString(Config.RECORD_KEY_FIELD_PROP);
this.partitionPathField = config.getString(Config.PARTITION_PATH_FIELD_PROP);
}
@Override
public HoodieKey getKey(GenericRecord record) {
return new HoodieKey(record.get(recordKeyField).toString(), record.get(partitionPathField).toString());
}
}

View File

@@ -18,9 +18,10 @@
package com.uber.hoodie.utilities.keygen;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.SimpleKeyGenerator;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.utilities.UtilHelpers;
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
import org.apache.avro.generic.GenericRecord;
@@ -30,7 +31,6 @@ import java.io.Serializable;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Calendar;
import java.util.Date;
import java.util.TimeZone;
@@ -65,12 +65,12 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
public TimestampBasedKeyGenerator(PropertiesConfiguration config) {
super(config);
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
this.inputDateFormat = new SimpleDateFormat(config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
}

View File

@@ -18,9 +18,9 @@
package com.uber.hoodie.utilities.schema;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.utilities.UtilHelpers;
import org.apache.avro.Schema;
import org.apache.commons.configuration.PropertiesConfiguration;
@@ -53,7 +53,7 @@ public class FilebasedSchemaProvider extends SchemaProvider {
super(config);
this.fs = FSUtils.getFs();
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP));
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP));
try {
this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP))));
this.targetSchema = new Schema.Parser().parse(fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP))));

View File

@@ -18,10 +18,10 @@
package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.utilities.UtilHelpers;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import org.apache.avro.generic.GenericRecord;
@@ -66,7 +66,7 @@ public class DFSSource extends Source {
public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
super(config, sparkContext, dataFormat, schemaProvider);
this.fs = FSUtils.getFs();
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
}

View File

@@ -18,9 +18,9 @@
package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.utilities.UtilHelpers;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import org.apache.avro.generic.GenericRecord;
@@ -74,7 +74,7 @@ public class HiveIncrPullSource extends Source {
public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
super(config, sparkContext, dataFormat, schemaProvider);
this.fs = FSUtils.getFs();
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP);
}

View File

@@ -18,8 +18,8 @@
package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.utilities.UtilHelpers;
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
import com.uber.hoodie.utilities.schema.SchemaProvider;
@@ -175,7 +175,7 @@ public class KafkaSource extends Source {
Stream<String> keys = StreamSupport.stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false);
keys.forEach(k -> kafkaParams.put(k, config.getString(k)));
UtilHelpers.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME));
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME));
topicName = config.getString(Config.KAFKA_TOPIC_NAME);
}

View File

@@ -16,5 +16,5 @@
#
#
hoodie.deltastreamer.keygen.simple.recordkey.field=_row_key
hoodie.deltastreamer.keygen.simple.partitionpath.field=driver
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.partitionpath.field=driver

View File

@@ -1,43 +0,0 @@
/*
* 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 org.apache.spark.sql.SparkSession;
/**
* Examples to do Spark SQL on Hoodie dataset.
*/
public class HoodieSparkSQLExample {
public static void main(String[] args) throws Exception {
SparkSession spark = SparkSession.builder()
.appName("Hoodie SparkSQL")
.config("hive.metastore.uris","thrift://localhost:10000")
.config("spark.sql.hive.convertMetastoreParquet", false)
.enableHiveSupport()
.master("local[2]")
.getOrCreate();
spark.sql("describe hoodie_rt").show();
spark.sql("select * from hoodie_rt").show();
spark.sql("select end_lon as e1, driver, rider as r1, datestr, driver, datestr, rider, _hoodie_record_key from hoodie_rt").show();
spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_rt where fare > 2.0").show();
spark.sql("select count(*) as cnt, _hoodie_file_name as file from hoodie_rt group by _hoodie_file_name").show();
}
}