1
0

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:
Balaji Varadarajan
2018-10-10 10:31:34 -07:00
committed by vinoth chandar
parent c70dbc13e9
commit 3a0044216c
65 changed files with 2752 additions and 911 deletions

View File

@@ -85,8 +85,12 @@ Usage: <main class> [options]
exist first time around. If exists, expected to be a hoodie dataset) exist first time around. If exists, expected to be a hoodie dataset)
* --target-table * --target-table
name of the target table in Hive name of the target table in Hive
--transformer-class
subclass of com.uber.hoodie.utilities.transform.Transformer. UDF to
transform raw source dataset to a target dataset (conforming to target
schema) before writing. Default : Not set. E:g -
com.uber.hoodie.utilities.transform.SqlQueryBasedTransformer (which
allows a SQL query template to be passed as a transformation function)
``` ```

View File

@@ -33,7 +33,7 @@ Hoodie requires Java 8 to be installed. Hoodie works with Spark-2.x versions. We
| Hadoop | Hive | Spark | Instructions to Build Hoodie | | Hadoop | Hive | Spark | Instructions to Build Hoodie |
| ---- | ----- | ---- | ---- | | ---- | ----- | ---- | ---- |
| 2.6.0-cdh5.7.2 | 1.1.0-cdh5.7.2 | spark-2.[1-3].x | Use "mvn clean install -DskipTests -Dhive11". Jars will have ".hive11" as suffix | | 2.6.0-cdh5.7.2 | 1.1.0-cdh5.7.2 | spark-2.[1-3].x | Use "mvn clean install -DskipTests -Dhadoop.version=2.6.0-cdh5.7.2 -Dhive.version=1.1.0-cdh5.7.2" |
| Apache hadoop-2.8.4 | Apache hive-2.3.3 | spark-2.[1-3].x | Use "mvn clean install -DskipTests" | | Apache hadoop-2.8.4 | Apache hive-2.3.3 | spark-2.[1-3].x | Use "mvn clean install -DskipTests" |
| Apache hadoop-2.7.3 | Apache hive-1.2.1 | spark-2.[1-3].x | Use "mvn clean install -DskipTests" | | Apache hadoop-2.7.3 | Apache hive-1.2.1 | spark-2.[1-3].x | Use "mvn clean install -DskipTests" |

View File

@@ -17,6 +17,7 @@
package com.uber.hoodie.cli.commands; package com.uber.hoodie.cli.commands;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry; import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.avro.model.HoodieCommitMetadata;
import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper; import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.cli.TableHeader;
@@ -32,6 +33,7 @@ import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.specific.SpecificData;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
@@ -48,6 +50,92 @@ public class ArchivedCommitsCommand implements CommandMarker {
return HoodieCLI.tableMetadata != null; return HoodieCLI.tableMetadata != null;
} }
@CliCommand(value = "show archived commit stats", help = "Read commits from archived files and show details")
public String showArchivedCommits(
@CliOption(key = {"archiveFolderPattern"}, help = "Archive Folder", unspecifiedDefaultValue = "") String folder,
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
@CliOption(key = {
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
throws IOException {
System.out.println("===============> Showing only " + limit + " archived commits <===============");
String basePath = HoodieCLI.tableMetadata.getBasePath();
Path archivePath = new Path(basePath + "/.hoodie/.commits_.archive*");
if (folder != null && !folder.isEmpty()) {
archivePath = new Path(basePath + "/.hoodie/" + folder);
}
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
List<Comparable[]> allStats = new ArrayList<>();
for (FileStatus fs : fsStatuses) {
//read the archived file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
List<IndexedRecord> readRecords = new ArrayList<>();
//read the avro blocks
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
List<IndexedRecord> records = blk.getRecords();
readRecords.addAll(records);
}
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
.filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION)
|| r.get("actionType").toString().equals(HoodieTimeline.DELTA_COMMIT_ACTION))
.flatMap(r -> {
HoodieCommitMetadata metadata =
(HoodieCommitMetadata) SpecificData.get().deepCopy(HoodieCommitMetadata.SCHEMA$,
r.get("hoodieCommitMetadata"));
final String instantTime = r.get("commitTime").toString();
final String action = r.get("actionType").toString();
return metadata.getPartitionToWriteStats().values().stream().flatMap(hoodieWriteStats -> {
return hoodieWriteStats.stream().map(hoodieWriteStat -> {
List<Comparable> row = new ArrayList<>();
row.add(action);
row.add(instantTime);
row.add(hoodieWriteStat.getPartitionPath());
row.add(hoodieWriteStat.getFileId());
row.add(hoodieWriteStat.getPrevCommit());
row.add(hoodieWriteStat.getNumWrites());
row.add(hoodieWriteStat.getNumInserts());
row.add(hoodieWriteStat.getNumDeletes());
row.add(hoodieWriteStat.getNumUpdateWrites());
row.add(hoodieWriteStat.getTotalLogFiles());
row.add(hoodieWriteStat.getTotalLogBlocks());
row.add(hoodieWriteStat.getTotalCorruptLogBlock());
row.add(hoodieWriteStat.getTotalRollbackBlocks());
row.add(hoodieWriteStat.getTotalLogRecords());
row.add(hoodieWriteStat.getTotalUpdatedRecordsCompacted());
row.add(hoodieWriteStat.getTotalWriteBytes());
row.add(hoodieWriteStat.getTotalWriteErrors());
return row;
});
}).map(rowList -> rowList.toArray(new Comparable[0]));
}).collect(Collectors.toList());
allStats.addAll(readCommits);
reader.close();
}
TableHeader header = new TableHeader().addTableHeaderField("action")
.addTableHeaderField("instant")
.addTableHeaderField("partition")
.addTableHeaderField("file_id")
.addTableHeaderField("prev_instant")
.addTableHeaderField("num_writes")
.addTableHeaderField("num_inserts")
.addTableHeaderField("num_deletes")
.addTableHeaderField("num_update_writes")
.addTableHeaderField("total_log_files")
.addTableHeaderField("total_log_blocks")
.addTableHeaderField("total_corrupt_log_blocks")
.addTableHeaderField("total_rollback_blocks")
.addTableHeaderField("total_log_records")
.addTableHeaderField("total_updated_records_compacted")
.addTableHeaderField("total_write_bytes")
.addTableHeaderField("total_write_errors");
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allStats);
}
@CliCommand(value = "show archived commits", help = "Read commits from archived files and show details") @CliCommand(value = "show archived commits", help = "Read commits from archived files and show details")
public String showCommits( public String showCommits(
@CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata", unspecifiedDefaultValue = "true") @CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata", unspecifiedDefaultValue = "true")

View File

@@ -171,6 +171,13 @@
<artifactId>spark-sql_2.11</artifactId> <artifactId>spark-sql_2.11</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive.version}</version>
<scope>test</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.hbase</groupId> <groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId> <artifactId>hbase-client</artifactId>
@@ -218,39 +225,4 @@
</exclusions> </exclusions>
</dependency> </dependency>
</dependencies> </dependencies>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive12.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive11.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -16,8 +16,8 @@
package com.uber.hoodie.metrics; package com.uber.hoodie.metrics;
import com.codahale.metrics.Gauge; import static com.uber.hoodie.metrics.Metrics.registerGauge;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer; import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieCommitMetadata;
@@ -177,18 +177,6 @@ public class HoodieMetrics {
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric); return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
} }
void registerGauge(String metricName, final long value) {
try {
MetricRegistry registry = Metrics.getInstance().getRegistry();
registry.register(metricName, (Gauge<Long>) () -> value);
} catch (Exception e) {
// Here we catch all exception, so the major upsert pipeline will not be affected if the
// metrics system
// has some issues.
logger.error("Failed to send metrics: ", e);
}
}
/** /**
* By default, the timer context returns duration with nano seconds. Convert it to millisecond. * By default, the timer context returns duration with nano seconds. Convert it to millisecond.
*/ */

View File

@@ -16,17 +16,21 @@
package com.uber.hoodie.metrics; package com.uber.hoodie.metrics;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.MetricRegistry;
import com.google.common.io.Closeables; import com.google.common.io.Closeables;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import java.io.Closeable; import java.io.Closeable;
import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.configuration.ConfigurationException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/** /**
* This is the main class of the metrics system. * This is the main class of the metrics system.
*/ */
public class Metrics { public class Metrics {
private static Logger logger = LogManager.getLogger(Metrics.class);
private static volatile boolean initialized = false; private static volatile boolean initialized = false;
private static Metrics metrics = null; private static Metrics metrics = null;
@@ -72,6 +76,18 @@ public class Metrics {
initialized = true; initialized = true;
} }
public static void registerGauge(String metricName, final long value) {
try {
MetricRegistry registry = Metrics.getInstance().getRegistry();
registry.register(metricName, (Gauge<Long>) () -> value);
} catch (Exception e) {
// Here we catch all exception, so the major upsert pipeline will not be affected if the
// metrics system
// has some issues.
logger.error("Failed to send metrics: ", e);
}
}
public MetricRegistry getRegistry() { public MetricRegistry getRegistry() {
return registry; return registry;
} }

View File

@@ -75,7 +75,9 @@ public class HoodieTestDataGenerator {
+ "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lat\", \"type\": \"double\"},"
+ "{\"name\": \"end_lon\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"},"
+ "{\"name\":\"fare\",\"type\": \"double\"}]}"; + "{\"name\":\"fare\",\"type\": \"double\"}]}";
public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); public static Schema avroSchema = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
public static Schema avroSchemaWithMetadataFields = HoodieAvroUtils.addMetadataFields(avroSchema);
private static Random rand = new Random(46474747); private static Random rand = new Random(46474747);
private List<KeyPartition> existingKeysList = new ArrayList<>(); private List<KeyPartition> existingKeysList = new ArrayList<>();
@@ -100,7 +102,6 @@ public class HoodieTestDataGenerator {
*/ */
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException { public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0); GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA); return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
} }

View File

@@ -160,7 +160,8 @@ public class TestHoodieCompactor {
// Write them to corresponding avro logfiles // Write them to corresponding avro logfiles
HoodieTestUtils HoodieTestUtils
.writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, updatedRecords); .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchemaWithMetadataFields,
updatedRecords);
// Verify that all data file has one log file // Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.metrics; package com.uber.hoodie.metrics;
import static com.uber.hoodie.metrics.Metrics.registerGauge;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
@@ -39,7 +40,7 @@ public class TestHoodieMetrics {
@Test @Test
public void testRegisterGauge() { public void testRegisterGauge() {
metrics.registerGauge("metric1", 123L); registerGauge("metric1", 123L);
assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123")); assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123"));
} }
} }

View File

@@ -671,8 +671,8 @@ public class TestMergeOnReadTable {
// Write them to corresponding avro logfiles // Write them to corresponding avro logfiles
HoodieTestUtils HoodieTestUtils
.writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(),
updatedRecords); HoodieTestDataGenerator.avroSchemaWithMetadataFields, updatedRecords);
// Verify that all data file has one log file // Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);

View File

@@ -15,47 +15,58 @@
"fields":[ "fields":[
{ {
"name":"fileId", "name":"fileId",
"type":["null","string"] "type":["null","string"],
"default" : null
}, },
{ {
"name":"path", "name":"path",
"type":["null","string"] "type":["null","string"],
"default" : null
}, },
{ {
"name":"prevCommit", "name":"prevCommit",
"type":["null","string"] "type":["null","string"],
"default" : null
}, },
{ {
"name":"numWrites", "name":"numWrites",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"numDeletes", "name":"numDeletes",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"numUpdateWrites", "name":"numUpdateWrites",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"totalWriteBytes", "name":"totalWriteBytes",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"totalWriteErrors", "name":"totalWriteErrors",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"partitionPath", "name":"partitionPath",
"type":["null","string"] "type":["null","string"],
"default" : null
}, },
{ {
"name":"totalLogRecords", "name":"totalLogRecords",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"totalLogFiles", "name":"totalLogFiles",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"totalUpdatedRecordsCompacted", "name":"totalUpdatedRecordsCompacted",
@@ -69,15 +80,18 @@
}, },
{ {
"name":"totalLogBlocks", "name":"totalLogBlocks",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"totalCorruptLogBlock", "name":"totalCorruptLogBlock",
"type":["null","long"] "type":["null","long"],
"default" : null
}, },
{ {
"name":"totalRollbackBlocks", "name":"totalRollbackBlocks",
"type":["null","long"] "type":["null","long"],
"default" : null
} }
] ]
} }

View File

@@ -17,7 +17,9 @@
package com.uber.hoodie.common.model; package com.uber.hoodie.common.model;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.collect.ImmutableList;
import java.io.Serializable; import java.io.Serializable;
import java.util.List;
import java.util.Optional; 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 PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path";
public static String FILENAME_METADATA_FIELD = "_hoodie_file_name"; 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 * Identifies the record across the table
*/ */

View File

@@ -73,6 +73,20 @@ public class DFSPropertiesConfiguration {
} }
visitedFiles.add(file.getName()); visitedFiles.add(file.getName());
BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file))); 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; String line;
while ((line = reader.readLine()) != null) { while ((line = reader.readLine()) != null) {
if (line.startsWith("#") || line.equals("") || !line.contains("=")) { if (line.startsWith("#") || line.equals("") || !line.contains("=")) {
@@ -85,10 +99,8 @@ public class DFSPropertiesConfiguration {
props.setProperty(split[0], split[1]); props.setProperty(split[0], split[1]);
} }
} }
} finally {
reader.close(); reader.close();
} catch (IOException ioe) {
log.error("Error reading in properies from dfs", ioe);
throw new IllegalArgumentException("Cannot read properties from dfs", ioe);
} }
} }

View File

@@ -37,8 +37,8 @@ import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.EncoderFactory; import org.apache.avro.io.EncoderFactory;
import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.JsonNode;
@@ -48,6 +48,10 @@ import org.codehaus.jackson.JsonNode;
*/ */
public class HoodieAvroUtils { 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. // All metadata fields are optional strings.
private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList( private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.NULL),
@@ -62,7 +66,8 @@ public class HoodieAvroUtils {
GenericDatumWriter<GenericRecord> writer = GenericDatumWriter<GenericRecord> writer =
new GenericDatumWriter<>(record.getSchema()); new GenericDatumWriter<>(record.getSchema());
ByteArrayOutputStream out = new ByteArrayOutputStream(); 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); writer.write(record, encoder);
encoder.flush(); encoder.flush();
out.close(); out.close();
@@ -73,7 +78,8 @@ public class HoodieAvroUtils {
* Convert serialized bytes back into avro record * Convert serialized bytes back into avro record
*/ */
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException { 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); GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
return reader.read(null, decoder); return reader.read(null, decoder);
} }

View File

