New Features in DeltaStreamer :
(1) Apply transformation when using delta-streamer to ingest data. (2) Add Hudi Incremental Source for Delta Streamer (3) Allow delta-streamer config-property to be passed as command-line (4) Add Hive Integration to Delta-Streamer and address Review comments (5) Ensure MultiPartKeysValueExtractor handle hive style partition description (6) Reuse same spark session on both source and transformer (7) Support extracting partition fields from _hoodie_partition_path for HoodieIncrSource (8) Reuse Binary Avro coders (9) Add push down filter for Incremental source (10) Add Hoodie DeltaStreamer metrics to track total time taken
This commit is contained in:
committed by
vinoth chandar
parent
c70dbc13e9
commit
3a0044216c
@@ -15,47 +15,58 @@
|
||||
"fields":[
|
||||
{
|
||||
"name":"fileId",
|
||||
"type":["null","string"]
|
||||
"type":["null","string"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"path",
|
||||
"type":["null","string"]
|
||||
"type":["null","string"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"prevCommit",
|
||||
"type":["null","string"]
|
||||
"type":["null","string"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"numWrites",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"numDeletes",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"numUpdateWrites",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalWriteBytes",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalWriteErrors",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"partitionPath",
|
||||
"type":["null","string"]
|
||||
"type":["null","string"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalLogRecords",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalLogFiles",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalUpdatedRecordsCompacted",
|
||||
@@ -69,15 +80,18 @@
|
||||
},
|
||||
{
|
||||
"name":"totalLogBlocks",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalCorruptLogBlock",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"totalRollbackBlocks",
|
||||
"type":["null","long"]
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
}
|
||||
]
|
||||
}
|
||||
|
||||
@@ -17,7 +17,9 @@
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
@@ -31,6 +33,14 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
|
||||
public static String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path";
|
||||
public static String FILENAME_METADATA_FIELD = "_hoodie_file_name";
|
||||
|
||||
public static final List<String> HOODIE_META_COLUMNS =
|
||||
new ImmutableList.Builder<String>().add(COMMIT_TIME_METADATA_FIELD)
|
||||
.add(COMMIT_SEQNO_METADATA_FIELD)
|
||||
.add(RECORD_KEY_METADATA_FIELD)
|
||||
.add(PARTITION_PATH_METADATA_FIELD)
|
||||
.add(FILENAME_METADATA_FIELD)
|
||||
.build();
|
||||
|
||||
/**
|
||||
* Identifies the record across the table
|
||||
*/
|
||||
|
||||
@@ -73,6 +73,20 @@ public class DFSPropertiesConfiguration {
|
||||
}
|
||||
visitedFiles.add(file.getName());
|
||||
BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file)));
|
||||
addProperties(reader);
|
||||
} catch (IOException ioe) {
|
||||
log.error("Error reading in properies from dfs", ioe);
|
||||
throw new IllegalArgumentException("Cannot read properties from dfs", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add properties from input stream
|
||||
* @param reader Buffered Reader
|
||||
* @throws IOException
|
||||
*/
|
||||
public void addProperties(BufferedReader reader) throws IOException {
|
||||
try {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (line.startsWith("#") || line.equals("") || !line.contains("=")) {
|
||||
@@ -85,10 +99,8 @@ public class DFSPropertiesConfiguration {
|
||||
props.setProperty(split[0], split[1]);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
reader.close();
|
||||
} catch (IOException ioe) {
|
||||
log.error("Error reading in properies from dfs", ioe);
|
||||
throw new IllegalArgumentException("Cannot read properties from dfs", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -37,8 +37,8 @@ import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericDatumWriter;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.BinaryDecoder;
|
||||
import org.apache.avro.io.BinaryEncoder;
|
||||
import org.apache.avro.io.Decoder;
|
||||
import org.apache.avro.io.DecoderFactory;
|
||||
import org.apache.avro.io.EncoderFactory;
|
||||
import org.codehaus.jackson.JsonNode;
|
||||
@@ -48,6 +48,10 @@ import org.codehaus.jackson.JsonNode;
|
||||
*/
|
||||
public class HoodieAvroUtils {
|
||||
|
||||
private static ThreadLocal<BinaryEncoder> reuseEncoder = ThreadLocal.withInitial(() -> null);
|
||||
|
||||
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
|
||||
|
||||
// All metadata fields are optional strings.
|
||||
private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
|
||||
Schema.create(Schema.Type.NULL),
|
||||
@@ -62,7 +66,8 @@ public class HoodieAvroUtils {
|
||||
GenericDatumWriter<GenericRecord> writer =
|
||||
new GenericDatumWriter<>(record.getSchema());
|
||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
|
||||
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, reuseEncoder.get());
|
||||
reuseEncoder.set(encoder);
|
||||
writer.write(record, encoder);
|
||||
encoder.flush();
|
||||
out.close();
|
||||
@@ -73,7 +78,8 @@ public class HoodieAvroUtils {
|
||||
* Convert serialized bytes back into avro record
|
||||
*/
|
||||
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException {
|
||||
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
|
||||
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, reuseDecoder.get());
|
||||
reuseDecoder.set(decoder);
|
||||
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
|
||||
return reader.read(null, decoder);
|
||||
}
|
||||
|
||||
@@ -19,7 +19,10 @@
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Type-aware extension of {@link java.util.Properties}
|
||||
@@ -49,6 +52,13 @@ public class TypedProperties extends Properties implements Serializable {
|
||||
return containsKey(property) ? getProperty(property) : defaultValue;
|
||||
}
|
||||
|
||||
public List<String> getStringList(String property, String delimiter, List<String> defaultVal) {
|
||||
if (!containsKey(property)) {
|
||||
return defaultVal;
|
||||
}
|
||||
return Arrays.stream(getProperty(property).split(delimiter)).map(String::trim).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public int getInteger(String property) {
|
||||
checkKey(property);
|
||||
return Integer.valueOf(getProperty(property));
|
||||
|
||||
@@ -0,0 +1,209 @@
|
||||
/*
|
||||
* Copyright (c) 2016 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.common.util.collection;
|
||||
|
||||
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.common.model.AvroBinaryTestPayload;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
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.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
import com.uber.hoodie.common.util.SpillableMapTestUtils;
|
||||
import com.uber.hoodie.common.util.SpillableMapUtils;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestDiskBasedMap {
|
||||
|
||||
private static final String BASE_OUTPUT_PATH = "/tmp/";
|
||||
|
||||
@Test
|
||||
public void testSimpleInsert() throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
|
||||
// make sure records have spilled to disk
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
List<HoodieRecord> oRecords = new ArrayList<>();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
oRecords.add(rec);
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
|
||||
Schema schema = getSimpleSchema();
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH);
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil
|
||||
.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
|
||||
Set<String> recordKeys = new HashSet<>();
|
||||
// insert generated records into the map
|
||||
hoodieRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
recordKeys.add(r.getRecordKey());
|
||||
});
|
||||
// make sure records have spilled to disk
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
List<HoodieRecord> oRecords = new ArrayList<>();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
oRecords.add(rec);
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleUpsert() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
|
||||
// perform some inserts
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
|
||||
long fileSize = records.sizeOfFileOnDiskInBytes();
|
||||
// make sure records have spilled to disk
|
||||
assertTrue(fileSize > 0);
|
||||
|
||||
// generate updates from inserts
|
||||
List<IndexedRecord> updatedRecords =
|
||||
SchemaTestUtil
|
||||
.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100),
|
||||
HoodieActiveTimeline.createNewCommitTime());
|
||||
String newCommitTime = ((GenericRecord) updatedRecords.get(0))
|
||||
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
|
||||
// perform upserts
|
||||
recordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
|
||||
// upserts should be appended to the existing file, hence increasing the sizeOfFile on disk
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > fileSize);
|
||||
|
||||
// Upserted records (on disk) should have the latest commit time
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
try {
|
||||
IndexedRecord indexedRecord = (IndexedRecord) rec.getData().getInsertValue(schema).get();
|
||||
String latestCommitTime = ((GenericRecord) indexedRecord)
|
||||
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
assertEquals(latestCommitTime, newCommitTime);
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSizeEstimator() throws IOException, URISyntaxException {
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
|
||||
// Test sizeEstimator without hoodie metadata fields
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
|
||||
long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Test sizeEstimator with hoodie metadata fields
|
||||
schema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Following tests payloads without an Avro Schema in the Record
|
||||
|
||||
// Test sizeEstimator without hoodie metadata fields and without schema object in the payload
|
||||
schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Test sizeEstimator with hoodie metadata fields and without schema object in the payload
|
||||
final Schema simpleSchemaWithMetadata = HoodieAvroUtils
|
||||
.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Optional
|
||||
.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
|
||||
.collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* @na: Leaving this test here for a quick performance test
|
||||
*/
|
||||
@Ignore
|
||||
@Test
|
||||
public void testSizeEstimatorPerformance() throws IOException, URISyntaxException {
|
||||
// Test sizeEstimatorPerformance with simpleSchema
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
HoodieRecordSizeEstimator sizeEstimator =
|
||||
new HoodieRecordSizeEstimator(schema);
|
||||
HoodieRecord record = hoodieRecords.remove(0);
|
||||
long startTime = System.currentTimeMillis();
|
||||
SpillableMapUtils.computePayloadSize(record, sizeEstimator);
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
System.out.println("Time taken :" + timeTaken);
|
||||
assertTrue(timeTaken < 100);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user