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

@@ -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));
}
}