@@ -19,7 +19,10 @@
package com.uber.hoodie.common.util; package com.uber.hoodie.common.util;
import java.io.Serializable; import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.stream.Collectors;
/** /**
* Type-aware extension of {@link java.util.Properties} * 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; 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) { public int getInteger(String property) {
checkKey(property); checkKey(property);
return Integer.valueOf(getProperty(property)); return Integer.valueOf(getProperty(property));

View File

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

View File

@@ -60,6 +60,22 @@
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId> <artifactId>hadoop-hdfs</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency> <dependency>
<groupId>commons-logging</groupId> <groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId> <artifactId>commons-logging</artifactId>
@@ -105,58 +121,4 @@
</plugin> </plugin>
</plugins> </plugins>
</build> </build>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive12.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive11.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -99,6 +99,27 @@
<groupId>junit</groupId> <groupId>junit</groupId>
<artifactId>junit</artifactId> <artifactId>junit</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId> <artifactId>hadoop-hdfs</artifactId>
@@ -175,67 +196,4 @@
</plugin> </plugin>
</plugins> </plugins>
</build> </build>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive12.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-service</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
<version>${hive11.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -90,7 +90,7 @@ public class HoodieHiveClient {
private Connection connection; private Connection connection;
private HoodieTimeline activeTimeline; private HoodieTimeline activeTimeline;
HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
this.syncConfig = cfg; this.syncConfig = cfg;
this.fs = fs; this.fs = fs;
this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true); this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true);
@@ -231,7 +231,7 @@ public class HoodieHiveClient {
/** /**
* Scan table partitions * Scan table partitions
*/ */
List<Partition> scanTablePartitions() throws TException { public List<Partition> scanTablePartitions() throws TException {
return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1);
} }
@@ -268,7 +268,7 @@ public class HoodieHiveClient {
/** /**
* Get the table schema * Get the table schema
*/ */
Map<String, String> getTableSchema() { public Map<String, String> getTableSchema() {
if (!doesTableExist()) { if (!doesTableExist()) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"Failed to get schema for table " + syncConfig.tableName + " does not exist"); "Failed to get schema for table " + syncConfig.tableName + " does not exist");
@@ -435,7 +435,7 @@ public class HoodieHiveClient {
/** /**
* @return true if the configured table exists * @return true if the configured table exists
*/ */
boolean doesTableExist() { public boolean doesTableExist() {
try { try {
return client.tableExists(syncConfig.databaseName, syncConfig.tableName); return client.tableExists(syncConfig.databaseName, syncConfig.tableName);
} catch (TException e) { } catch (TException e) {
@@ -449,7 +449,7 @@ public class HoodieHiveClient {
* *
* @param s SQL to execute * @param s SQL to execute
*/ */
void updateHiveSQL(String s) { public void updateHiveSQL(String s) {
Statement stmt = null; Statement stmt = null;
try { try {
stmt = connection.createStatement(); stmt = connection.createStatement();
@@ -468,8 +468,10 @@ public class HoodieHiveClient {
BasicDataSource ds = new HiveDataSource(); BasicDataSource ds = new HiveDataSource();
ds.setDriverClassName(HiveDriver.class.getCanonicalName()); ds.setDriverClassName(HiveDriver.class.getCanonicalName());
ds.setUrl(getHiveJdbcUrlWithDefaultDBName()); ds.setUrl(getHiveJdbcUrlWithDefaultDBName());
if (syncConfig.hiveUser != null) {
ds.setUsername(syncConfig.hiveUser); ds.setUsername(syncConfig.hiveUser);
ds.setPassword(syncConfig.hivePass); ds.setPassword(syncConfig.hivePass);
}
LOG.info("Getting Hive Connection from Datasource " + ds); LOG.info("Getting Hive Connection from Datasource " + ds);
try { try {
this.connection = ds.getConnection(); this.connection = ds.getConnection();
@@ -520,7 +522,7 @@ public class HoodieHiveClient {
return fs; return fs;
} }
Optional<String> getLastCommitTimeSynced() { public Optional<String> getLastCommitTimeSynced() {
// Get the last commit time from the TBLproperties // Get the last commit time from the TBLproperties
try { try {
Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName); Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName);
@@ -532,7 +534,7 @@ public class HoodieHiveClient {
} }
} }
void close() { public void close() {
try { try {
if (connection != null) { if (connection != null) {
connection.close(); connection.close();
@@ -548,7 +550,7 @@ public class HoodieHiveClient {
@SuppressWarnings("OptionalUsedAsFieldOrParameterType") @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
List<String> getPartitionsWrittenToSince(Optional<String> lastCommitTimeSynced) { List<String> getPartitionsWrittenToSince(Optional<String> lastCommitTimeSynced) {
if (!lastCommitTimeSynced.isPresent()) { if (!lastCommitTimeSynced.isPresent()) {
LOG.info("Last commit time synced is not known, listing all partitions"); LOG.info("Last commit time synced is not known, listing all partitions in " + syncConfig.basePath + ",FS :" + fs);
try { try {
return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath, return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath,
syncConfig.assumeDatePartitioning); syncConfig.assumeDatePartitioning);
@@ -573,6 +575,10 @@ public class HoodieHiveClient {
} }
} }
List<String> getAllTables(String db) throws Exception {
return client.getAllTables(db);
}
void updateLastCommitTimeSynced() { void updateLastCommitTimeSynced() {
// Set the last commit time from the TBLproperties // Set the last commit time from the TBLproperties
String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp(); String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp();

View File

@@ -16,8 +16,10 @@
package com.uber.hoodie.hive; package com.uber.hoodie.hive;
import com.google.common.base.Preconditions;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.stream.Collectors;
/** /**
* Partition Key extractor treating each value delimited by slash as separate key. * Partition Key extractor treating each value delimited by slash as separate key.
@@ -27,6 +29,14 @@ public class MultiPartKeysValueExtractor implements PartitionValueExtractor {
@Override @Override
public List<String> extractPartitionValuesInPath(String partitionPath) { public List<String> extractPartitionValuesInPath(String partitionPath) {
String[] splits = partitionPath.split("/"); String[] splits = partitionPath.split("/");
return Arrays.asList(splits); return Arrays.stream(splits).map(s -> {
if (s.contains("=")) {
String[] moreSplit = s.split("=");
Preconditions.checkArgument(moreSplit.length == 2,
"Partition Field (" + s + ") not in expected format");
return moreSplit[1];
}
return s;
}).collect(Collectors.toList());
} }
} }

View File

@@ -18,6 +18,7 @@
package com.uber.hoodie.hive; package com.uber.hoodie.hive;
import java.io.Serializable;
import java.util.List; import java.util.List;
/** /**
@@ -28,7 +29,7 @@ import java.util.List;
* e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path
* /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] * /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd]
*/ */
public interface PartitionValueExtractor { public interface PartitionValueExtractor extends Serializable {
List<String> extractPartitionValuesInPath(String partitionPath); List<String> extractPartitionValuesInPath(String partitionPath);
} }

View File

@@ -33,12 +33,19 @@ import org.joda.time.format.DateTimeFormatter;
*/ */
public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor { public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor {
private final DateTimeFormatter dtfOut; private transient DateTimeFormatter dtfOut;
public SlashEncodedDayPartitionValueExtractor() { public SlashEncodedDayPartitionValueExtractor() {
this.dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd"); this.dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd");
} }
private DateTimeFormatter getDtfOut() {
if (dtfOut == null) {
dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd");
}
return dtfOut;
}
@Override @Override
public List<String> extractPartitionValuesInPath(String partitionPath) { public List<String> extractPartitionValuesInPath(String partitionPath) {
// partition path is expected to be in this format yyyy/mm/dd // partition path is expected to be in this format yyyy/mm/dd
@@ -52,6 +59,6 @@ public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExt
int mm = Integer.parseInt(splits[1]); int mm = Integer.parseInt(splits[1]);
int dd = Integer.parseInt(splits[2]); int dd = Integer.parseInt(splits[2]);
DateTime dateTime = new DateTime(year, mm, dd, 0, 0); DateTime dateTime = new DateTime(year, mm, dd, 0, 0);
return Lists.newArrayList(dtfOut.print(dateTime)); return Lists.newArrayList(getDtfOut().print(dateTime));
} }
} }

View File

@@ -221,6 +221,30 @@
<artifactId>commons-configuration2</artifactId> <artifactId>commons-configuration2</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-client</artifactId> <artifactId>hoodie-client</artifactId>
@@ -264,67 +288,4 @@
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
</dependencies> </dependencies>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive12.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive11.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -29,8 +29,13 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.hive.HiveSyncConfig;
import com.uber.hoodie.hive.PartitionValueExtractor;
import com.uber.hoodie.hive.SlashEncodedDayPartitionValueExtractor;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@@ -86,6 +91,17 @@ public class DataSourceUtils {
} }
} }
/**
* Create a partition value extractor class via reflection, passing in any configs needed
*/
public static PartitionValueExtractor createPartitionExtractor(String partitionExtractorClass) {
try {
return (PartitionValueExtractor) ReflectionUtils.loadClass(partitionExtractorClass);
} catch (Throwable e) {
throw new HoodieException("Could not load partition extractor class " + partitionExtractorClass, e);
}
}
/** /**
* Create a payload class via reflection, passing in an ordering/precombine value. * Create a payload class via reflection, passing in an ordering/precombine value.
*/ */
@@ -169,4 +185,28 @@ public class DataSourceUtils {
.withProps(parameters).build(); .withProps(parameters).build();
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig); return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
} }
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath) {
checkRequiredProperties(props, Arrays.asList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.assumeDatePartitioning =
props.getBoolean(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(),
Boolean.valueOf(DataSourceWriteOptions.DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL()));
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL());
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
hiveSyncConfig.hiveUser = props.getString(DataSourceWriteOptions.HIVE_USER_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_USER_OPT_VAL());
hiveSyncConfig.hivePass = props.getString(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_PASS_OPT_VAL());
hiveSyncConfig.jdbcUrl = props.getString(DataSourceWriteOptions.HIVE_URL_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_URL_OPT_VAL());
hiveSyncConfig.partitionFields =
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), ",", new ArrayList<>());
hiveSyncConfig.partitionValueExtractorClass =
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
SlashEncodedDayPartitionValueExtractor.class.getName());
return hiveSyncConfig;
}
} }

View File

@@ -22,13 +22,18 @@ import java.sql.{Date, Timestamp}
import java.util import java.util
import com.databricks.spark.avro.SchemaConverters import com.databricks.spark.avro.SchemaConverters
import org.apache.avro.generic.GenericData.Record import com.databricks.spark.avro.SchemaConverters.IncompatibleSchemaException
import org.apache.avro.generic.GenericRecord import org.apache.avro.Schema.Type._
import org.apache.avro.generic.GenericData.{Fixed, Record}
import org.apache.avro.generic.{GenericData, GenericRecord}
import org.apache.avro.{Schema, SchemaBuilder} import org.apache.avro.{Schema, SchemaBuilder}
import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession}
import scala.collection.JavaConverters._
object AvroConversionUtils { object AvroConversionUtils {
@@ -46,6 +51,22 @@ object AvroConversionUtils {
} }
} }
def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss : SparkSession): Dataset[Row] = {
if (rdd.isEmpty()) {
ss.emptyDataFrame
} else {
ss.createDataFrame(rdd.mapPartitions { records =>
if (records.isEmpty) Iterator.empty
else {
val schema = Schema.parse(schemaStr)
val dataType = convertAvroSchemaToStructType(schema)
val convertor = createConverterToRow(schema, dataType)
records.map { x => convertor(x).asInstanceOf[Row] }
}
}, convertAvroSchemaToStructType(Schema.parse(schemaStr))).asInstanceOf[Dataset[Row]]
}
}
def getNewRecordNamespace(elementDataType: DataType, def getNewRecordNamespace(elementDataType: DataType,
currentRecordNamespace: String, currentRecordNamespace: String,
elementName: String): String = { elementName: String): String = {
@@ -56,6 +77,185 @@ object AvroConversionUtils {
} }
} }
/**
* NOTE : This part of code is copied from com.databricks.spark.avro.SchemaConverters.scala (133:310) (spark-avro)
*
* Returns a converter function to convert row in avro format to GenericRow of catalyst.
*
* @param sourceAvroSchema Source schema before conversion inferred from avro file by passed in
* by user.
* @param targetSqlType Target catalyst sql type after the conversion.
* @return returns a converter function to convert row in avro format to GenericRow of catalyst.
*/
def createConverterToRow(sourceAvroSchema: Schema,
targetSqlType: DataType): AnyRef => AnyRef = {
def createConverter(avroSchema: Schema,
sqlType: DataType, path: List[String]): AnyRef => AnyRef = {
val avroType = avroSchema.getType
(sqlType, avroType) match {
// Avro strings are in Utf8, so we have to call toString on them
case (StringType, STRING) | (StringType, ENUM) =>
(item: AnyRef) => if (item == null) null else item.toString
// Byte arrays are reused by avro, so we have to make a copy of them.
case (IntegerType, INT) | (BooleanType, BOOLEAN) | (DoubleType, DOUBLE) |
(FloatType, FLOAT) | (LongType, LONG) =>
identity
case (BinaryType, FIXED) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
item.asInstanceOf[Fixed].bytes().clone()
}
case (BinaryType, BYTES) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
val byteBuffer = item.asInstanceOf[ByteBuffer]
val bytes = new Array[Byte](byteBuffer.remaining)
byteBuffer.get(bytes)
bytes
}
case (struct: StructType, RECORD) =>
val length = struct.fields.length
val converters = new Array[AnyRef => AnyRef](length)
val avroFieldIndexes = new Array[Int](length)
var i = 0
while (i < length) {
val sqlField = struct.fields(i)
val avroField = avroSchema.getField(sqlField.name)
if (avroField != null) {
val converter = createConverter(avroField.schema(), sqlField.dataType,
path :+ sqlField.name)
converters(i) = converter
avroFieldIndexes(i) = avroField.pos()
} else if (!sqlField.nullable) {
throw new IncompatibleSchemaException(
s"Cannot find non-nullable field ${sqlField.name} at path ${path.mkString(".")} " +
"in Avro schema\n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
s"Target Catalyst type: $targetSqlType")
}
i += 1
}
(item: AnyRef) => {
if (item == null) {
null
} else {
val record = item.asInstanceOf[GenericRecord]
val result = new Array[Any](length)
var i = 0
while (i < converters.length) {
if (converters(i) != null) {
val converter = converters(i)
result(i) = converter(record.get(avroFieldIndexes(i)))
}
i += 1
}
new GenericRow(result)
}
}
case (arrayType: ArrayType, ARRAY) =>
val elementConverter = createConverter(avroSchema.getElementType, arrayType.elementType,
path)
val allowsNull = arrayType.containsNull
(item: AnyRef) => {
if (item == null) {
null
} else {
item.asInstanceOf[java.lang.Iterable[AnyRef]].asScala.map { element =>
if (element == null && !allowsNull) {
throw new RuntimeException(s"Array value at path ${path.mkString(".")} is not " +
"allowed to be null")
} else {
elementConverter(element)
}
}
}
}
case (mapType: MapType, MAP) if mapType.keyType == StringType =>
val valueConverter = createConverter(avroSchema.getValueType, mapType.valueType, path)
val allowsNull = mapType.valueContainsNull
(item: AnyRef) => {
if (item == null) {
null
} else {
item.asInstanceOf[java.util.Map[AnyRef, AnyRef]].asScala.map { x =>
if (x._2 == null && !allowsNull) {
throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " +
"allowed to be null")
} else {
(x._1.toString, valueConverter(x._2))
}
}.toMap
}
}
case (sqlType, UNION) =>
if (avroSchema.getTypes.asScala.exists(_.getType == NULL)) {
val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL)
if (remainingUnionTypes.size == 1) {
createConverter(remainingUnionTypes.head, sqlType, path)
} else {
createConverter(Schema.createUnion(remainingUnionTypes.asJava), sqlType, path)
}
} else avroSchema.getTypes.asScala.map(_.getType) match {
case Seq(t1) => createConverter(avroSchema.getTypes.get(0), sqlType, path)
case Seq(a, b) if Set(a, b) == Set(INT, LONG) && sqlType == LongType =>
(item: AnyRef) => {
item match {
case null => null
case l: java.lang.Long => l
case i: java.lang.Integer => new java.lang.Long(i.longValue())
}
}
case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && sqlType == DoubleType =>
(item: AnyRef) => {
item match {
case null => null
case d: java.lang.Double => d
case f: java.lang.Float => new java.lang.Double(f.doubleValue())
}
}
case other =>
sqlType match {
case t: StructType if t.fields.length == avroSchema.getTypes.size =>
val fieldConverters = t.fields.zip(avroSchema.getTypes.asScala).map {
case (field, schema) =>
createConverter(schema, field.dataType, path :+ field.name)
}
(item: AnyRef) => if (item == null) {
null
} else {
val i = GenericData.get().resolveUnion(avroSchema, item)
val converted = new Array[Any](fieldConverters.length)
converted(i) = fieldConverters(i)(item)
new GenericRow(converted)
}
case _ => throw new IncompatibleSchemaException(
s"Cannot convert Avro schema to catalyst type because schema at path " +
s"${path.mkString(".")} is not compatible " +
s"(avroType = $other, sqlType = $sqlType). \n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
s"Target Catalyst type: $targetSqlType")
}
}
case (left, right) =>
throw new IncompatibleSchemaException(
s"Cannot convert Avro schema to catalyst type because schema at path " +
s"${path.mkString(".")} is not compatible (avroType = $left, sqlType = $right). \n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
s"Target Catalyst type: $targetSqlType")
}
}
createConverter(sourceAvroSchema, targetSqlType, List.empty[String])
}
def createConverterToAvro(dataType: DataType, def createConverterToAvro(dataType: DataType,
structName: String, structName: String,
recordNamespace: String): Any => Any = { recordNamespace: String): Any => Any = {

View File

@@ -43,7 +43,7 @@ object DataSourceReadOptions {
val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental" val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental"
val VIEW_TYPE_REALTIME_OPT_VAL = "realtime" val VIEW_TYPE_REALTIME_OPT_VAL = "realtime"
val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
val DEFAULTPUSH_DOWN_FILTERS_OPT_VAL = ""
/** /**
* Instant time to start incrementally pulling data from. The instanttime here need not * Instant time to start incrementally pulling data from. The instanttime here need not
@@ -64,6 +64,13 @@ object DataSourceReadOptions {
* *
*/ */
val END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.end.instanttime" val END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.end.instanttime"
/**
* For use-cases like DeltaStreamer which reads from Hoodie Incremental table and applies opaque map functions,
* filters appearing late in the sequence of transformations cannot be automatically pushed down.
* This option allows setting filters directly on Hoodie Source
*/
val PUSH_DOWN_INCR_FILTERS_OPT_KEY = "hoodie.datasource.read.incr.filters"
} }
/** /**

View File

@@ -64,21 +64,33 @@ class IncrementalRelation(val sqlContext: SQLContext,
throw new HoodieException(s"Specify the begin instant time to pull from using " + throw new HoodieException(s"Specify the begin instant time to pull from using " +
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}") s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
} }
val lastInstant = commitTimeline.lastInstant().get()
val commitsToReturn = commitTimeline.findInstantsInRange( val commitsToReturn = commitTimeline.findInstantsInRange(
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY), optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp))
commitTimeline.lastInstant().get().getTimestamp))
.getInstants.iterator().toList .getInstants.iterator().toList
// use schema from a file produced in the latest instant // use schema from a file produced in the latest instant
val latestSchema = { val latestSchema = {
// use last instant if instant range is empty
val instant = commitsToReturn.lastOption.getOrElse(lastInstant)
val latestMeta = HoodieCommitMetadata val latestMeta = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get, classOf[HoodieCommitMetadata]) .fromBytes(commitTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata])
val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next() val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next()
AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema( AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema(
sqlContext.sparkContext.hadoopConfiguration, new Path(metaFilePath))) sqlContext.sparkContext.hadoopConfiguration, new Path(metaFilePath)))
} }
val filters = {
if (optParams.contains(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY)) {
val filterStr = optParams.get(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY).getOrElse("")
filterStr.split(",").filter(!_.isEmpty)
}
Array[String]()
}
override def schema: StructType = latestSchema override def schema: StructType = latestSchema
override def buildScan(): RDD[Row] = { override def buildScan(): RDD[Row] = {
@@ -92,12 +104,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
// will filter out all the files incorrectly. // will filter out all the files incorrectly.
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class") sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class")
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
sqlContext.read.options(sOpts) if (fileIdToFullPath.isEmpty) {
.schema(latestSchema) // avoid AnalysisException for empty input sqlContext.sparkContext.emptyRDD[Row]
} else {
log.info("Additional Filters to be applied to incremental source are :" + filters)
filters.foldLeft(sqlContext.read.options(sOpts)
.schema(latestSchema)
.parquet(fileIdToFullPath.values.toList: _*) .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.head.getTimestamp))
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)) .filter(String.format("%s <= '%s'",
HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)))((e, f) => e.filter(f))
.toDF().rdd .toDF().rdd
}
} }
} }

View File

@@ -100,7 +100,6 @@ class DataSourceTest extends AssertionsForJUnit {
.load(basePath + "/*/*/*/*"); .load(basePath + "/*/*/*/*");
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
// Read Incremental View // Read Incremental View
// we have 2 commits, try pulling the first commit (which is not the latest) // we have 2 commits, try pulling the first commit (which is not the latest)
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0); val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0);

View File

@@ -53,9 +53,10 @@
<configuration> <configuration>
<dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml
</dependencyReducedPomLocation> </dependencyReducedPomLocation>
<minimizeJar>true</minimizeJar>
<artifactSet> <artifactSet>
<includes> <includes>
<include>commons-dbcp:commons-dbcp</include>
<include>commons-pool:commons-pool</include>
<include>com.uber.hoodie:hoodie-common</include> <include>com.uber.hoodie:hoodie-common</include>
<include>com.uber.hoodie:hoodie-client</include> <include>com.uber.hoodie:hoodie-client</include>
<include>com.uber.hoodie:hoodie-spark</include> <include>com.uber.hoodie:hoodie-spark</include>
@@ -76,8 +77,50 @@
<include>com.yammer.metrics:metrics-core</include> <include>com.yammer.metrics:metrics-core</include>
<include>com.101tec:zkclient</include> <include>com.101tec:zkclient</include>
<include>org.apache.kafka:kafka-clients</include> <include>org.apache.kafka:kafka-clients</include>
<include>org.apache.hive:hive-common</include>
<include>org.apache.hive:hive-service</include>
<include>org.apache.hive:hive-metastore</include>
<include>org.apache.hive:hive-jdbc</include>
</includes> </includes>
</artifactSet> </artifactSet>
<relocations>
<relocation>
<pattern>org.apache.commons.dbcp.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.commons.dbcp.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.commons.pool.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.commons.pool.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hive.jdbc.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hive.jdbc.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hadoop.hive.metastore.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hadoop_hive.metastore.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hive.common.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hive.common.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hadoop.hive.common.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hadoop_hive.common.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hadoop.hive.conf.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hadoop_hive.conf.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hive.service.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hive.service.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.hadoop.hive.service.</pattern>
<shadedPattern>com.uber.hoodie.org.apache.hadoop_hive.service.</shadedPattern>
</relocation>
</relocations>
</configuration> </configuration>
</execution> </execution>
</executions> </executions>
@@ -123,6 +166,15 @@
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-hive</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-spark</artifactId> <artifactId>hoodie-spark</artifactId>
@@ -154,6 +206,30 @@
</exclusions> </exclusions>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
<classifier>standalone</classifier>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-hive</artifactId> <artifactId>hoodie-hive</artifactId>
@@ -185,6 +261,11 @@
<groupId>commons-dbcp</groupId> <groupId>commons-dbcp</groupId>
<artifactId>commons-dbcp</artifactId> <artifactId>commons-dbcp</artifactId>
</dependency> </dependency>
<dependency>
<groupId>commons-pool</groupId>
<artifactId>commons-pool</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.apache.httpcomponents</groupId> <groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId> <artifactId>httpcore</artifactId>
@@ -303,59 +384,4 @@
</dependency> </dependency>
</dependencies> </dependencies>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
<classifier>standalone</classifier>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
<classifier>standalone</classifier>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -0,0 +1,115 @@
/*
* 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;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
public class HoodieCleaner {
private static volatile Logger log = LogManager.getLogger(HoodieDeltaStreamer.class);
/**
* Config for Cleaner
*/
private final Config cfg;
/**
* Filesystem used
*/
private transient FileSystem fs;
/**
* Spark context
*/
private transient JavaSparkContext jssc;
/**
* Bag of properties with source, hoodie client, key generator etc.
*/
TypedProperties props;
public HoodieCleaner(Config cfg, JavaSparkContext jssc) throws IOException {
this.cfg = cfg;
this.jssc = jssc;
this.fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration());
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
log.info("Creating Cleaner with configs : " + props.toString());
}
public void run() throws Exception {
HoodieWriteConfig hoodieCfg = getHoodieClientConfig();
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, false);
client.clean();
}
private HoodieWriteConfig getHoodieClientConfig() throws Exception {
return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.basePath)
.withAutoCommit(false)
.withProps(props).build();
}
public static class Config implements Serializable {
@Parameter(names = {"--target-base-path"}, description = "base path for the hoodie dataset to be cleaner.",
required = true)
public String basePath;
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
+ "hoodie client for cleaning")
public String propsFilePath =
"file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties";
@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+ "(using the CLI parameter \"--propsFilePath\") can also be passed command line using this parameter")
public List<String> configs = new ArrayList<>();
@Parameter(names = {"--spark-master"}, description = "spark master to use.")
public String sparkMaster = "local[2]";
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
}
public static void main(String[] args) throws Exception {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, args);
if (cfg.help || args.length == 0) {
cmd.usage();
System.exit(1);
}
String dirName = new Path(cfg.basePath).getName();
JavaSparkContext jssc = UtilHelpers.buildSparkContext("hoodie-cleaner-" + dirName, cfg.sparkMaster);
new HoodieCleaner(cfg, jssc).run();
}
}

View File

@@ -30,9 +30,13 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.Source; import com.uber.hoodie.utilities.sources.Source;
import com.uber.hoodie.utilities.transform.Transformer;
import java.io.BufferedReader;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.StringReader;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List;
import java.util.Optional; import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@@ -43,6 +47,7 @@ import org.apache.spark.Accumulator;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
/** /**
* Bunch of helper methods * Bunch of helper methods
@@ -51,12 +56,12 @@ public class UtilHelpers {
private static Logger logger = LogManager.getLogger(UtilHelpers.class); private static Logger logger = LogManager.getLogger(UtilHelpers.class);
public static Source createSource(String sourceClass, TypedProperties cfg, public static Source createSource(String sourceClass, TypedProperties cfg,
JavaSparkContext jssc, SchemaProvider schemaProvider) JavaSparkContext jssc, SparkSession sparkSession, SchemaProvider schemaProvider)
throws IOException { throws IOException {
try { try {
return (Source) ReflectionUtils.loadClass(sourceClass, return (Source) ReflectionUtils.loadClass(sourceClass,
new Class<?>[]{TypedProperties.class, JavaSparkContext.class, SchemaProvider.class}, new Class<?>[]{TypedProperties.class, JavaSparkContext.class, SparkSession.class, SchemaProvider.class},
cfg, jssc, schemaProvider); cfg, jssc, sparkSession, schemaProvider);
} catch (Throwable e) { } catch (Throwable e) {
throw new IOException("Could not load source class " + sourceClass, e); throw new IOException("Could not load source class " + sourceClass, e);
} }
@@ -65,17 +70,31 @@ public class UtilHelpers {
public static SchemaProvider createSchemaProvider(String schemaProviderClass, public static SchemaProvider createSchemaProvider(String schemaProviderClass,
TypedProperties cfg, JavaSparkContext jssc) throws IOException { TypedProperties cfg, JavaSparkContext jssc) throws IOException {
try { try {
return (SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc); return schemaProviderClass == null ? null :
(SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc);
} catch (Throwable e) { } catch (Throwable e) {
throw new IOException("Could not load schema provider class " + schemaProviderClass, e); throw new IOException("Could not load schema provider class " + schemaProviderClass, e);
} }
} }
public static Transformer createTransformer(String transformerClass) throws IOException {
try {
return transformerClass == null ? null : (Transformer) ReflectionUtils.loadClass(transformerClass);
} catch (Throwable e) {
throw new IOException("Could not load transformer class " + transformerClass, e);
}
}
/** /**
*/ */
public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath) { public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) {
try { try {
return new DFSPropertiesConfiguration(fs, cfgPath); DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath);
if (!overriddenProps.isEmpty()) {
logger.info("Adding overridden properties to file properties.");
conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
}
return conf;
} catch (Exception e) { } catch (Exception e) {
throw new HoodieException("Unable to read props file at :" + cfgPath, e); throw new HoodieException("Unable to read props file at :" + cfgPath, e);
} }
@@ -109,7 +128,7 @@ public class UtilHelpers {
sparkConf.set("spark.eventLog.overwrite", "true"); sparkConf.set("spark.eventLog.overwrite", "true");
sparkConf.set("spark.eventLog.enabled", "true"); sparkConf.set("spark.eventLog.enabled", "true");
} }
sparkConf.set("spark.driver.maxResultSize", "2g"); sparkConf.setIfMissing("spark.driver.maxResultSize", "2g");
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
sparkConf.set("spark.hadoop.mapred.output.compress", "true"); sparkConf.set("spark.hadoop.mapred.output.compress", "true");
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true"); sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");

View File

@@ -18,10 +18,15 @@
package com.uber.hoodie.utilities.deltastreamer; package com.uber.hoodie.utilities.deltastreamer;
import static com.uber.hoodie.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
import static com.uber.hoodie.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
import com.beust.jcommander.IStringConverter; import com.beust.jcommander.IStringConverter;
import com.beust.jcommander.JCommander; import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameter;
import com.beust.jcommander.ParameterException; import com.beust.jcommander.ParameterException;
import com.codahale.metrics.Timer;
import com.uber.hoodie.AvroConversionUtils;
import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.KeyGenerator; import com.uber.hoodie.KeyGenerator;
@@ -36,32 +41,40 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.hive.HiveSyncConfig;
import com.uber.hoodie.hive.HiveSyncTool;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.utilities.HiveIncrementalPuller; import com.uber.hoodie.utilities.HiveIncrementalPuller;
import com.uber.hoodie.utilities.UtilHelpers; import com.uber.hoodie.utilities.UtilHelpers;
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import com.uber.hoodie.utilities.schema.RowBasedSchemaProvider;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.InputBatch;
import com.uber.hoodie.utilities.sources.JsonDFSSource; import com.uber.hoodie.utilities.sources.JsonDFSSource;
import com.uber.hoodie.utilities.sources.Source; import com.uber.hoodie.utilities.transform.Transformer;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import scala.collection.JavaConversions; import scala.collection.JavaConversions;
/** /**
@@ -81,7 +94,7 @@ public class HoodieDeltaStreamer implements Serializable {
/** /**
* Source to pull deltas from * Source to pull deltas from
*/ */
private transient Source source; private transient SourceFormatAdapter formatAdapter;
/** /**
* Schema provider that supplies the command for reading the input and writing out the target * Schema provider that supplies the command for reading the input and writing out the target
@@ -89,6 +102,11 @@ public class HoodieDeltaStreamer implements Serializable {
*/ */
private transient SchemaProvider schemaProvider; private transient SchemaProvider schemaProvider;
/**
* Allows transforming source to target dataset before writing
*/
private transient Transformer transformer;
/** /**
* Extract the key for the target dataset * Extract the key for the target dataset
*/ */
@@ -109,16 +127,30 @@ public class HoodieDeltaStreamer implements Serializable {
*/ */
private transient JavaSparkContext jssc; private transient JavaSparkContext jssc;
/**
* Spark Session
*/
private transient SparkSession sparkSession;
/**
* Hive Config
*/
private transient HiveConf hiveConf;
/** /**
* Bag of properties with source, hoodie client, key generator etc. * Bag of properties with source, hoodie client, key generator etc.
*/ */
TypedProperties props; TypedProperties props;
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException { public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException {
this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()),
getDefaultHiveConf(jssc.hadoopConfiguration()));
}
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf) throws IOException {
this.cfg = cfg; this.cfg = cfg;
this.jssc = jssc; this.jssc = jssc;
this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate();
this.fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); this.fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
if (fs.exists(new Path(cfg.targetBasePath))) { if (fs.exists(new Path(cfg.targetBasePath))) {
@@ -129,19 +161,28 @@ public class HoodieDeltaStreamer implements Serializable {
this.commitTimelineOpt = Optional.empty(); this.commitTimelineOpt = Optional.empty();
} }
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath)).getConfig(); this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
log.info("Creating delta streamer with configs : " + props.toString()); log.info("Creating delta streamer with configs : " + props.toString());
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc); this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc);
this.transformer = UtilHelpers.createTransformer(cfg.transformerClassName);
this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, props); this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, props);
this.source = UtilHelpers.createSource(cfg.sourceClassName, props, jssc, schemaProvider);
// register the schemas, so that shuffle does not serialize the full schemas this.formatAdapter =
List<Schema> schemas = Arrays.asList(schemaProvider.getSourceSchema(), new SourceFormatAdapter(UtilHelpers.createSource(cfg.sourceClassName, props, jssc, sparkSession,
schemaProvider.getTargetSchema()); schemaProvider));
jssc.sc().getConf().registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList());
this.hiveConf = hiveConf;
}
private static HiveConf getDefaultHiveConf(Configuration cfg) {
HiveConf hiveConf = new HiveConf();
hiveConf.addResource(cfg);
return hiveConf;
} }
public void sync() throws Exception { public void sync() throws Exception {
HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(null));
Timer.Context overallTimerContext = metrics.getOverallTimerContext();
// Retrieve the previous round checkpoints, if any // Retrieve the previous round checkpoints, if any
Optional<String> resumeCheckpointStr = Optional.empty(); Optional<String> resumeCheckpointStr = Optional.empty();
if (commitTimelineOpt.isPresent()) { if (commitTimelineOpt.isPresent()) {
@@ -163,16 +204,42 @@ public class HoodieDeltaStreamer implements Serializable {
} }
log.info("Checkpoint to resume from : " + resumeCheckpointStr); log.info("Checkpoint to resume from : " + resumeCheckpointStr);
// Pull the data from the source & prepare the write final Optional<JavaRDD<GenericRecord>> avroRDDOptional;
Pair<Optional<JavaRDD<GenericRecord>>, String> dataAndCheckpoint = source.fetchNewData( final String checkpointStr;
final SchemaProvider schemaProvider;
if (transformer != null) {
// Transformation is needed. Fetch New rows in Row Format, apply transformation and then convert them
// to generic records for writing
InputBatch<Dataset<Row>> dataAndCheckpoint = formatAdapter.fetchNewDataInRowFormat(
resumeCheckpointStr, cfg.sourceLimit); resumeCheckpointStr, cfg.sourceLimit);
if (!dataAndCheckpoint.getKey().isPresent()) { Optional<Dataset<Row>> transformed =
dataAndCheckpoint.getBatch().map(data -> transformer.apply(jssc, sparkSession, data, props));
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
avroRDDOptional = transformed.map(t ->
AvroConversionUtils.createRdd(t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()
);
// Use Transformed Row's schema if not overridden
schemaProvider =
this.schemaProvider == null ? transformed.map(r -> (SchemaProvider)new RowBasedSchemaProvider(r.schema()))
.orElse(dataAndCheckpoint.getSchemaProvider()) : this.schemaProvider;
} else {
// Pull the data from the source & prepare the write
InputBatch<JavaRDD<GenericRecord>> dataAndCheckpoint =
formatAdapter.fetchNewDataInAvroFormat(resumeCheckpointStr, cfg.sourceLimit);
avroRDDOptional = dataAndCheckpoint.getBatch();
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
schemaProvider = dataAndCheckpoint.getSchemaProvider();
}
if ((!avroRDDOptional.isPresent()) || (avroRDDOptional.get().isEmpty())) {
log.info("No new data, nothing to commit.. "); log.info("No new data, nothing to commit.. ");
return; return;
} }
JavaRDD<GenericRecord> avroRDD = dataAndCheckpoint.getKey().get(); registerAvroSchemas(schemaProvider);
JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
JavaRDD<HoodieRecord> records = avroRDD.map(gr -> { JavaRDD<HoodieRecord> records = avroRDD.map(gr -> {
HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr, HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr,
(Comparable) gr.get(cfg.sourceOrderingField)); (Comparable) gr.get(cfg.sourceOrderingField));
@@ -180,20 +247,20 @@ public class HoodieDeltaStreamer implements Serializable {
}); });
// filter dupes if needed // filter dupes if needed
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(); HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
if (cfg.filterDupes) { if (cfg.filterDupes) {
// turn upserts to insert // turn upserts to insert
cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation; cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation;
records = DataSourceUtils.dropDuplicates(jssc, records, hoodieCfg); records = DataSourceUtils.dropDuplicates(jssc, records, hoodieCfg);
}
if (records.isEmpty()) { if (records.isEmpty()) {
log.info("No new data, nothing to commit.. "); log.info("No new data, nothing to commit.. ");
return; return;
} }
}
// Perform the write // Perform the write
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg); HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, true);
String commitTime = client.startCommit(); String commitTime = client.startCommit();
log.info("Starting commit : " + commitTime); log.info("Starting commit : " + commitTime);
@@ -210,7 +277,7 @@ public class HoodieDeltaStreamer implements Serializable {
// Simply commit for now. TODO(vc): Support better error handlers later on // Simply commit for now. TODO(vc): Support better error handlers later on
HashMap<String, String> checkpointCommitMetadata = new HashMap<>(); HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue()); checkpointCommitMetadata.put(CHECKPOINT_KEY, checkpointStr);
boolean success = client.commit(commitTime, writeStatusRDD, boolean success = client.commit(commitTime, writeStatusRDD,
Optional.of(checkpointCommitMetadata)); Optional.of(checkpointCommitMetadata));
@@ -220,17 +287,54 @@ public class HoodieDeltaStreamer implements Serializable {
} else { } else {
log.info("Commit " + commitTime + " failed!"); log.info("Commit " + commitTime + " failed!");
} }
// Sync to hive if enabled
Timer.Context hiveSyncContext = metrics.getHiveSyncTimerContext();
syncHive();
long hiveSyncTimeMs = hiveSyncContext != null ? hiveSyncContext.stop() : 0;
client.close(); client.close();
long overallTimeMs = overallTimerContext != null ? overallTimerContext.stop() : 0;
// Send DeltaStreamer Metrics
metrics.updateDeltaStreamerMetrics(overallTimeMs, hiveSyncTimeMs);
} }
private HoodieWriteConfig getHoodieClientConfig() throws Exception { public void syncHive() {
return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) if (cfg.enableHiveSync) {
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath);
log.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName
+ "). Hive metastore URL :" + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
}
}
/**
* Register Avro Schemas
* @param schemaProvider Schema Provider
*/
private void registerAvroSchemas(SchemaProvider schemaProvider) {
// register the schemas, so that shuffle does not serialize the full schemas
if (null != schemaProvider) {
List<Schema> schemas = Arrays.asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema());
log.info("Registering Schema :" + schemas);
jssc.sc().getConf().registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList());
}
}
private HoodieWriteConfig getHoodieClientConfig(SchemaProvider schemaProvider) throws Exception {
HoodieWriteConfig.Builder builder =
HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath)
.withAutoCommit(false) .withAutoCommit(false)
.withSchema(schemaProvider.getTargetSchema().toString())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build())
.forTable(cfg.targetTableName) .forTable(cfg.targetTableName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withProps(props).build(); .withProps(props);
if (null != schemaProvider) {
builder = builder.withSchema(schemaProvider.getTargetSchema().toString());
}
return builder.build();
} }
public enum Operation { public enum Operation {
@@ -266,6 +370,10 @@ public class HoodieDeltaStreamer implements Serializable {
public String propsFilePath = public String propsFilePath =
"file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties"; "file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties";
@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+ "(using the CLI parameter \"--propsFilePath\") can also be passed command line using this parameter")
public List<String> configs = new ArrayList<>();
@Parameter(names = {"--source-class"}, description = "Subclass of com.uber.hoodie.utilities.sources to read data. " @Parameter(names = {"--source-class"}, description = "Subclass of com.uber.hoodie.utilities.sources to read data. "
+ "Built-in options: com.uber.hoodie.utilities.sources.{JsonDFSSource (default), AvroDFSSource, " + "Built-in options: com.uber.hoodie.utilities.sources.{JsonDFSSource (default), AvroDFSSource, "
+ "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}") + "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}")
@@ -285,11 +393,22 @@ public class HoodieDeltaStreamer implements Serializable {
public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
@Parameter(names = {"--schemaprovider-class"}, description = "subclass of com.uber.hoodie.utilities.schema" @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") + ".SchemaProvider to attach schemas to input & target table data, built in options: "
public String schemaProviderClassName = FilebasedSchemaProvider.class.getName(); + "com.uber.hoodie.utilities.schema.FilebasedSchemaProvider."
+ "Source (See com.uber.hoodie.utilities.sources.Source) implementation can implement their own SchemaProvider."
+ " For Sources that return Dataset<Row>, the schema is obtained implicitly. "
+ "However, this CLI option allows overriding the schemaprovider returned by Source.")
public String schemaProviderClassName = null;
@Parameter(names = {"--transformer-class"},
description = "subclass of com.uber.hoodie.utilities.transform.Transformer"
+ ". Allows transforming raw source dataset to a target dataset (conforming to target schema) before writing."
+ " Default : Not set. E:g - com.uber.hoodie.utilities.transform.SqlQueryBasedTransformer (which allows"
+ "a SQL query templated to be passed as a transformation function)")
public String transformerClassName = null;
@Parameter(names = {"--source-limit"}, description = "Maximum amount of data to read from source. " @Parameter(names = {"--source-limit"}, description = "Maximum amount of data to read from source. "
+ "Default: No limit For e.g: DFSSource => max bytes to read, KafkaSource => max events to read") + "Default: No limit For e.g: DFS-Source => max bytes to read, Kafka-Source => max events to read")
public long sourceLimit = Long.MAX_VALUE; public long sourceLimit = Long.MAX_VALUE;
@Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input " @Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input "
@@ -301,6 +420,9 @@ public class HoodieDeltaStreamer implements Serializable {
+ "before insert/bulk-insert") + "before insert/bulk-insert")
public Boolean filterDupes = false; public Boolean filterDupes = false;
@Parameter(names = {"--enable-hive-sync"}, description = "Enable syncing to hive")
public Boolean enableHiveSync = false;
@Parameter(names = {"--spark-master"}, description = "spark master to use.") @Parameter(names = {"--spark-master"}, description = "spark master to use.")
public String sparkMaster = "local[2]"; public String sparkMaster = "local[2]";
@@ -319,4 +441,44 @@ public class HoodieDeltaStreamer implements Serializable {
JavaSparkContext jssc = UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName, cfg.sparkMaster); JavaSparkContext jssc = UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName, cfg.sparkMaster);
new HoodieDeltaStreamer(cfg, jssc).sync(); new HoodieDeltaStreamer(cfg, jssc).sync();
} }
public SourceFormatAdapter getFormatAdapter() {
return formatAdapter;
}
public SchemaProvider getSchemaProvider() {
return schemaProvider;
}
public Transformer getTransformer() {
return transformer;
}
public KeyGenerator getKeyGenerator() {
return keyGenerator;
}
public FileSystem getFs() {
return fs;
}
public Optional<HoodieTimeline> getCommitTimelineOpt() {
return commitTimelineOpt;
}
public JavaSparkContext getJssc() {
return jssc;
}
public SparkSession getSparkSession() {
return sparkSession;
}
public HiveConf getHiveConf() {
return hiveConf;
}
public TypedProperties getProps() {
return props;
}
} }

View File

@@ -0,0 +1,61 @@
package com.uber.hoodie.utilities.deltastreamer;
import static com.uber.hoodie.metrics.Metrics.registerGauge;
import com.codahale.metrics.Timer;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.metrics.Metrics;
public class HoodieDeltaStreamerMetrics {
private HoodieWriteConfig config = null;
private String tableName = null;
public String overallTimerName = null;
public String hiveSyncTimerName = null;
private Timer overallTimer = null;
public Timer hiveSyncTimer = null;
public HoodieDeltaStreamerMetrics(HoodieWriteConfig config) {
this.config = config;
this.tableName = config.getTableName();
if (config.isMetricsOn()) {
Metrics.init(config);
this.overallTimerName = getMetricsName("timer", "deltastreamer");
this.hiveSyncTimerName = getMetricsName("timer", "deltastreamerHiveSync");
}
}
public Timer.Context getOverallTimerContext() {
if (config.isMetricsOn() && overallTimer == null) {
overallTimer = createTimer(overallTimerName);
}
return overallTimer == null ? null : overallTimer.time();
}
public Timer.Context getHiveSyncTimerContext() {
if (config.isMetricsOn() && hiveSyncTimer == null) {
hiveSyncTimer = createTimer(hiveSyncTimerName);
}
return hiveSyncTimer == null ? null : hiveSyncTimer.time();
}
private Timer createTimer(String name) {
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
}
String getMetricsName(String action, String metric) {
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
}
public void updateDeltaStreamerMetrics(long durationInNs, long hiveSyncNs) {
if (config.isMetricsOn()) {
registerGauge(getMetricsName("deltastreamer", "duration"), getDurationInMs(durationInNs));
registerGauge(getMetricsName("deltastreamer", "hiveSyncDuration"), getDurationInMs(hiveSyncNs));
}
}
public long getDurationInMs(long ctxDuration) {
return ctxDuration / 1000000;
}
}

View File

@@ -0,0 +1,112 @@
/*
* Copyright (c) 2018 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 static com.uber.hoodie.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
import static com.uber.hoodie.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
import com.uber.hoodie.AvroConversionUtils;
import com.uber.hoodie.utilities.sources.AvroSource;
import com.uber.hoodie.utilities.sources.InputBatch;
import com.uber.hoodie.utilities.sources.JsonSource;
import com.uber.hoodie.utilities.sources.RowSource;
import com.uber.hoodie.utilities.sources.Source;
import com.uber.hoodie.utilities.sources.helpers.AvroConvertor;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.types.StructType;
/**
* Adapts data-format provided by the source to the data-format required by the client (DeltaStreamer)
*/
public final class SourceFormatAdapter {
private final Source source;
public SourceFormatAdapter(Source source) {
this.source = source;
}
/**
* Fetch new data in avro format. If the source provides data in different format, they are translated
* to Avro format
* @param lastCkptStr
* @param sourceLimit
* @return
*/
public InputBatch<JavaRDD<GenericRecord>> fetchNewDataInAvroFormat(Optional<String> lastCkptStr,
long sourceLimit) {
switch (source.getSourceType()) {
case AVRO:
return ((AvroSource)source).fetchNext(lastCkptStr, sourceLimit);
case JSON: {
InputBatch<JavaRDD<String>> r = ((JsonSource)source).fetchNext(lastCkptStr, sourceLimit);
AvroConvertor convertor = new AvroConvertor(r.getSchemaProvider().getSourceSchema());
return new InputBatch<>(Optional.ofNullable(
r.getBatch().map(rdd -> rdd.map(convertor::fromJson))
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
}
case ROW: {
InputBatch<Dataset<Row>> r = ((RowSource)source).fetchNext(lastCkptStr, sourceLimit);
return new InputBatch<>(Optional.ofNullable(r.getBatch().map(
rdd -> (AvroConversionUtils.createRdd(rdd, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()))
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
}
default:
throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")");
}
}
/**
* Fetch new data in row format. If the source provides data in different format, they are translated
* to Row format
* @param lastCkptStr
* @param sourceLimit
* @return
*/
public InputBatch<Dataset<Row>> fetchNewDataInRowFormat(Optional<String> lastCkptStr, long sourceLimit) {
switch (source.getSourceType()) {
case ROW:
return ((RowSource)source).fetchNext(lastCkptStr, sourceLimit);
case AVRO: {
InputBatch<JavaRDD<GenericRecord>> r = ((AvroSource)source).fetchNext(lastCkptStr, sourceLimit);
Schema sourceSchema = r.getSchemaProvider().getSourceSchema();
return new InputBatch<>(Optional.ofNullable(
r.getBatch().map(rdd -> AvroConversionUtils.createDataFrame(JavaRDD.toRDD(rdd),
sourceSchema.toString(), source.getSparkSession()))
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
}
case JSON: {
InputBatch<JavaRDD<String>> r = ((JsonSource)source).fetchNext(lastCkptStr, sourceLimit);
Schema sourceSchema = r.getSchemaProvider().getSourceSchema();
StructType dataType = AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema);
return new InputBatch<>(Optional.ofNullable(
r.getBatch().map(rdd -> source.getSparkSession().read().schema(dataType).json(rdd))
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
}
default:
throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")");
}
}
}

View File

@@ -0,0 +1,25 @@
package com.uber.hoodie.utilities.schema;
import com.uber.hoodie.AvroConversionUtils;
import org.apache.avro.Schema;
import org.apache.spark.sql.types.StructType;
public class RowBasedSchemaProvider extends SchemaProvider {
// Used in GenericRecord conversions
public static final String HOODIE_RECORD_NAMESPACE = "hoodie.source";
public static final String HOODIE_RECORD_STRUCT_NAME = "hoodie_source";
private StructType rowStruct;
public RowBasedSchemaProvider(StructType rowStruct) {
super(null, null);
this.rowStruct = rowStruct;
}
@Override
public Schema getSourceSchema() {
return AvroConversionUtils.convertStructTypeToAvroSchema(rowStruct, HOODIE_RECORD_STRUCT_NAME,
HOODIE_RECORD_NAMESPACE);
}
}

View File

@@ -42,12 +42,15 @@ public class SchemaRegistryProvider extends SchemaProvider {
*/ */
public static class Config { public static class Config {
private static final String SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url"; private static final String SRC_SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url";
private static final String TARGET_SCHEMA_REGISTRY_URL_PROP =
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
} }
private final Schema schema; private final Schema schema;
private final Schema targetSchema;
private String fetchSchemaFromRegistry(String registryUrl) throws IOException { private static String fetchSchemaFromRegistry(String registryUrl) throws IOException {
URL registry = new URL(registryUrl); URL registry = new URL(registryUrl);
ObjectMapper mapper = new ObjectMapper(); ObjectMapper mapper = new ObjectMapper();
JsonNode node = mapper.readTree(registry.openStream()); JsonNode node = mapper.readTree(registry.openStream());
@@ -56,17 +59,32 @@ public class SchemaRegistryProvider extends SchemaProvider {
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) { public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc); super(props, jssc);
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SCHEMA_REGISTRY_URL_PROP)); DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
String registryUrl = props.getString(Config.SCHEMA_REGISTRY_URL_PROP); String registryUrl = props.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
String targetRegistryUrl = props.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);
try { try {
this.schema = new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl)); this.schema = getSchema(registryUrl);
if (!targetRegistryUrl.equals(registryUrl)) {
this.targetSchema = getSchema(targetRegistryUrl);
} else {
this.targetSchema = schema;
}
} catch (IOException ioe) { } catch (IOException ioe) {
throw new HoodieIOException("Error reading schema from registry :" + registryUrl, ioe); throw new HoodieIOException("Error reading schema from registry :" + registryUrl, ioe);
} }
} }
private static Schema getSchema(String registryUrl) throws IOException {
return new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
}
@Override @Override
public Schema getSourceSchema() { public Schema getSourceSchema() {
return schema; return schema;
} }
@Override
public Schema getTargetSchema() {
return targetSchema;
}
} }

View File

@@ -19,7 +19,10 @@
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.helpers.DFSPathSelector;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.avro.mapreduce.AvroKeyInputFormat;
@@ -27,18 +30,33 @@ import org.apache.hadoop.io.NullWritable;
import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
/** /**
* DFS Source that reads avro data * DFS Source that reads avro data
*/ */
public class AvroDFSSource extends DFSSource { public class AvroDFSSource extends AvroSource {
public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { private final DFSPathSelector pathSelector;
super(props, sparkContext, schemaProvider);
public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration());
} }
@Override @Override
protected JavaRDD<GenericRecord> fromFiles(AvroConvertor convertor, String pathStr) { protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Optional<String> lastCkptStr,
long sourceLimit) {
Pair<Optional<String>, String> selectPathsWithMaxModificationTime =
pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit);
return selectPathsWithMaxModificationTime.getLeft().map(pathStr -> new InputBatch<>(
Optional.of(fromFiles(pathStr)),
selectPathsWithMaxModificationTime.getRight()))
.orElseGet(() -> new InputBatch<>(Optional.empty(), selectPathsWithMaxModificationTime.getRight()));
}
private JavaRDD<GenericRecord> fromFiles(String pathStr) {
JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr, JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr,
AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, AvroKeyInputFormat.class, AvroKey.class, NullWritable.class,
sparkContext.hadoopConfiguration()); sparkContext.hadoopConfiguration());

View File

@@ -20,27 +20,55 @@ package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen;
import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
import io.confluent.kafka.serializers.KafkaAvroDecoder; import io.confluent.kafka.serializers.KafkaAvroDecoder;
import java.util.Optional;
import kafka.serializer.StringDecoder; import kafka.serializer.StringDecoder;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.KafkaUtils;
import org.apache.spark.streaming.kafka.OffsetRange; import org.apache.spark.streaming.kafka.OffsetRange;
/** /**
* Reads avro serialized Kafka data, based on the confluent schema-registry * Reads avro serialized Kafka data, based on the confluent schema-registry
*/ */
public class AvroKafkaSource extends KafkaSource { public class AvroKafkaSource extends AvroSource {
public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { private static Logger log = LogManager.getLogger(AvroKafkaSource.class);
super(props, sparkContext, schemaProvider);
private final KafkaOffsetGen offsetGen;
public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
offsetGen = new KafkaOffsetGen(props);
} }
@Override @Override
protected JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Optional<String> lastCheckpointStr,
return KafkaUtils long sourceLimit) {
.createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, kafkaParams, OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit);
offsetRanges).values().map(obj -> (GenericRecord) obj); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
if (totalNewMsgs <= 0) {
return new InputBatch<>(Optional.empty(),
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
} else {
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName());
}
JavaRDD<GenericRecord> newDataRDD = toRDD(offsetRanges);
return new InputBatch<>(Optional.of(newDataRDD),
KafkaOffsetGen.CheckpointUtils.offsetsToStr(offsetRanges));
}
private JavaRDD<GenericRecord> toRDD(OffsetRange[] offsetRanges) {
JavaRDD<GenericRecord> recordRDD = KafkaUtils
.createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class,
offsetGen.getKafkaParams(), offsetRanges).values().map(obj -> (GenericRecord) obj);
return recordRDD;
} }
} }

View File

@@ -0,0 +1,36 @@
/*
* Copyright (c) 2018 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.sources;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
public abstract class AvroSource extends Source<JavaRDD<GenericRecord>> {
public AvroSource(TypedProperties props,
JavaSparkContext sparkContext,
SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider, SourceType.AVRO);
}
}

View File

@@ -21,8 +21,6 @@ package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.ImmutablePair;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.io.IOException; import java.io.IOException;
@@ -44,19 +42,20 @@ import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
/** /**
* Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit * Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit by commit and apply
* by commit and apply to the target table * to the target table
* <p> * <p>
* The general idea here is to have commits sync across the data pipeline. * The general idea here is to have commits sync across the data pipeline.
* <p> * <p>
* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable * [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable {c1,c2,c3,...}
* {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...} * {c1,c2,c3,...} {c1,c2,c3,...}
* <p> * <p>
* This produces beautiful causality, that makes data issues in ETLs very easy to debug * This produces beautiful causality, that makes data issues in ETLs very easy to debug
*/ */
public class HiveIncrPullSource extends Source { public class HiveIncrPullSource extends AvroSource {
private static volatile Logger log = LogManager.getLogger(HiveIncrPullSource.class); private static volatile Logger log = LogManager.getLogger(HiveIncrPullSource.class);
@@ -73,9 +72,9 @@ public class HiveIncrPullSource extends Source {
private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root"; private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root";
} }
public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext, public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) { SchemaProvider schemaProvider) {
super(props, sparkContext, schemaProvider); super(props, sparkContext, sparkSession, schemaProvider);
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP)); DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP));
this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP); this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP);
this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration()); this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration());
@@ -113,15 +112,15 @@ public class HiveIncrPullSource extends Source {
} }
@Override @Override
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData( protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(
Optional<String> lastCheckpointStr, long sourceLimit) { Optional<String> lastCheckpointStr, long sourceLimit) {
try { try {
// find the source commit to pull // find the source commit to pull
Optional<String> commitToPull = findCommitToPull(lastCheckpointStr); Optional<String> commitToPull = findCommitToPull(lastCheckpointStr);
if (!commitToPull.isPresent()) { if (!commitToPull.isPresent()) {
return new ImmutablePair<>(Optional.empty(), return new InputBatch<>(Optional.empty(),
lastCheckpointStr.orElse("")); lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
} }
// read the files out. // read the files out.
@@ -132,7 +131,7 @@ public class HiveIncrPullSource extends Source {
JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr, JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr,
AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, AvroKeyInputFormat.class, AvroKey.class, NullWritable.class,
sparkContext.hadoopConfiguration()); sparkContext.hadoopConfiguration());
return new ImmutablePair<>(Optional.of(avroRDD.keys().map(r -> ((GenericRecord) r.datum()))), return new InputBatch<>(Optional.of(avroRDD.keys().map(r -> ((GenericRecord) r.datum()))),
String.valueOf(commitToPull.get())); String.valueOf(commitToPull.get()));
} catch (IOException ioe) { } catch (IOException ioe) {
throw new HoodieIOException( throw new HoodieIOException(

View File

@@ -0,0 +1,144 @@
package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.DataSourceReadOptions;
import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.hive.SlashEncodedDayPartitionValueExtractor;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.helpers.IncrSourceHelper;
import java.util.Arrays;
import java.util.Optional;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.DataFrameReader;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
public class HoodieIncrSource extends RowSource {
/**
* Configs supported
*/
protected static class Config {
/**
* {@value #HOODIE_SRC_BASE_PATH} is the base-path for the source Hoodie table
*/
private static final String HOODIE_SRC_BASE_PATH = "hoodie.deltastreamer.source.hoodieincr.path";
/**
* {@value #NUM_INSTANTS_PER_FETCH} allows the max number of instants whose changes can be incrementally fetched
*/
private static final String NUM_INSTANTS_PER_FETCH = "hoodie.deltastreamer.source.hoodieincr.num_instants";
private static final Integer DEFAULT_NUM_INSTANTS_PER_FETCH = 1;
/**
* {@value #HOODIE_SRC_PARTITION_FIELDS} specifies partition fields that needs to be added to source table after
* parsing _hoodie_partition_path
*/
private static final String HOODIE_SRC_PARTITION_FIELDS = "hoodie.deltastreamer.source.hoodieincr.partition.fields";
/**
* {@value #HOODIE_SRC_PARTITION_EXTRACTORCLASS} PartitionValueExtractor class to extract partition fields from
* _hoodie_partition_path
*/
private static final String HOODIE_SRC_PARTITION_EXTRACTORCLASS =
"hoodie.deltastreamer.source.hoodieincr.partition.extractor.class";
private static final String DEFAULT_HOODIE_SRC_PARTITION_EXTRACTORCLASS =
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName();
/**
* {@value #READ_LATEST_INSTANT_ON_MISSING_CKPT} allows delta-streamer to incrementally fetch from latest committed
* instant when checkpoint is not provided.
*/
private static final String READ_LATEST_INSTANT_ON_MISSING_CKPT =
"hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt";
private static final Boolean DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT = false;
}
public HoodieIncrSource(TypedProperties props,
JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
}
@Override
public Pair<Optional<Dataset<Row>>, String> fetchNextBatch(Optional<String> lastCkptStr, long sourceLimit) {
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH));
/**
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH,
Config.HOODIE_SRC_PARTITION_FIELDS));
List<String> partitionFields = props.getStringList(Config.HOODIE_SRC_PARTITION_FIELDS, ",",
new ArrayList<>());
PartitionValueExtractor extractor = DataSourceUtils.createPartitionExtractor(props.getString(
Config.HOODIE_SRC_PARTITION_EXTRACTORCLASS, Config.DEFAULT_HOODIE_SRC_PARTITION_EXTRACTORCLASS));
**/
String srcPath = props.getString(Config.HOODIE_SRC_BASE_PATH);
int numInstantsPerFetch = props.getInteger(Config.NUM_INSTANTS_PER_FETCH, Config.DEFAULT_NUM_INSTANTS_PER_FETCH);
boolean readLatestOnMissingCkpt = props.getBoolean(Config.READ_LATEST_INSTANT_ON_MISSING_CKPT,
Config.DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT);
// Use begin Instant if set and non-empty
Optional<String> beginInstant =
lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Optional.empty() : lastCkptStr : Optional.empty();
Pair<String, String> instantEndpts = IncrSourceHelper.calculateBeginAndEndInstants(sparkContext, srcPath,
numInstantsPerFetch, beginInstant, readLatestOnMissingCkpt);
if (instantEndpts.getKey().equals(instantEndpts.getValue())) {
log.warn("Already caught up. Begin Checkpoint was :" + instantEndpts.getKey());
return Pair.of(Optional.empty(), instantEndpts.getKey());
}
// Do Incr pull. Set end instant if available
DataFrameReader reader = sparkSession.read().format("com.uber.hoodie")
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), instantEndpts.getLeft())
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY(), instantEndpts.getRight());
Dataset<Row> source = reader.load(srcPath);
/**
log.info("Partition Fields are : (" + partitionFields + "). Initial Source Schema :" + source.schema());
StructType newSchema = new StructType(source.schema().fields());
for (String field : partitionFields) {
newSchema = newSchema.add(field, DataTypes.StringType, true);
}
/**
* Validates if the commit time is sane and also generates Partition fields from _hoodie_partition_path if
* configured
*
Dataset<Row> validated = source.map((MapFunction<Row, Row>) (Row row) -> {
// _hoodie_instant_time
String instantTime = row.getString(0);
IncrSourceHelper.validateInstantTime(row, instantTime, instantEndpts.getKey(), instantEndpts.getValue());
if (!partitionFields.isEmpty()) {
// _hoodie_partition_path
String hoodiePartitionPath = row.getString(3);
List<Object> partitionVals = extractor.extractPartitionValuesInPath(hoodiePartitionPath).stream()
.map(o -> (Object) o).collect(Collectors.toList());
Preconditions.checkArgument(partitionVals.size() == partitionFields.size(),
"#partition-fields != #partition-values-extracted");
List<Object> rowObjs = new ArrayList<>(scala.collection.JavaConversions.seqAsJavaList(row.toSeq()));
rowObjs.addAll(partitionVals);
return RowFactory.create(rowObjs.toArray());
}
return row;
}, RowEncoder.apply(newSchema));
log.info("Validated Source Schema :" + validated.schema());
**/
// Remove Hoodie meta columns except partition path from input source
final Dataset<Row> src = source.drop(HoodieRecord.HOODIE_META_COLUMNS.stream()
.filter(x -> !x.equals(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toArray(String[]::new));
//log.info("Final Schema from Source is :" + src.schema());
return Pair.of(Optional.of(src), instantEndpts.getRight());
}
}

View File

@@ -0,0 +1,54 @@
/*
* Copyright (c) 2018 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.sources;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.util.Optional;
public class InputBatch<T> {
private final Optional<T> batch;
private final String checkpointForNextBatch;
private final SchemaProvider schemaProvider;
public InputBatch(Optional<T> batch, String checkpointForNextBatch,
SchemaProvider schemaProvider) {
this.batch = batch;
this.checkpointForNextBatch = checkpointForNextBatch;
this.schemaProvider = schemaProvider;
}
public InputBatch(Optional<T> batch, String checkpointForNextBatch) {
this.batch = batch;
this.checkpointForNextBatch = checkpointForNextBatch;
this.schemaProvider = null;
}
public Optional<T> getBatch() {
return batch;
}
public String getCheckpointForNextBatch() {
return checkpointForNextBatch;
}
public SchemaProvider getSchemaProvider() {
return schemaProvider;
}
}

View File

@@ -19,22 +19,38 @@
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import org.apache.avro.generic.GenericRecord; import com.uber.hoodie.utilities.sources.helpers.DFSPathSelector;
import java.util.Optional;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
/** /**
* DFS Source that reads json data * DFS Source that reads json data
*/ */
public class JsonDFSSource extends DFSSource { public class JsonDFSSource extends JsonSource {
public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { private final DFSPathSelector pathSelector;
super(props, sparkContext, schemaProvider);
public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration());
} }
@Override @Override
protected JavaRDD<GenericRecord> fromFiles(AvroConvertor convertor, String pathStr) { protected InputBatch<JavaRDD<String>> fetchNewData(Optional<String> lastCkptStr,
return sparkContext.textFile(pathStr).map(convertor::fromJson); long sourceLimit) {
Pair<Optional<String>, String> selPathsWithMaxModificationTime =
pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit);
return selPathsWithMaxModificationTime.getLeft().map(pathStr -> new InputBatch<>(
Optional.of(fromFiles(pathStr)), selPathsWithMaxModificationTime.getRight()))
.orElse(new InputBatch<>(Optional.empty(), selPathsWithMaxModificationTime.getRight()));
}
private JavaRDD<String> fromFiles(String pathStr) {
return sparkContext.textFile(pathStr);
} }
} }

View File

@@ -20,26 +20,49 @@ package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen;
import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
import java.util.Optional;
import kafka.serializer.StringDecoder; import kafka.serializer.StringDecoder;
import org.apache.avro.generic.GenericRecord; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.KafkaUtils;
import org.apache.spark.streaming.kafka.OffsetRange; import org.apache.spark.streaming.kafka.OffsetRange;
/** /**
* Read json kafka data * Read json kafka data
*/ */
public class JsonKafkaSource extends KafkaSource { public class JsonKafkaSource extends JsonSource {
public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { private static Logger log = LogManager.getLogger(JsonKafkaSource.class);
super(properties, sparkContext, schemaProvider);
private final KafkaOffsetGen offsetGen;
public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(properties, sparkContext, sparkSession, schemaProvider);
offsetGen = new KafkaOffsetGen(properties);
} }
@Override @Override
protected JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { protected InputBatch<JavaRDD<String>> fetchNewData(Optional<String> lastCheckpointStr,
long sourceLimit) {
OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit);
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
if (totalNewMsgs <= 0) {
return new InputBatch<>(Optional.empty(),
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
}
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName());
JavaRDD<String> newDataRDD = toRDD(offsetRanges);
return new InputBatch<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges));
}
private JavaRDD<String> toRDD(OffsetRange[] offsetRanges) {
return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class, return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class,
kafkaParams, offsetRanges) offsetGen.getKafkaParams(), offsetRanges).values();
.values().map(avroConvertor::fromJson);
} }
} }

View File

@@ -0,0 +1,35 @@
/*
* Copyright (c) 2018 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.sources;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
public abstract class JsonSource extends Source<JavaRDD<String>> {
public JsonSource(TypedProperties props,
JavaSparkContext sparkContext,
SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider, SourceType.JSON);
}
}

View File

@@ -0,0 +1,51 @@
/*
* Copyright (c) 2018 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.sources;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.schema.RowBasedSchemaProvider;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.util.Optional;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
public abstract class RowSource extends Source<Dataset<Row>> {
public RowSource(TypedProperties props,
JavaSparkContext sparkContext,
SparkSession sparkSession,
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider, SourceType.ROW);
}
protected abstract Pair<Optional<Dataset<Row>>, String> fetchNextBatch(Optional<String> lastCkptStr,
long sourceLimit);
@Override
protected final InputBatch<Dataset<Row>> fetchNewData(Optional<String> lastCkptStr, long sourceLimit) {
Pair<Optional<Dataset<Row>>, String> res = fetchNextBatch(lastCkptStr, sourceLimit);
return res.getKey().map(dsr -> {
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(dsr.schema());
return new InputBatch<>(res.getKey(), res.getValue(), rowSchemaProvider);
}).orElseGet(() -> new InputBatch<>(res.getKey(), res.getValue()));
}
}

View File

@@ -19,36 +19,67 @@
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.io.Serializable; import java.io.Serializable;
import java.util.Optional; import java.util.Optional;
import org.apache.avro.generic.GenericRecord; import org.apache.log4j.LogManager;
import org.apache.spark.api.java.JavaRDD; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
/** /**
* Represents a source from which we can tail data. Assumes a constructor that takes properties. * Represents a source from which we can tail data. Assumes a constructor that takes properties.
*/ */
public abstract class Source implements Serializable { public abstract class Source<T> implements Serializable {
protected static volatile Logger log = LogManager.getLogger(Source.class);
public enum SourceType {
JSON,
AVRO,
ROW
}
protected transient TypedProperties props; protected transient TypedProperties props;
protected transient JavaSparkContext sparkContext; protected transient JavaSparkContext sparkContext;
protected transient SparkSession sparkSession;
private transient SchemaProvider overriddenSchemaProvider;
protected transient SchemaProvider schemaProvider; private final SourceType sourceType;
protected Source(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
this(props, sparkContext, sparkSession, schemaProvider, SourceType.AVRO);
}
protected Source(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { protected Source(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider, SourceType sourceType) {
this.props = props; this.props = props;
this.sparkContext = sparkContext; this.sparkContext = sparkContext;
this.schemaProvider = schemaProvider; this.sparkSession = sparkSession;
this.overriddenSchemaProvider = schemaProvider;
this.sourceType = sourceType;
} }
protected abstract InputBatch<T> fetchNewData(Optional<String> lastCkptStr, long sourceLimit);
/** /**
* Fetches new data upto sourceLimit, from the provided checkpoint and returns an RDD of the * Main API called by Hoodie Delta Streamer to fetch records
* data, as well as the checkpoint to be written as a result of that. * @param lastCkptStr Last Checkpoint
* @param sourceLimit Source Limit
* @return
*/ */
public abstract Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData( public final InputBatch<T> fetchNext(Optional<String> lastCkptStr, long sourceLimit) {
Optional<String> lastCheckpointStr, long sourceLimit); InputBatch<T> batch = fetchNewData(lastCkptStr, sourceLimit);
// If overriddenSchemaProvider is passed in CLI, use it
return overriddenSchemaProvider == null ? batch : new InputBatch<>(batch.getBatch(),
batch.getCheckpointForNextBatch(), overriddenSchemaProvider);
}
public SourceType getSourceType() {
return sourceType;
}
public SparkSession getSparkSession() {
return sparkSession;
}
} }

View File

@@ -16,7 +16,7 @@
* *
*/ */
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources.helpers;
import com.twitter.bijection.Injection; import com.twitter.bijection.Injection;
import com.twitter.bijection.avro.GenericAvroCodecs; import com.twitter.bijection.avro.GenericAvroCodecs;
@@ -55,6 +55,10 @@ public class AvroConvertor implements Serializable {
this.schemaStr = schemaStr; this.schemaStr = schemaStr;
} }
public AvroConvertor(Schema schema) {
this.schemaStr = schema.toString();
this.schema = schema;
}
private void initSchema() { private void initSchema() {
if (schema == null) { if (schema == null) {

View File

@@ -1,5 +1,5 @@
/* /*
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
* *
* Licensed under the Apache License, Version 2.0 (the "License"); * Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License. * you may not use this file except in compliance with the License.
@@ -16,7 +16,7 @@
* *
*/ */
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources.helpers;
import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
@@ -24,45 +24,38 @@ import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.ImmutablePair; import com.uber.hoodie.common.util.collection.ImmutablePair;
import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.io.IOException; import java.io.IOException;
import java.util.*; import java.util.*;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.RemoteIterator;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
/** public class DFSPathSelector {
* Source to read data from a given DFS directory structure, incrementally
*/
public abstract class DFSSource extends Source {
/** /**
* Configs supported * Configs supported
*/ */
static class Config { static class Config {
private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
} }
private static final List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_"); private static final List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
private final transient FileSystem fs; private final transient FileSystem fs;
private final TypedProperties props;
public DFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { public DFSPathSelector(TypedProperties props, Configuration hadoopConf) {
super(props, sparkContext, schemaProvider); DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP)); this.props = props;
this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), sparkContext.hadoopConfiguration()); this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), hadoopConf);
} }
protected abstract JavaRDD<GenericRecord> fromFiles(final AvroConvertor convertor, String pathStr); public Pair<Optional<String>, String> getNextFilePathsAndMaxModificationTime(
@Override
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(
Optional<String> lastCheckpointStr, long sourceLimit) { Optional<String> lastCheckpointStr, long sourceLimit) {
try { try {
@@ -111,11 +104,9 @@ public abstract class DFSSource extends Source {
// read the files out. // read the files out.
String pathStr = filteredFiles.stream().map(f -> f.getPath().toString()) String pathStr = filteredFiles.stream().map(f -> f.getPath().toString())
.collect(Collectors.joining(",")); .collect(Collectors.joining(","));
String schemaStr = schemaProvider.getSourceSchema().toString();
final AvroConvertor avroConvertor = new AvroConvertor(schemaStr);
return new ImmutablePair<>( return new ImmutablePair<>(
Optional.of(fromFiles(avroConvertor, pathStr)), Optional.ofNullable(pathStr),
String.valueOf(maxModificationTime)); String.valueOf(maxModificationTime));
} catch (IOException ioe) { } catch (IOException ioe) {
throw new HoodieIOException( throw new HoodieIOException(

View File

@@ -0,0 +1,88 @@
package com.uber.hoodie.utilities.sources.helpers;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.collection.Pair;
import java.util.Optional;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Row;
/**
* Helper for Hudi Incremental Source. Has APIs to
* (a) calculate begin and end instant time for incrementally pulling from Hudi source
* (b) Find max seen instant to be set as checkpoint for next fetch.
*/
public class IncrSourceHelper {
/**
* Get a timestamp which is the next value in a descending sequence
*
* @param timestamp Timestamp
*/
private static String getStrictlyLowerTimestamp(String timestamp) {
long ts = Long.parseLong(timestamp);
Preconditions.checkArgument(ts > 0, "Timestamp must be positive");
Long lower = ts - 1;
return "" + lower;
}
/**
* Find begin and end instants to be set for the next fetch
*
* @param jssc Java Spark Context
* @param srcBasePath Base path of Hudi source table
* @param numInstantsPerFetch Max Instants per fetch
* @param beginInstant Last Checkpoint String
* @param readLatestOnMissingBeginInstant when begin instant is missing, allow reading from latest committed instant
* @return begin and end instants
*/
public static Pair<String, String> calculateBeginAndEndInstants(
JavaSparkContext jssc, String srcBasePath, int numInstantsPerFetch, Optional<String> beginInstant,
boolean readLatestOnMissingBeginInstant) {
Preconditions.checkArgument(numInstantsPerFetch > 0, "Make sure the config"
+ " hoodie.deltastreamer.source.hoodieincr.num_instants is set to a positive value");
HoodieTableMetaClient srcMetaClient = new HoodieTableMetaClient(jssc.hadoopConfiguration(),
srcBasePath, true);
final HoodieTimeline activeCommitTimeline =
srcMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
String beginInstantTime = beginInstant.orElseGet(() -> {
if (readLatestOnMissingBeginInstant) {
Optional<HoodieInstant> lastInstant = activeCommitTimeline.lastInstant();
return lastInstant.map(hoodieInstant -> getStrictlyLowerTimestamp(hoodieInstant.getTimestamp())).orElse("000");
} else {
throw new IllegalArgumentException("Missing begin instant for incremental pull. For reading from latest "
+ "committed instant set hoodie.deltastreamer.source.hoodie.read_latest_on_midding_ckpt to true");
}
});
Optional<HoodieInstant> nthInstant =
activeCommitTimeline.findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstants().reduce((x, y) -> y);
return Pair.of(beginInstantTime, nthInstant.map(instant -> instant.getTimestamp()).orElse(beginInstantTime));
}
/**
* Validate instant time seen in the incoming row
*
* @param row Input Row
* @param instantTime Hoodie Instant time of the row
* @param sinceInstant begin instant of the batch
* @param endInstant end instant of the batch
*/
public static void validateInstantTime(Row row, String instantTime, String sinceInstant, String endInstant) {
Preconditions.checkNotNull(instantTime);
Preconditions.checkArgument(HoodieTimeline.compareTimestamps(instantTime,
sinceInstant, HoodieTimeline.GREATER),
"Instant time(_hoodie_commit_time) in row (" + row + ") was : " + instantTime
+ "but expected to be between " + sinceInstant + "(excl) - "
+ endInstant + "(incl)");
Preconditions.checkArgument(HoodieTimeline.compareTimestamps(instantTime,
endInstant, HoodieTimeline.LESSER_OR_EQUAL),
"Instant time(_hoodie_commit_time) in row (" + row + ") was : " + instantTime
+ "but expected to be between " + sinceInstant + "(excl) - " + endInstant + "(incl)");
}
}

View File

@@ -16,24 +16,22 @@
* *
*/ */
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources.helpers;
import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.DataSourceUtils;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.ImmutablePair;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
import com.uber.hoodie.utilities.schema.SchemaProvider; import java.util.Arrays;
import java.util.Collections;
import java.util.*; import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import kafka.common.TopicAndPartition; import kafka.common.TopicAndPartition;
import org.apache.avro.generic.GenericRecord;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.streaming.kafka.KafkaCluster; import org.apache.spark.streaming.kafka.KafkaCluster;
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset; import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset;
import org.apache.spark.streaming.kafka.OffsetRange; import org.apache.spark.streaming.kafka.OffsetRange;
@@ -49,14 +47,13 @@ import scala.util.Either;
/** /**
* Source to read data from Kafka, incrementally * Source to read data from Kafka, incrementally
*/ */
public abstract class KafkaSource extends Source { public class KafkaOffsetGen {
private static volatile Logger log = LogManager.getLogger(KafkaSource.class); private static volatile Logger log = LogManager.getLogger(KafkaOffsetGen.class);
private static long DEFAULT_MAX_EVENTS_TO_READ = 1000000; // 1M events max private static long DEFAULT_MAX_EVENTS_TO_READ = 1000000; // 1M events max
public static class CheckpointUtils {
static class CheckpointUtils {
/** /**
* Reconstruct checkpoint from string. * Reconstruct checkpoint from string.
@@ -90,7 +87,6 @@ public abstract class KafkaSource extends Source {
return sb.toString(); return sb.toString();
} }
/** /**
* Compute the offset ranges to read from Kafka, while handling newly added partitions, skews, event limits. * Compute the offset ranges to read from Kafka, while handling newly added partitions, skews, event limits.
* *
@@ -174,19 +170,18 @@ public abstract class KafkaSource extends Source {
* Configs to be passed for this source. All standard Kafka consumer configs are also respected * Configs to be passed for this source. All standard Kafka consumer configs are also respected
*/ */
static class Config { static class Config {
private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LARGEST; private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LARGEST;
} }
private final HashMap<String, String> kafkaParams;
protected HashMap<String, String> kafkaParams; private final TypedProperties props;
protected final String topicName; protected final String topicName;
public KafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { public KafkaOffsetGen(TypedProperties props) {
super(props, sparkContext, schemaProvider); this.props = props;
kafkaParams = new HashMap<String, String>();
kafkaParams = new HashMap<>();
for (Object prop : props.keySet()) { for (Object prop : props.keySet()) {
kafkaParams.put(prop.toString(), props.getString(prop.toString())); kafkaParams.put(prop.toString(), props.getString(prop.toString()));
} }
@@ -194,11 +189,7 @@ public abstract class KafkaSource extends Source {
topicName = props.getString(Config.KAFKA_TOPIC_NAME); topicName = props.getString(Config.KAFKA_TOPIC_NAME);
} }
protected abstract JavaRDD<GenericRecord> toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor); public OffsetRange[] getNextOffsetRanges(Optional<String> lastCheckpointStr, long sourceLimit) {
@Override
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(
Optional<String> lastCheckpointStr, long sourceLimit) {
// Obtain current metadata for the topic // Obtain current metadata for the topic
KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams));
@@ -240,16 +231,15 @@ public abstract class KafkaSource extends Source {
// Come up with final set of OffsetRanges to read (account for new partitions, limit number of events) // Come up with final set of OffsetRanges to read (account for new partitions, limit number of events)
long numEvents = Math.min(DEFAULT_MAX_EVENTS_TO_READ, sourceLimit); long numEvents = Math.min(DEFAULT_MAX_EVENTS_TO_READ, sourceLimit);
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents); OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents);
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
if (totalNewMsgs <= 0) { return offsetRanges;
return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.orElse(""));
} else {
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName);
} }
// Produce a RDD[GenericRecord] public String getTopicName() {
final AvroConvertor avroConvertor = new AvroConvertor(schemaProvider.getSourceSchema().toString()); return topicName;
JavaRDD<GenericRecord> newDataRDD = toAvroRDD(offsetRanges, avroConvertor); }
return new ImmutablePair<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges));
public HashMap<String, String> getKafkaParams() {
return kafkaParams;
} }
} }

View File

@@ -0,0 +1,37 @@
/*
* Copyright (c) 2018 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.transform;
import com.uber.hoodie.common.util.TypedProperties;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
/**
* Identity transformer
*/
public class IdentityTransformer implements Transformer {
@Override
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
Dataset<Row> rowDataset, TypedProperties properties) {
return rowDataset;
}
}

View File

@@ -0,0 +1,66 @@
/*
* Copyright (c) 2018 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.transform;
import com.uber.hoodie.common.util.TypedProperties;
import java.util.UUID;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
/**
* A transformer that allows a sql-query template be used to transform the source before writing to Hudi data-set.
*
* The query should reference the source as a table named "\<SRC\>"
*/
public class SqlQueryBasedTransformer implements Transformer {
private static volatile Logger log = LogManager.getLogger(SqlQueryBasedTransformer.class);
private static final String SRC_PATTERN = "<SRC>";
private static final String TMP_TABLE = "HOODIE_SRC_TMP_TABLE_";
/**
* Configs supported
*/
static class Config {
private static final String TRANSFORMER_SQL = "hoodie.deltastreamer.transformer.sql";
}
@Override
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
Dataset<Row> rowDataset, TypedProperties properties) {
String transformerSQL = properties.getString(Config.TRANSFORMER_SQL);
if (null == transformerSQL) {
throw new IllegalArgumentException("Missing configuration : (" + Config.TRANSFORMER_SQL + ")");
}
// tmp table name doesn't like dashes
String tmpTable = TMP_TABLE.concat(UUID.randomUUID().toString().replace("-", "_"));
log.info("Registering tmp table : " + tmpTable);
rowDataset.registerTempTable(tmpTable);
String sqlStr = transformerSQL.replaceAll(SRC_PATTERN, tmpTable);
log.info("SQL Query for transformation : (" + sqlStr + ")");
return sparkSession.sql(sqlStr);
}
}

View File

@@ -0,0 +1,43 @@
/*
* Copyright (c) 2018 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.transform;
import com.uber.hoodie.common.util.TypedProperties;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
/**
* Transform source to target dataset before writing
*/
public interface Transformer {
/**
* Transform source RDD to target RDD
*
* @param jsc JavaSparkContext
* @param rowDataset Source DataSet
* @param sparkSession Spark Session
* @param properties Config properties
* @return Transformed Dataset
*/
Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
Dataset<Row> rowDataset, TypedProperties properties);
}

View File

@@ -19,8 +19,10 @@
package com.uber.hoodie.utilities; package com.uber.hoodie.utilities;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import com.uber.hoodie.DataSourceWriteOptions;
import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
@@ -28,17 +30,31 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.DFSPropertiesConfiguration; import com.uber.hoodie.common.util.DFSPropertiesConfiguration;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.hive.HiveSyncConfig;
import com.uber.hoodie.hive.HoodieHiveClient;
import com.uber.hoodie.hive.MultiPartKeysValueExtractor;
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer; import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer;
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.Operation; import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
import com.uber.hoodie.utilities.sources.HoodieIncrSource;
import com.uber.hoodie.utilities.sources.TestDataSource; import com.uber.hoodie.utilities.sources.TestDataSource;
import com.uber.hoodie.utilities.transform.SqlQueryBasedTransformer;
import com.uber.hoodie.utilities.transform.Transformer;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row; import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.api.java.UDF4;
import org.apache.spark.sql.functions;
import org.apache.spark.sql.types.DataTypes;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@@ -55,17 +71,43 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
@BeforeClass @BeforeClass
public static void initClass() throws Exception { public static void initClass() throws Exception {
UtilitiesTestBase.initClass(); UtilitiesTestBase.initClass(true);
// prepare the configs. // prepare the configs.
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/base.properties"); UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/base.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/sql-transformer.properties", dfs,
dfsBasePath + "/sql-transformer.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc"); UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target.avsc", dfs, dfsBasePath + "/target.avsc");
TypedProperties props = new TypedProperties(); TypedProperties props = new TypedProperties();
props.setProperty("include", "base.properties"); props.setProperty("include", "sql-transformer.properties");
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there"); props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
// Hive Configs
props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/");
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1");
props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "hive_trips");
props.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), "false");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
MultiPartKeysValueExtractor.class.getName());
UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source.properties"); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source.properties");
// Properties used for the delta-streamer which incrementally pulls from upstream Hudi source table and writes to
// downstream hudi table
TypedProperties downstreamProps = new TypedProperties();
downstreamProps.setProperty("include", "base.properties");
downstreamProps.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
downstreamProps.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
// Source schema is the target schema of upstream table
downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/target.avsc");
downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
UtilitiesTestBase.Helpers.savePropsToDFS(downstreamProps, dfs,
dfsBasePath + "/test-downstream-source.properties");
} }
@AfterClass @AfterClass
@@ -86,17 +128,48 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
} }
static class TestHelpers { static class TestHelpers {
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op) { static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op) {
return makeConfig(basePath, op, TripsWithDistanceTransformer.class.getName());
}
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String transformerClassName) {
return makeConfig(basePath, op, transformerClassName, false);
}
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String transformerClassName,
boolean enableHiveSync) {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config(); HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetBasePath = basePath; cfg.targetBasePath = basePath;
cfg.targetTableName = "hoodie_trips"; cfg.targetTableName = "hoodie_trips";
cfg.storageType = "COPY_ON_WRITE"; cfg.storageType = "COPY_ON_WRITE";
cfg.sourceClassName = TestDataSource.class.getName(); cfg.sourceClassName = TestDataSource.class.getName();
cfg.transformerClassName = transformerClassName;
cfg.operation = op; cfg.operation = op;
cfg.enableHiveSync = enableHiveSync;
cfg.sourceOrderingField = "timestamp"; cfg.sourceOrderingField = "timestamp";
cfg.propsFilePath = dfsBasePath + "/test-source.properties"; cfg.propsFilePath = dfsBasePath + "/test-source.properties";
cfg.sourceLimit = 1000; cfg.sourceLimit = 1000;
cfg.schemaProviderClassName = FilebasedSchemaProvider.class.getName();
return cfg;
}
static HoodieDeltaStreamer.Config makeConfigForHudiIncrSrc(String srcBasePath, String basePath, Operation op,
boolean addReadLatestOnMissingCkpt) {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetBasePath = basePath;
cfg.targetTableName = "hoodie_trips_copy";
cfg.storageType = "COPY_ON_WRITE";
cfg.sourceClassName = HoodieIncrSource.class.getName();
cfg.operation = op;
cfg.sourceOrderingField = "timestamp";
cfg.propsFilePath = dfsBasePath + "/test-downstream-source.properties";
cfg.sourceLimit = 1000;
List<String> cfgs = new ArrayList<>();
cfgs.add("hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt=" + addReadLatestOnMissingCkpt);
cfgs.add("hoodie.deltastreamer.source.hoodieincr.path=" + srcBasePath);
// No partition
cfgs.add("hoodie.deltastreamer.source.hoodieincr.partition.fields=datestr");
cfg.configs = cfgs;
return cfg; return cfg;
} }
@@ -110,15 +183,30 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
.sort("_hoodie_commit_time").collectAsList(); .sort("_hoodie_commit_time").collectAsList();
} }
static void assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits) static void assertDistanceCount(long expected, String datasetPath, SQLContext sqlContext) {
sqlContext.read().format("com.uber.hoodie").load(datasetPath).registerTempTable("tmp_trips");
long recordCount =
sqlContext.sparkSession().sql("select * from tmp_trips where haversine_distance is not NULL").count();
assertEquals(expected, recordCount);
}
static void assertDistanceCountWithExactValue(long expected, String datasetPath, SQLContext sqlContext) {
sqlContext.read().format("com.uber.hoodie").load(datasetPath).registerTempTable("tmp_trips");
long recordCount =
sqlContext.sparkSession().sql("select * from tmp_trips where haversine_distance = 1.0").count();
assertEquals(expected, recordCount);
}
static String assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits)
throws IOException { throws IOException {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath); HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieInstant lastCommit = timeline.lastInstant().get(); HoodieInstant lastInstant = timeline.lastInstant().get();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class); timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class);
assertEquals(totalCommits, timeline.countInstants()); assertEquals(totalCommits, timeline.countInstants());
assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY)); assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY));
return lastInstant.getTimestamp();
} }
} }
@@ -152,12 +240,14 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT); HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT);
new HoodieDeltaStreamer(cfg, jsc).sync(); new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
// No new data => no commits. // No new data => no commits.
cfg.sourceLimit = 0; cfg.sourceLimit = 0;
new HoodieDeltaStreamer(cfg, jsc).sync(); new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
// upsert() #1 // upsert() #1
@@ -165,11 +255,94 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
cfg.operation = Operation.UPSERT; cfg.operation = Operation.UPSERT;
new HoodieDeltaStreamer(cfg, jsc).sync(); new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(2000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2); TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext); List<Row> counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext);
assertEquals(2000, counts.get(0).getLong(1)); assertEquals(2000, counts.get(0).getLong(1));
} }
/**
* Test Bulk Insert and upserts with hive syncing. Tests Hudi incremental processing using a 2 step pipeline
* The first step involves using a SQL template to transform a source
* TEST-DATA-SOURCE ============================> HUDI TABLE 1 ===============> HUDI TABLE 2
* (incr-pull with transform) (incr-pull)
* Hudi Table 1 is synced with Hive.
* @throws Exception
*/
@Test
public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() throws Exception {
String datasetBasePath = dfsBasePath + "/test_dataset2";
String downstreamDatasetBasePath = dfsBasePath + "/test_downstream_dataset2";
HiveSyncConfig hiveSyncConfig = getHiveSyncConfig(datasetBasePath, "hive_trips");
// Initial bulk insert to ingest to first hudi table
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT,
SqlQueryBasedTransformer.class.getName(), true);
new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, datasetBasePath + "/*/*.parquet", sqlContext);
String lastInstantForUpstreamTable = TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
// Now incrementally pull from the above hudi table and ingest to second table
HoodieDeltaStreamer.Config downstreamCfg =
TestHelpers.makeConfigForHudiIncrSrc(datasetBasePath, downstreamDatasetBasePath, Operation.BULK_INSERT, true);
new HoodieDeltaStreamer(downstreamCfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamDatasetBasePath, dfs, 1);
// No new data => no commits for upstream table
cfg.sourceLimit = 0;
new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
// with no change in upstream table, no change in downstream too when pulled.
new HoodieDeltaStreamer(downstreamCfg, jsc).sync();
TestHelpers.assertRecordCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamDatasetBasePath, dfs, 1);
// upsert() #1 on upstream hudi table
cfg.sourceLimit = 2000;
cfg.operation = Operation.UPSERT;
new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(2000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(2000, datasetBasePath + "/*/*.parquet", sqlContext);
lastInstantForUpstreamTable = TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext);
assertEquals(2000, counts.get(0).getLong(1));
// Incrementally pull changes in upstream hudi table and apply to downstream table
downstreamCfg =
TestHelpers.makeConfigForHudiIncrSrc(datasetBasePath, downstreamDatasetBasePath, Operation.UPSERT, false);
downstreamCfg.sourceLimit = 2000;
new HoodieDeltaStreamer(downstreamCfg, jsc).sync();
TestHelpers.assertRecordCount(2000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(2000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(2000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
String finalInstant =
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamDatasetBasePath, dfs, 2);
counts = TestHelpers.countsPerCommit(downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
assertEquals(2000, counts.get(0).getLong(1));
// Test Hive integration
HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), dfs);
assertTrue("Table " + hiveSyncConfig.tableName + " should exist",
hiveClient.doesTableExist());
assertEquals("Table partitions should match the number of partitions we wrote", 1,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
lastInstantForUpstreamTable, hiveClient.getLastCommitTimeSynced().get());
}
@Test @Test
public void testFilterDupes() throws Exception { public void testFilterDupes() throws Exception {
String datasetBasePath = dfsBasePath + "/test_dupes_dataset"; String datasetBasePath = dfsBasePath + "/test_dupes_dataset";
@@ -192,4 +365,57 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
assertEquals(1000, counts.get(0).getLong(1)); assertEquals(1000, counts.get(0).getLong(1));
assertEquals(1000, counts.get(1).getLong(1)); assertEquals(1000, counts.get(1).getLong(1));
} }
/**
* UDF to calculate Haversine distance
*/
public static class DistanceUDF implements UDF4<Double, Double, Double, Double, Double> {
/**
*
* Taken from https://stackoverflow.com/questions/3694380/calculating-distance-between-two-points-using-latitude-
* longitude-what-am-i-doi
* Calculate distance between two points in latitude and longitude taking
* into account height difference. If you are not interested in height
* difference pass 0.0. Uses Haversine method as its base.
*
* lat1, lon1 Start point lat2, lon2 End point el1 Start altitude in meters
* el2 End altitude in meters
* @returns Distance in Meters
*/
@Override
public Double call(Double lat1, Double lat2, Double lon1, Double lon2) {
final int R = 6371; // Radius of the earth
double latDistance = Math.toRadians(lat2 - lat1);
double lonDistance = Math.toRadians(lon2 - lon1);
double a = Math.sin(latDistance / 2) * Math.sin(latDistance / 2)
+ Math.cos(Math.toRadians(lat1)) * Math.cos(Math.toRadians(lat2))
* Math.sin(lonDistance / 2) * Math.sin(lonDistance / 2);
double c = 2 * Math.atan2(Math.sqrt(a), Math.sqrt(1 - a));
double distance = R * c * 1000; // convert to meters
double height = 0;
distance = Math.pow(distance, 2) + Math.pow(height, 2);
return Math.sqrt(distance);
}
}
/**
* Adds a new field "haversine_distance" to the row
*/
public static class TripsWithDistanceTransformer implements Transformer {
@Override
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
Dataset<Row> rowDataset, TypedProperties properties) {
rowDataset.sqlContext().udf().register("distance_udf", new DistanceUDF(), DataTypes.DoubleType);
return rowDataset.withColumn("haversine_distance",
functions.callUDF("distance_udf", functions.col("begin_lat"),
functions.col("end_lat"), functions.col("begin_lon"), functions.col("end_lat")));
}
}
} }

View File

@@ -18,10 +18,16 @@
package com.uber.hoodie.utilities; package com.uber.hoodie.utilities;
import com.google.common.collect.ImmutableList;
import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.minicluster.HdfsTestService; import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.hive.HiveSyncConfig;
import com.uber.hoodie.hive.HoodieHiveClient;
import com.uber.hoodie.hive.util.HiveTestService;
import com.uber.hoodie.utilities.sources.TestDataSource; import com.uber.hoodie.utilities.sources.TestDataSource;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.IOException; import java.io.IOException;
@@ -32,8 +38,11 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hive.service.server.HiveServer2;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.SparkSession;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@@ -51,15 +60,26 @@ public class UtilitiesTestBase {
protected static MiniDFSCluster dfsCluster; protected static MiniDFSCluster dfsCluster;
protected static DistributedFileSystem dfs; protected static DistributedFileSystem dfs;
protected transient JavaSparkContext jsc = null; protected transient JavaSparkContext jsc = null;
protected transient SparkSession sparkSession = null;
protected transient SQLContext sqlContext; protected transient SQLContext sqlContext;
protected static HiveServer2 hiveServer;
@BeforeClass @BeforeClass
public static void initClass() throws Exception { public static void initClass() throws Exception {
initClass(false);
}
static void initClass(boolean startHiveService) throws Exception {
hdfsTestService = new HdfsTestService(); hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true); dfsCluster = hdfsTestService.start(true);
dfs = dfsCluster.getFileSystem(); dfs = dfsCluster.getFileSystem();
dfsBasePath = dfs.getWorkingDirectory().toString(); dfsBasePath = dfs.getWorkingDirectory().toString();
dfs.mkdirs(new Path(dfsBasePath)); dfs.mkdirs(new Path(dfsBasePath));
if (startHiveService) {
HiveTestService hiveService = new HiveTestService(hdfsTestService.getHadoopConf());
hiveServer = hiveService.start();
clearHiveDb();
}
} }
@AfterClass @AfterClass
@@ -67,6 +87,9 @@ public class UtilitiesTestBase {
if (hdfsTestService != null) { if (hdfsTestService != null) {
hdfsTestService.stop(); hdfsTestService.stop();
} }
if (hiveServer != null) {
hiveServer.stop();
}
} }
@Before @Before
@@ -74,6 +97,7 @@ public class UtilitiesTestBase {
TestDataSource.initDataGen(); TestDataSource.initDataGen();
jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]"); jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
sqlContext = new SQLContext(jsc); sqlContext = new SQLContext(jsc);
sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
} }
@After @After
@@ -84,6 +108,42 @@ public class UtilitiesTestBase {
} }
} }
/**
* Helper to get hive sync config
* @param basePath
* @param tableName
* @return
*/
protected static HiveSyncConfig getHiveSyncConfig(String basePath, String tableName) {
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.jdbcUrl = "jdbc:hive2://127.0.0.1:9999/";
hiveSyncConfig.hiveUser = "";
hiveSyncConfig.hivePass = "";
hiveSyncConfig.databaseName = "testdb1";
hiveSyncConfig.tableName = tableName;
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.assumeDatePartitioning = false;
hiveSyncConfig.partitionFields = new ImmutableList.Builder<String>().add("datestr").build();
return hiveSyncConfig;
}
/**
* Initialize Hive DB
* @throws IOException
*/
private static void clearHiveDb() throws IOException {
HiveConf hiveConf = new HiveConf();
// Create Dummy hive sync config
HiveSyncConfig hiveSyncConfig = getHiveSyncConfig("/dummy", "dummy");
hiveConf.addResource(hiveServer.getHiveConf());
HoodieTableMetaClient.initTableType(dfs.getConf(), hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
hiveSyncConfig.tableName, null);
HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveConf, dfs);
client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName);
client.updateHiveSQL("create database " + hiveSyncConfig.databaseName);
client.close();
}
public static class Helpers { public static class Helpers {
// to get hold of resources bundled with jar // to get hold of resources bundled with jar

View File

@@ -20,16 +20,20 @@ package com.uber.hoodie.utilities.sources;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import com.uber.hoodie.AvroConversionUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.UtilitiesTestBase; import com.uber.hoodie.utilities.UtilitiesTestBase;
import com.uber.hoodie.utilities.deltastreamer.SourceFormatAdapter;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
import java.io.IOException; import java.io.IOException;
import java.util.Optional; import java.util.Optional;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@@ -37,7 +41,7 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
/** /**
* Basic tests against all subclasses of {@link DFSSource} * Basic tests against all subclasses of {@link JsonDFSSource}
*/ */
public class TestDFSSource extends UtilitiesTestBase { public class TestDFSSource extends UtilitiesTestBase {
@@ -71,34 +75,47 @@ public class TestDFSSource extends UtilitiesTestBase {
TypedProperties props = new TypedProperties(); TypedProperties props = new TypedProperties();
props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/jsonFiles"); props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/jsonFiles");
JsonDFSSource jsonSource = new JsonDFSSource(props, jsc, schemaProvider); JsonDFSSource jsonDFSSource = new JsonDFSSource(props, jsc, sparkSession, schemaProvider);
SourceFormatAdapter jsonSource = new SourceFormatAdapter(jsonDFSSource);
// 1. Extract without any checkpoint => get all the data, respecting sourceLimit // 1. Extract without any checkpoint => get all the data, respecting sourceLimit
assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), Long.MAX_VALUE).getKey()); assertEquals(Optional.empty(), jsonSource.fetchNewDataInAvroFormat(Optional.empty(), Long.MAX_VALUE).getBatch());
UtilitiesTestBase.Helpers.saveStringsToDFS( UtilitiesTestBase.Helpers.saveStringsToDFS(
Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs,
dfsBasePath + "/jsonFiles/1.json"); dfsBasePath + "/jsonFiles/1.json");
assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), 10).getKey()); assertEquals(Optional.empty(), jsonSource.fetchNewDataInAvroFormat(Optional.empty(), 10).getBatch());
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch1 = jsonSource.fetchNewData(Optional.empty(), 1000000); InputBatch<JavaRDD<GenericRecord>> fetch1 =
assertEquals(100, fetch1.getKey().get().count()); jsonSource.fetchNewDataInAvroFormat(Optional.empty(), 1000000);
assertEquals(100, fetch1.getBatch().get().count());
// Test json -> Row format
InputBatch<Dataset<Row>> fetch1AsRows =
jsonSource.fetchNewDataInRowFormat(Optional.empty(), 1000000);
assertEquals(100, fetch1AsRows.getBatch().get().count());
// Test Avro -> Row format
Dataset<Row> fetch1Rows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()),
schemaProvider.getSourceSchema().toString(), jsonDFSSource.getSparkSession());
assertEquals(100, fetch1Rows.count());
// 2. Produce new data, extract new data // 2. Produce new data, extract new data
UtilitiesTestBase.Helpers.saveStringsToDFS( UtilitiesTestBase.Helpers.saveStringsToDFS(
Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)), Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)),
dfs, dfsBasePath + "/jsonFiles/2.json"); dfs, dfsBasePath + "/jsonFiles/2.json");
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch2 = jsonSource.fetchNewData( InputBatch<Dataset<Row>> fetch2 = jsonSource.fetchNewDataInRowFormat(
Optional.of(fetch1.getValue()), Long.MAX_VALUE); Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(10000, fetch2.getKey().get().count()); assertEquals(10000, fetch2.getBatch().get().count());
// 3. Extract with previous checkpoint => gives same data back (idempotent) // 3. Extract with previous checkpoint => gives same data back (idempotent)
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch3 = jsonSource.fetchNewData( InputBatch<Dataset<Row>> fetch3 = jsonSource.fetchNewDataInRowFormat(
Optional.of(fetch1.getValue()), Long.MAX_VALUE); Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(10000, fetch3.getKey().get().count()); assertEquals(10000, fetch3.getBatch().get().count());
assertEquals(fetch2.getValue(), fetch3.getValue()); assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch());
fetch3.getBatch().get().registerTempTable("test_dfs_table");
Dataset<Row> rowDataset = new SQLContext(jsc.sc()).sql("select * from test_dfs_table");
assertEquals(10000, rowDataset.count());
// 4. Extract with latest checkpoint => no new data returned // 4. Extract with latest checkpoint => no new data returned
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch4 = jsonSource.fetchNewData( InputBatch<JavaRDD<GenericRecord>> fetch4 = jsonSource.fetchNewDataInAvroFormat(
Optional.of(fetch2.getValue()), Long.MAX_VALUE); Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(Optional.empty(), fetch4.getKey()); assertEquals(Optional.empty(), fetch4.getBatch());
} }
} }

View File

@@ -21,8 +21,6 @@ package com.uber.hoodie.utilities.sources;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.ImmutablePair;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@@ -35,11 +33,12 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
/** /**
* An implementation of {@link Source}, that emits test upserts. * An implementation of {@link Source}, that emits test upserts.
*/ */
public class TestDataSource extends Source { public class TestDataSource extends AvroSource {
private static volatile Logger log = LogManager.getLogger(TestDataSource.class); private static volatile Logger log = LogManager.getLogger(TestDataSource.class);
@@ -54,8 +53,9 @@ public class TestDataSource extends Source {
dataGenerator = null; dataGenerator = null;
} }
public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
super(props, sparkContext, schemaProvider); SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
} }
private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) { private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
@@ -68,14 +68,14 @@ public class TestDataSource extends Source {
} }
@Override @Override
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr, protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Optional<String> lastCheckpointStr,
long sourceLimit) { long sourceLimit) {
int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0); int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0);
String commitTime = String.format("%05d", nextCommitNum); String commitTime = String.format("%05d", nextCommitNum);
// No new data. // No new data.
if (sourceLimit <= 0) { if (sourceLimit <= 0) {
return new ImmutablePair<>(Optional.empty(), commitTime); return new InputBatch<>(Optional.empty(), commitTime);
} }
// generate `sourceLimit` number of upserts each time. // generate `sourceLimit` number of upserts each time.
@@ -94,6 +94,6 @@ public class TestDataSource extends Source {
} }
JavaRDD<GenericRecord> avroRDD = sparkContext.<GenericRecord>parallelize(records, 4); JavaRDD<GenericRecord> avroRDD = sparkContext.<GenericRecord>parallelize(records, 4);
return new ImmutablePair<>(Optional.of(avroRDD), commitTime); return new InputBatch<>(Optional.of(avroRDD), commitTime);
} }
} }

View File

@@ -18,20 +18,23 @@
package com.uber.hoodie.utilities.sources; package com.uber.hoodie.utilities.sources;
import static com.uber.hoodie.utilities.sources.KafkaSource.CheckpointUtils;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import com.uber.hoodie.AvroConversionUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.UtilitiesTestBase; import com.uber.hoodie.utilities.UtilitiesTestBase;
import com.uber.hoodie.utilities.deltastreamer.SourceFormatAdapter;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.Optional; import java.util.Optional;
import kafka.common.TopicAndPartition; import kafka.common.TopicAndPartition;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset; import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset;
import org.apache.spark.streaming.kafka.KafkaTestUtils; import org.apache.spark.streaming.kafka.KafkaTestUtils;
import org.apache.spark.streaming.kafka.OffsetRange; import org.apache.spark.streaming.kafka.OffsetRange;
@@ -42,7 +45,7 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
/** /**
* Tests against {@link KafkaSource} * Tests against {@link AvroKafkaSource}
*/ */
public class TestKafkaSource extends UtilitiesTestBase { public class TestKafkaSource extends UtilitiesTestBase {
@@ -89,30 +92,44 @@ public class TestKafkaSource extends UtilitiesTestBase {
props.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); props.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
props.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); props.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
Source kafkaSource = new JsonKafkaSource(props, jsc, schemaProvider); Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider);
SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource);
// 1. Extract without any checkpoint => get all the data, respecting sourceLimit // 1. Extract without any checkpoint => get all the data, respecting sourceLimit
assertEquals(Optional.empty(), kafkaSource.fetchNewData(Optional.empty(), Long.MAX_VALUE).getKey()); assertEquals(Optional.empty(), kafkaSource.fetchNewDataInAvroFormat(Optional.empty(), Long.MAX_VALUE).getBatch());
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000)));
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch1 = kafkaSource.fetchNewData(Optional.empty(), 900); InputBatch<JavaRDD<GenericRecord>> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Optional.empty(), 900);
assertEquals(900, fetch1.getKey().get().count()); assertEquals(900, fetch1.getBatch().get().count());
// Test Avro To DataFrame<Row> path
Dataset<Row> fetch1AsRows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()),
schemaProvider.getSourceSchema().toString(), jsonSource.getSparkSession());
assertEquals(900, fetch1AsRows.count());
// 2. Produce new data, extract new data // 2. Produce new data, extract new data
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000)));
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch2 = kafkaSource.fetchNewData( InputBatch<Dataset<Row>> fetch2 = kafkaSource.fetchNewDataInRowFormat(
Optional.of(fetch1.getValue()), Long.MAX_VALUE); Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(1100, fetch2.getKey().get().count()); assertEquals(1100, fetch2.getBatch().get().count());
// 3. Extract with previous checkpoint => gives same data back (idempotent) // 3. Extract with previous checkpoint => gives same data back (idempotent)
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch3 = kafkaSource.fetchNewData( InputBatch<JavaRDD<GenericRecord>> fetch3 = kafkaSource.fetchNewDataInAvroFormat(
Optional.of(fetch1.getValue()), Long.MAX_VALUE); Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(fetch2.getKey().get().count(), fetch3.getKey().get().count()); assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count());
assertEquals(fetch2.getValue(), fetch3.getValue()); assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch());
// Same using Row API
InputBatch<Dataset<Row>> fetch3AsRows =
kafkaSource.fetchNewDataInRowFormat(Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(fetch2.getBatch().get().count(), fetch3AsRows.getBatch().get().count());
assertEquals(fetch2.getCheckpointForNextBatch(), fetch3AsRows.getCheckpointForNextBatch());
// 4. Extract with latest checkpoint => no new data returned // 4. Extract with latest checkpoint => no new data returned
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch4 = kafkaSource.fetchNewData( InputBatch<JavaRDD<GenericRecord>> fetch4 = kafkaSource.fetchNewDataInAvroFormat(
Optional.of(fetch2.getValue()), Long.MAX_VALUE); Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(Optional.empty(), fetch4.getKey()); assertEquals(Optional.empty(), fetch4.getBatch());
// Same using Row API
InputBatch<Dataset<Row>> fetch4AsRows =
kafkaSource.fetchNewDataInRowFormat(Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(Optional.empty(), fetch4AsRows.getBatch());
} }

View File

@@ -0,0 +1,19 @@
#
# Copyright (c) 2018 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.
#
#
#
include=base.properties
hoodie.deltastreamer.transformer.sql=SELECT a.timestamp, a._row_key, a.rider, a.driver, a.begin_lat, a.begin_lon, a.end_lat, a.end_lon, a.fare, CAST(1.0 AS DOUBLE) AS haversine_distance FROM <SRC> a

View File

@@ -0,0 +1,37 @@
{
"type" : "record",
"name" : "triprec",
"fields" : [
{
"name" : "timestamp",
"type" : "double"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "rider",
"type" : "string"
}, {
"name" : "driver",
"type" : "string"
}, {
"name" : "begin_lat",
"type" : "double"
}, {
"name" : "begin_lon",
"type" : "double"
}, {
"name" : "end_lat",
"type" : "double"
}, {
"name" : "end_lon",
"type" : "double"
}, {
"name" : "fare",
"type" : "double"
}, {
"name" : "haversine_distance",
"type" : "double"
}]
}

View File

@@ -66,6 +66,48 @@
<artifactId>hadoop-auth</artifactId> <artifactId>hadoop-auth</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-shims</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-serde</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId> <artifactId>hadoop-hdfs</artifactId>
@@ -182,116 +224,4 @@
<properties> <properties>
<checkstyle.skip>true</checkstyle.skip> <checkstyle.skip>true</checkstyle.skip>
</properties> </properties>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<properties>
<hiveJarSuffix />
</properties>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-shims</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-serde</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive12.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<properties>
<hiveJarSuffix>.hive11</hiveJarSuffix>
</properties>
<dependencies>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-shims</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-serde</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive11.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -44,6 +44,26 @@
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-auth</artifactId> <artifactId>hadoop-auth</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency> <dependency>
<groupId>com.google.guava</groupId> <groupId>com.google.guava</groupId>
<artifactId>guava</artifactId> <artifactId>guava</artifactId>
@@ -195,73 +215,4 @@
<properties> <properties>
<checkstyle.skip>true</checkstyle.skip> <checkstyle.skip>true</checkstyle.skip>
</properties> </properties>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<properties>
<hiveJarSuffix />
</properties>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive12.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<properties>
<hiveJarSuffix>.hive11</hiveJarSuffix>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-service</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
<version>${hive11.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

View File

@@ -239,6 +239,26 @@
<groupId>org.apache.avro</groupId> <groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId> <artifactId>avro</artifactId>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
</dependency>
<dependency> <dependency>
<groupId>org.apache.commons</groupId> <groupId>org.apache.commons</groupId>
<artifactId>commons-configuration2</artifactId> <artifactId>commons-configuration2</artifactId>
@@ -269,74 +289,5 @@
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
</dependencies> </dependencies>
<profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<properties>
<hiveJarSuffix />
</properties>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive12.version}</version>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive12.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<properties>
<hiveJarSuffix>.hive11</hiveJarSuffix>
</properties>
<dependencies>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>${hive11.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive11.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project> </project>

157
pom.xml
View File

@@ -129,10 +129,8 @@
<log4j.version>1.2.17</log4j.version> <log4j.version>1.2.17</log4j.version>
<joda.version>2.9.9</joda.version> <joda.version>2.9.9</joda.version>
<hadoop.version>2.7.3</hadoop.version> <hadoop.version>2.7.3</hadoop.version>
<hive12.groupid>org.apache.hive</hive12.groupid> <hive.groupid>org.apache.hive</hive.groupid>
<hive12.version>1.2.1</hive12.version> <hive.version>1.2.1</hive.version>
<hive11.groupid>org.apache.hive</hive11.groupid>
<hive11.version>1.1.1</hive11.version>
<metrics.version>3.1.1</metrics.version> <metrics.version>3.1.1</metrics.version>
<spark.version>2.1.0</spark.version> <spark.version>2.1.0</spark.version>
<avro.version>1.7.7</avro.version> <avro.version>1.7.7</avro.version>
@@ -589,6 +587,11 @@
<artifactId>commons-dbcp</artifactId> <artifactId>commons-dbcp</artifactId>
<version>1.4</version> <version>1.4</version>
</dependency> </dependency>
<dependency>
<groupId>commons-pool</groupId>
<artifactId>commons-pool</artifactId>
<version>1.4</version>
</dependency>
<dependency> <dependency>
<groupId>org.apache.httpcomponents</groupId> <groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId> <artifactId>httpcore</artifactId>
@@ -656,7 +659,48 @@
<artifactId>jackson-mapper-asl</artifactId> <artifactId>jackson-mapper-asl</artifactId>
<version>1.9.13</version> <version>1.9.13</version>
</dependency> </dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-shims</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-serde</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId> <artifactId>hadoop-hdfs</artifactId>
@@ -708,109 +752,6 @@
</distributionManagement> </distributionManagement>
<profiles> <profiles>
<profile>
<id>hive12</id>
<activation>
<property>
<name>!hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-service</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-shims</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-serde</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-common</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${hive12.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive12.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>
</profile>
<profile>
<id>hive11</id>
<activation>
<property>
<name>hive11</name>
</property>
</activation>
<dependencies>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-service</artifactId>
<version>${hive11.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-shims</artifactId>
<version>${hive11.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive11.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-serde</artifactId>
<version>${hive11.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive11.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
<version>${hive11.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive11.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>
</profile>
<profile> <profile>
<id>release</id> <id>release</id>
<activation> <activation>