From 3dcaacfb679a086d248f0f1f0bc504bb2b0e5667 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Sun, 10 Dec 2017 07:50:37 -0800 Subject: [PATCH 001/374] Update README.md Reflect hudi --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 7d298971a1f06..be2a80a6164a0 100644 --- a/README.md +++ b/README.md @@ -1,7 +1,7 @@ -# Hoodie -Hoodie manages storage of large analytical datasets on [HDFS](http://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) and serve them out via two types of tables +# Hudi +Hudi (pronounced Hoodie) stands for `Hadoop Upserts anD Incrementals`. Hudi manages storage of large analytical datasets on [HDFS](http://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) and serve them out via two types of tables * **Read Optimized Table** - Provides excellent query performance via purely columnar storage (e.g. [Parquet](https://parquet.apache.org/)) * **Near-Real time Table (WIP)** - Provides queries on real-time data, using a combination of columnar & row based storage (e.g Parquet + [Avro](http://avro.apache.org/docs/current/mr.html)) -For more, head over [here](https://uber.github.io/hoodie) +For more, head over [here](https://uber.github.io/hudi) From c9adb71b2561760cf966f3a8950b7831e4331689 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 5 Dec 2017 00:58:53 -0800 Subject: [PATCH 002/374] Removing compaction action type and associated compaction timeline operations, replace with commit action type --- .../cli/commands/ArchivedCommitsCommand.java | 6 - .../hoodie/cli/commands/CommitsCommand.java | 12 +- .../cli/commands/HoodieSyncCommand.java | 4 +- .../com/uber/hoodie/HoodieReadClient.java | 2 +- .../com/uber/hoodie/HoodieWriteClient.java | 32 +++--- .../hoodie/index/bloom/HoodieBloomIndex.java | 2 +- .../hoodie/io/HoodieCommitArchiveLog.java | 32 ++---- .../hoodie/io/compact/HoodieCompactor.java | 13 +-- .../compact/HoodieRealtimeTableCompactor.java | 61 +++++----- .../hoodie/table/HoodieCopyOnWriteTable.java | 5 +- .../hoodie/table/HoodieMergeOnReadTable.java | 13 +-- .../com/uber/hoodie/table/HoodieTable.java | 56 +++------ .../TestHoodieClientOnCopyOnWriteStorage.java | 6 +- .../hoodie/io/TestHoodieCommitArchiveLog.java | 14 +-- .../uber/hoodie/io/TestHoodieCompactor.java | 9 +- .../hoodie/table/TestMergeOnReadTable.java | 14 +-- .../src/main/avro/HoodieCommitMetadata.avsc | 16 +++ .../common/model/CompactionWriteStat.java | 108 ------------------ .../common/model/HoodieCommitMetadata.java | 32 ++++-- .../model/HoodieCompactionMetadata.java | 86 -------------- .../hoodie/common/model/HoodieWriteStat.java | 61 ++++++++++ .../hoodie/common/table/HoodieTimeline.java | 11 -- .../table/timeline/HoodieActiveTimeline.java | 21 +--- .../common/table/timeline/HoodieInstant.java | 4 - .../hoodie/common/model/HoodieTestUtils.java | 2 +- .../uber/hoodie/hadoop/HoodieInputFormat.java | 2 +- .../realtime/HoodieRealtimeInputFormat.java | 1 - .../uber/hoodie/hive/HoodieHiveClient.java | 30 ++--- .../java/com/uber/hoodie/hive/TestUtil.java | 39 +++---- .../uber/hoodie/HoodieDataSourceHelpers.java | 7 +- .../com/uber/hoodie/IncrementalRelation.scala | 2 +- .../utilities/HiveIncrementalPuller.java | 6 +- .../utilities/HoodieSnapshotCopier.java | 4 +- .../deltastreamer/HoodieDeltaStreamer.java | 2 +- 34 files changed, 265 insertions(+), 450 deletions(-) delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionWriteStat.java delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCompactionMetadata.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 069c6564acb50..e1b03b271406b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -94,12 +94,6 @@ private String[] readCommit(GenericRecord record) { commitDetails.add(record.get("hoodieCommitMetadata").toString()); break; } - case HoodieTimeline.COMPACTION_ACTION: { - commitDetails.add(record.get("commitTime").toString()); - commitDetails.add(record.get("actionType").toString()); - commitDetails.add(record.get("hoodieCompactionMetadata").toString()); - break; - } case HoodieTimeline.DELTA_COMMIT_ACTION: { commitDetails.add(record.get("commitTime").toString()); commitDetails.add(record.get("actionType").toString()); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index c1a9e6dd97674..d6446a2c4dd55 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -69,7 +69,7 @@ public String showCommits( "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) throws IOException { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline() + HoodieTimeline timeline = activeTimeline.getCommitsTimeline() .filterCompletedInstants(); List commits = timeline.getInstants().collect(Collectors.toList()); String[][] rows = new String[commits.size()][]; @@ -108,7 +108,7 @@ public String rollbackCommit( @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath) throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline() + HoodieTimeline timeline = activeTimeline.getCommitsTimeline() .filterCompletedInstants(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); @@ -137,7 +137,7 @@ public String showCommitPartitions( @CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline() + HoodieTimeline timeline = activeTimeline.getCommitsTimeline() .filterCompletedInstants(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); @@ -187,7 +187,7 @@ public String showCommitFiles( @CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline() + HoodieTimeline timeline = activeTimeline.getCommitsTimeline() .filterCompletedInstants(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); @@ -225,11 +225,11 @@ public String compareCommits( @CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) throws Exception { HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path); - HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline() + HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); ; HoodieTableMetaClient source = HoodieCLI.tableMetadata; - HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline() + HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); ; String targetLatestCommit = diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java index b19608bed4323..dd9560a4a0f8e 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java @@ -61,9 +61,9 @@ public String validateSync( "hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") final String hivePass) throws Exception { HoodieTableMetaClient target = HoodieCLI.syncTableMetadata; - HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline(); + HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline(); HoodieTableMetaClient source = HoodieCLI.tableMetadata; - HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline(); + HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline(); long sourceCount = 0; long targetCount = 0; if ("complete".equals(mode)) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 851947286d60a..6c329c330bef9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -71,7 +71,7 @@ public HoodieReadClient(JavaSparkContext jsc, String basePath) { // Create a Hoodie table which encapsulated the commits and files visible this.hoodieTable = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); - this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline(); + this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); this.index = new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc); this.sqlContextOpt = Optional.absent(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index aa2e3f4793d27..0de261c8f706a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -25,7 +25,6 @@ import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.HoodieCommitMetadata; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -54,17 +53,6 @@ import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner; import com.uber.hoodie.table.WorkloadProfile; import com.uber.hoodie.table.WorkloadStat; -import java.io.IOException; -import java.io.Serializable; -import java.nio.charset.StandardCharsets; -import java.text.ParseException; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -77,6 +65,18 @@ import scala.Option; import scala.Tuple2; +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.text.ParseException; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + /** * Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient * mutations on a HDFS dataset [upsert()] @@ -605,7 +605,7 @@ public boolean rollbackToSavepoint(String savepointTime) { HoodieTable table = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline commitTimeline = table.getCommitTimeline(); + HoodieTimeline commitTimeline = table.getCommitsTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); @@ -624,7 +624,7 @@ public boolean rollbackToSavepoint(String savepointTime) { // Make sure the rollback was successful Optional lastInstant = - activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants() + activeTimeline.reload().getCommitsTimeline().filterCompletedInstants() .lastInstant(); Preconditions.checkArgument(lastInstant.isPresent()); Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), @@ -829,7 +829,7 @@ private void compact(String compactionCommitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); - Optional compactionMetadata = table.compact(jsc, compactionCommitTime); + Optional compactionMetadata = table.compact(jsc, compactionCommitTime); if (compactionMetadata.isPresent()) { logger.info("Compacted successfully on commit " + compactionCommitTime); } else { @@ -878,7 +878,7 @@ private JavaRDD> deduplicateRecords(JavaRDD> rec private void rollbackInflightCommits() { HoodieTable table = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); - HoodieTimeline inflightTimeline = table.getCommitTimeline().filterInflights(); + HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); Collections.reverse(commits); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 44dc910c13a69..5a0d69002c552 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -239,7 +239,7 @@ List> loadInvolvedFiles(List partitio .parallelize(partitions, Math.max(partitions.size(), 1)) .flatMapToPair(partitionPath -> { java.util.Optional latestCommitTime = - hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant(); + hoodieTable.getCommitsTimeline().filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { filteredFiles = diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index a707590c46cac..3cb697c7a1078 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -26,7 +26,6 @@ import com.uber.hoodie.common.model.ActionType; import com.uber.hoodie.common.model.HoodieArchivedLogFile; import com.uber.hoodie.common.model.HoodieCommitMetadata; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; @@ -39,12 +38,6 @@ import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; @@ -52,6 +45,13 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + /** * Archiver to bound the growth of .commit files */ @@ -228,14 +228,6 @@ private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, archivedMetaWrapper.setActionType(ActionType.commit.name()); break; } - case HoodieTimeline.COMPACTION_ACTION: { - com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata - .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get()); - archivedMetaWrapper - .setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata)); - archivedMetaWrapper.setActionType(ActionType.compaction.name()); - break; - } case HoodieTimeline.ROLLBACK_ACTION: { archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils .deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), @@ -271,14 +263,4 @@ private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter( com.uber.hoodie.avro.model.HoodieCommitMetadata.class); return avroMetaData; } - - private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter( - HoodieCompactionMetadata hoodieCompactionMetadata) { - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper - .convertValue(hoodieCompactionMetadata, - com.uber.hoodie.avro.model.HoodieCompactionMetadata.class); - return avroMetaData; - } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index d5bcd9ee6a91f..fe8227f2d6864 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -16,17 +16,16 @@ package com.uber.hoodie.io.compact; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; +import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; +import org.apache.spark.api.java.JavaSparkContext; + import java.io.Serializable; import java.util.Date; -import java.util.Optional; - -import org.apache.spark.api.java.JavaSparkContext; /** * A HoodieCompactor runs compaction on a hoodie table @@ -36,8 +35,8 @@ public interface HoodieCompactor extends Serializable { /** * Compact the delta files with the data files */ - HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws Exception; + HoodieCommitMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config, + HoodieTable hoodieTable, String compactionCommitTime) throws Exception; // Helper methods @@ -45,7 +44,7 @@ default String startCompactionCommit(HoodieTable hoodieTable) { String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline(); activeTimeline - .createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime)); + .createInflight(new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime)); return commitTime; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index e4da087c15f74..92107c0ccfc9a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -16,15 +16,13 @@ package com.uber.hoodie.io.compact; -import static java.util.stream.Collectors.toList; - import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; -import com.uber.hoodie.common.model.CompactionWriteStat; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; +import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; @@ -36,6 +34,13 @@ import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Collection; @@ -44,12 +49,8 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FlatMapFunction; + +import static java.util.stream.Collectors.toList; /** * HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all @@ -63,8 +64,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { private static Logger log = LogManager.getLogger(HoodieRealtimeTableCompactor.class); @Override - public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws IOException { + public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config, + HoodieTable hoodieTable, String compactionCommitTime) throws IOException { Preconditions.checkArgument( hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, "HoodieRealtimeTableCompactor can only compact table of type " @@ -99,20 +100,20 @@ public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig } log.info("After filtering, Compacting " + operations + " files"); - List updateStatusMap = + List updateStatusMap = jsc.parallelize(operations, operations.size()) .map(s -> executeCompaction(metaClient, config, s, compactionCommitTime)) - .flatMap(new FlatMapFunction, CompactionWriteStat>() { + .flatMap(new FlatMapFunction, HoodieWriteStat>() { @Override - public Iterator call( - List compactionWriteStats) + public Iterator call( + List hoodieWriteStats) throws Exception { - return compactionWriteStats.iterator(); + return hoodieWriteStats.iterator(); } }).collect(); - HoodieCompactionMetadata metadata = new HoodieCompactionMetadata(); - for (CompactionWriteStat stat : updateStatusMap) { + HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); + for (HoodieWriteStat stat : updateStatusMap) { metadata.addWriteStat(stat.getPartitionPath(), stat); } @@ -128,13 +129,13 @@ public Iterator call( return metadata; } - private boolean isCompactionSucceeded(HoodieCompactionMetadata result) { + private boolean isCompactionSucceeded(HoodieCommitMetadata result) { //TODO figure out a success factor for a compaction return true; } - private List executeCompaction(HoodieTableMetaClient metaClient, - HoodieWriteConfig config, CompactionOperation operation, String commitTime) + private List executeCompaction(HoodieTableMetaClient metaClient, + HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException { FileSystem fs = FSUtils.getFs(); Schema readerSchema = @@ -150,7 +151,6 @@ private List executeCompaction(HoodieTableMetaClient metaCl String maxInstantTime = metaClient.getActiveTimeline() .getTimelineOfActions( Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, - HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) .filterCompletedInstants().lastInstant().get().getTimestamp(); @@ -170,22 +170,23 @@ private List executeCompaction(HoodieTableMetaClient metaCl return StreamSupport.stream(resultIterable.spliterator(), false) .flatMap(Collection::stream) .map(WriteStatus::getStat) - .map(s -> CompactionWriteStat.newBuilder().withHoodieWriteStat(s) - .setTotalRecordsToUpdate(scanner.getTotalRecordsToUpdate()) - .setTotalLogFiles(scanner.getTotalLogFiles()) - .setTotalLogRecords(scanner.getTotalLogRecords()) - .onPartition(operation.getPartitionPath()).build()) + .map(s -> { + s.setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate()); + s.setTotalLogFiles(scanner.getTotalLogFiles()); + s.setTotalLogRecords(scanner.getTotalLogRecords()); + s.setPartitionPath(operation.getPartitionPath()); + return s;}) .collect(toList()); } public boolean commitCompaction(String commitTime, HoodieTableMetaClient metaClient, - HoodieCompactionMetadata metadata) { + HoodieCommitMetadata metadata) { log.info("Committing Compaction " + commitTime); HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); try { activeTimeline.saveAsComplete( - new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime), + new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime), Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } catch (IOException e) { throw new HoodieCompactionException( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 543b4c26d9157..a509411d771cb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -22,7 +22,6 @@ import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.HoodieCommitMetadata; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -486,7 +485,7 @@ public Iterator> handleInsertPartition(String commitTime, Inte } @Override - public Optional compact(JavaSparkContext jsc, String commitCompactionTime) { + public Optional compact(JavaSparkContext jsc, String commitCompactionTime) { logger.info("Nothing to compact in COW storage format"); return Optional.empty(); } @@ -544,7 +543,7 @@ protected Map deleteCleanedFiles(String partitionPath, List @Override public List rollback(JavaSparkContext jsc, List commits) throws IOException { - String actionType = this.getCompactedCommitActionType(); + String actionType = this.getCommitActionType(); HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); List inflights = this.getInflightCommitTimeline().getInstants() .map(HoodieInstant::getTimestamp) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 3be0479c7130c..19461b025368d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -21,7 +21,6 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.HoodieCommitMetadata; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -93,9 +92,9 @@ public Iterator> handleUpdate(String commitTime, String fileId } @Override - public Optional compact(JavaSparkContext jsc, String compactionCommitTime) { + public Optional compact(JavaSparkContext jsc, String compactionCommitTime) { logger.info("Checking if compaction needs to be run on " + config.getBasePath()); - Optional lastCompaction = getActiveTimeline().getCompactionTimeline() + Optional lastCompaction = getActiveTimeline().getCommitTimeline() .filterCompletedInstants().lastInstant(); String deltaCommitsSinceTs = "0"; if (lastCompaction.isPresent()) { @@ -130,8 +129,7 @@ public List rollback(JavaSparkContext jsc, List comm } Map commitsAndCompactions = this.getActiveTimeline() - .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION)) + .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION)) .getInstants() .filter(i -> commits.contains(i.getTimestamp())) .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); @@ -149,11 +147,10 @@ public List rollback(JavaSparkContext jsc, List comm List stats = null; switch (instant.getAction()) { case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.COMPACTION_ACTION: try { logger.info("Starting to rollback Commit/Compaction " + instant); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(this.getCommitTimeline().getInstantDetails( + .fromBytes(this.getCommitsTimeline().getInstantDetails( new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream() @@ -174,7 +171,7 @@ public List rollback(JavaSparkContext jsc, List comm logger.info("Starting to rollback delta commit " + instant); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(this.getCommitTimeline().getInstantDetails( + .fromBytes(this.getCommitsTimeline().getInstantDetails( new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream() diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index b434c7509daec..76ab92d99ca21 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -16,12 +16,11 @@ package com.uber.hoodie.table; -import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.avro.model.HoodieSavepointMetadata; import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieRollbackStat; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; +import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -35,6 +34,12 @@ import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieSavepointException; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaSparkContext; + import java.io.IOException; import java.io.Serializable; import java.util.Iterator; @@ -42,11 +47,6 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaSparkContext; /** * Abstract implementation of a HoodieTable @@ -116,21 +116,21 @@ public TableFileSystemView.RealtimeView getRTFileSystemView() { * Get the completed (commit + compaction) view of the file system for this table */ public TableFileSystemView getCompletedFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCommitTimeline()); + return new HoodieTableFileSystemView(metaClient, getCommitsTimeline()); } /** * Get only the completed (no-inflights) commit timeline */ public HoodieTimeline getCompletedCommitTimeline() { - return getCommitTimeline().filterCompletedInstants(); + return getCommitsTimeline().filterCompletedInstants(); } /** * Get only the inflights (no-completed) commit timeline */ public HoodieTimeline getInflightCommitTimeline() { - return getCommitTimeline().filterInflights(); + return getCommitsTimeline().filterInflights(); } @@ -185,38 +185,28 @@ public HoodieActiveTimeline getActiveTimeline() { /** * Get the commit timeline visible for this table */ - public HoodieTimeline getCommitTimeline() { + public HoodieTimeline getCommitsTimeline() { switch (metaClient.getTableType()) { case COPY_ON_WRITE: return getActiveTimeline().getCommitTimeline(); case MERGE_ON_READ: // We need to include the parquet files written out in delta commits // Include commit action to be able to start doing a MOR over a COW dataset - no migration required - return getActiveTimeline().getCommitsAndCompactionsTimeline(); + return getActiveTimeline().getCommitsTimeline(); default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); } } - /** - * Get only the completed (no-inflights) compaction commit timeline - */ - public HoodieTimeline getCompletedCompactionCommitTimeline() { - return getCompactionCommitTimeline().filterCompletedInstants(); - } - - /** * Get the compacted commit timeline visible for this table */ - public HoodieTimeline getCompactionCommitTimeline() { + public HoodieTimeline getCommitTimeline() { switch (metaClient.getTableType()) { case COPY_ON_WRITE: - return getActiveTimeline().getCommitsAndCompactionsTimeline(); case MERGE_ON_READ: // We need to include the parquet files written out in delta commits in tagging - return getActiveTimeline().getTimelineOfActions( - Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION)); + return getActiveTimeline().getCommitTimeline(); default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); } @@ -236,20 +226,6 @@ public String getCommitActionType() { "Could not commit on unknown storage type " + metaClient.getTableType()); } - /** - * Gets the action type for a compaction commit - */ - public String getCompactedCommitActionType() { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - return HoodieTimeline.COMMIT_ACTION; - case MERGE_ON_READ: - return HoodieTimeline.COMPACTION_ACTION; - } - throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); - } - - /** * Perform the ultimate IO for a given upserted (RDD) partition */ @@ -279,8 +255,8 @@ public static HoodieTable getHoodieTable( * Run Compaction on the table. Compaction arranges the data so that it is optimized for data * access */ - public abstract Optional compact(JavaSparkContext jsc, - String commitCompactionTime); + public abstract Optional compact(JavaSparkContext jsc, + String commitCompactionTime); /** * Clean partition paths according to cleaning policy and returns the number of files cleaned. diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index ccdd12839c8a9..f6995762d3e05 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -649,7 +649,7 @@ public void testInsertAndCleanByVersions() throws Exception { HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); table = HoodieTable.getHoodieTable(metadata, getConfig()); - timeline = table.getCommitTimeline(); + timeline = table.getCommitsTimeline(); TableFileSystemView fsView = table.getFileSystemView(); // Need to ensure the following @@ -1493,10 +1493,10 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieTestUtils.doesCommitExist(basePath, commitTime)); // Get parquet file paths from commit metadata - String actionType = table.getCompactedCommitActionType(); + String actionType = table.getCommitActionType(); HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); - HoodieTimeline commitTimeline = table.getCompletedCompactionCommitTimeline(); + HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); String basePath = table.getMetaClient().getBasePath(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index fb19bf7e8ad6f..5106e8eea716c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -84,7 +84,7 @@ public void testArchiveDatasetWithArchival() throws IOException { HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants(); + metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); @@ -155,13 +155,13 @@ public void testArchiveDatasetWithNoArchival() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "103"); HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants(); + metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); timeline = - metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline() + metadata.getActiveTimeline().reload().getCommitsTimeline() .filterCompletedInstants(); assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); @@ -183,12 +183,12 @@ public void testArchiveCommitSafety() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "105"); HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants(); + metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); timeline = - metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline() + metadata.getActiveTimeline().reload().getCommitsTimeline() .filterCompletedInstants(); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); @@ -217,12 +217,12 @@ public void testArchiveCommitSavepointNoHole() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "105"); HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants(); + metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); timeline = - metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline() + metadata.getActiveTimeline().reload().getCommitsTimeline() .filterCompletedInstants(); assertEquals( "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)", diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index e8ea130556437..c842bf5922cd9 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -21,7 +21,7 @@ import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.FileSlice; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; +import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; @@ -49,7 +49,6 @@ import java.io.File; import java.io.IOException; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; @@ -124,7 +123,7 @@ public void testCompactionEmpty() throws Exception { JavaRDD recordsRDD = jsc.parallelize(records, 1); writeClient.insert(recordsRDD, newCommitTime).collect(); - HoodieCompactionMetadata result = + HoodieCommitMetadata result = compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); String basePath = table.getMetaClient().getBasePath(); assertTrue("If there is nothing to compact, result will be empty", @@ -178,7 +177,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { metaClient = new HoodieTableMetaClient(fs, basePath); table = HoodieTable.getHoodieTable(metaClient, config); - HoodieCompactionMetadata result = + HoodieCommitMetadata result = compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); // Verify that recently written compacted data file has no log file @@ -199,7 +198,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { "After compaction there should be no log files visiable on a Realtime view", slice.getLogFiles().collect(Collectors.toList()).isEmpty()); } - assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath)); + assertTrue(result.getPartitionToWriteStats().containsKey(partitionPath)); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 337d06610d19f..42649159c86f3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -182,7 +182,7 @@ public void testSimpleInsertAndUpdate() throws Exception { FileStatus[] allFiles = HoodieTestUtils .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCompletedCompactionCommitTimeline(), allFiles); + hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -231,7 +231,7 @@ public void testSimpleInsertAndUpdate() throws Exception { // verify that there is a commit table = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false)); - HoodieTimeline timeline = table.getCompletedCompactionCommitTimeline(); + HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); @@ -299,7 +299,7 @@ public void testSimpleInsertAndDelete() throws Exception { FileStatus[] allFiles = HoodieTestUtils .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCompletedCompactionCommitTimeline(), allFiles); + hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -455,7 +455,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { FileStatus[] allFiles = HoodieTestUtils .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCompletedCompactionCommitTimeline(), allFiles); + hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -524,11 +524,11 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompactionCommitTimeline(), + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); final String compactedCommitTime = metaClient.getActiveTimeline().reload() - .getCommitsAndCompactionsTimeline().lastInstant().get().getTimestamp(); + .getCommitsTimeline().lastInstant().get().getTimestamp(); assertTrue(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -543,7 +543,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompactionCommitTimeline(), + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); assertFalse(roView.getLatestDataFiles().filter(file -> { diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index b8eced141fee3..ae3df4ffc524d 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -44,6 +44,22 @@ { "name":"totalWriteErrors", "type":["null","long"] + }, + { + "name":"partitionPath", + "type":["null","string"] + }, + { + "name":"totalLogRecords", + "type":["null","long"] + }, + { + "name":"totalLogFiles", + "type":["null","long"] + }, + { + "name":"totalRecordsToBeUpdate", + "type":["null","long"] } ] } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionWriteStat.java deleted file mode 100644 index 40f7fc3631710..0000000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionWriteStat.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.model; - -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import java.io.Serializable; - -@JsonIgnoreProperties(ignoreUnknown = true) -public class CompactionWriteStat implements Serializable { - - private HoodieWriteStat writeStat; - private String partitionPath; - private long totalLogRecords; - private long totalLogFiles; - private long totalRecordsToBeUpdate; - - public CompactionWriteStat(HoodieWriteStat writeStat, String partitionPath, long totalLogFiles, - long totalLogRecords, - long totalRecordsToUpdate) { - this.writeStat = writeStat; - this.partitionPath = partitionPath; - this.totalLogFiles = totalLogFiles; - this.totalLogRecords = totalLogRecords; - this.totalRecordsToBeUpdate = totalRecordsToUpdate; - } - - public CompactionWriteStat() { - // For de-serialization - } - - public long getTotalLogRecords() { - return totalLogRecords; - } - - public long getTotalLogFiles() { - return totalLogFiles; - } - - public long getTotalRecordsToBeUpdate() { - return totalRecordsToBeUpdate; - } - - public HoodieWriteStat getHoodieWriteStat() { - return writeStat; - } - - public String getPartitionPath() { - return partitionPath; - } - - public static Builder newBuilder() { - return new Builder(); - } - - public static class Builder { - - private HoodieWriteStat writeStat; - private long totalLogRecords; - private long totalRecordsToUpdate; - private long totalLogFiles; - private String partitionPath; - - - public Builder withHoodieWriteStat(HoodieWriteStat writeStat) { - this.writeStat = writeStat; - return this; - } - - public Builder setTotalLogRecords(long records) { - this.totalLogRecords = records; - return this; - } - - public Builder setTotalLogFiles(long totalLogFiles) { - this.totalLogFiles = totalLogFiles; - return this; - } - - public Builder setTotalRecordsToUpdate(long records) { - this.totalRecordsToUpdate = records; - return this; - } - - public Builder onPartition(String path) { - this.partitionPath = path; - return this; - } - - public CompactionWriteStat build() { - return new CompactionWriteStat(writeStat, partitionPath, totalLogFiles, totalLogRecords, - totalRecordsToUpdate); - } - } -} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index 47253637b97f4..8a49c5c39cfc4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -40,12 +40,19 @@ public class HoodieCommitMetadata implements Serializable { private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class); protected Map> partitionToWriteStats; + protected Boolean compacted; private Map extraMetadataMap; + // for ser/deser public HoodieCommitMetadata() { + this(false); + } + + public HoodieCommitMetadata(boolean compacted) { extraMetadataMap = new HashMap<>(); partitionToWriteStats = new HashMap<>(); + this.compacted = compacted; } public void addWriteStat(String partitionPath, HoodieWriteStat stat) { @@ -75,6 +82,14 @@ public String getMetadata(String metaKey) { return extraMetadataMap.get(metaKey); } + public Boolean getCompacted() { + return compacted; + } + + public void setCompacted(Boolean compacted) { + this.compacted = compacted; + } + public HashMap getFileIdAndRelativePaths() { HashMap filePaths = new HashMap<>(); // list all partitions paths @@ -200,24 +215,21 @@ public long fetchTotalWriteErrors() { @Override public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; HoodieCommitMetadata that = (HoodieCommitMetadata) o; - return partitionToWriteStats != null ? - partitionToWriteStats.equals(that.partitionToWriteStats) : - that.partitionToWriteStats == null; + if (!partitionToWriteStats.equals(that.partitionToWriteStats)) return false; + return compacted.equals(that.compacted); } @Override public int hashCode() { - return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0; + int result = partitionToWriteStats.hashCode(); + result = 31 * result + compacted.hashCode(); + return result; } public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCompactionMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCompactionMetadata.java deleted file mode 100644 index 043098f36ddea..0000000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCompactionMetadata.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.model; - -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.codehaus.jackson.annotate.JsonAutoDetect; -import org.codehaus.jackson.annotate.JsonMethod; -import org.codehaus.jackson.map.DeserializationConfig.Feature; -import org.codehaus.jackson.map.ObjectMapper; - -/** - * Place holder for the compaction specific meta-data, uses all the details used in a normal - * HoodieCommitMetadata - */ -public class HoodieCompactionMetadata extends HoodieCommitMetadata { - - private static volatile Logger log = LogManager.getLogger(HoodieCompactionMetadata.class); - protected HashMap> partitionToCompactionWriteStats; - - public HoodieCompactionMetadata() { - partitionToCompactionWriteStats = new HashMap<>(); - } - - public void addWriteStat(String partitionPath, CompactionWriteStat stat) { - addWriteStat(partitionPath, stat.getHoodieWriteStat()); - if (!partitionToCompactionWriteStats.containsKey(partitionPath)) { - partitionToCompactionWriteStats.put(partitionPath, new ArrayList<>()); - } - partitionToCompactionWriteStats.get(partitionPath).add(stat); - } - - public List getCompactionWriteStats(String partitionPath) { - return partitionToCompactionWriteStats.get(partitionPath); - } - - public Map> getPartitionToCompactionWriteStats() { - return partitionToCompactionWriteStats; - } - - public String toJsonString() throws IOException { - if (partitionToCompactionWriteStats.containsKey(null)) { - log.info("partition path is null for " + partitionToCompactionWriteStats.get(null)); - partitionToCompactionWriteStats.remove(null); - } - ObjectMapper mapper = new ObjectMapper(); - mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); - return mapper.defaultPrettyPrintingWriter().writeValueAsString(this); - } - - public static HoodieCompactionMetadata fromJsonString(String jsonStr) throws IOException { - if (jsonStr == null || jsonStr.isEmpty()) { - // For empty commit file (no data or somethings bad happen). - return new HoodieCompactionMetadata(); - } - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); - return mapper.readValue(jsonStr, HoodieCompactionMetadata.class); - } - - public static HoodieCompactionMetadata fromBytes(byte[] bytes) throws IOException { - return fromJsonString(new String(bytes, Charset.forName("utf-8"))); - } - -} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index b69aed36cd8b3..f1a58f740187c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -17,6 +17,8 @@ package com.uber.hoodie.common.model; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import javax.annotation.Nullable; import java.io.Serializable; /** @@ -68,6 +70,34 @@ public class HoodieWriteStat implements Serializable { */ private long totalWriteErrors; + /** + * Following properties are associated only with the result of a Compaction Operation + */ + + /** + * Partition Path associated with this writeStat + */ + @Nullable + private String partitionPath; + + /** + * Total number of log records that were compacted by a compaction operation + */ + @Nullable + private Long totalLogRecords; + + /** + * Total number of log files that were compacted by a compaction operation + */ + @Nullable + private Long totalLogFiles; + + /** + * Total number of records updated by a compaction operation + */ + @Nullable + private Long totalRecordsToBeUpdate; + public HoodieWriteStat() { // called by jackson json lib } @@ -136,6 +166,37 @@ public String getPath() { return path; } + public String getPartitionPath() { + return partitionPath; + } + + public void setPartitionPath(String partitionPath) { + this.partitionPath = partitionPath; + } + + public Long getTotalLogRecords() { + return totalLogRecords; + } + + public void setTotalLogRecords(Long totalLogRecords) { + this.totalLogRecords = totalLogRecords; + } + + public Long getTotalLogFiles() { + return totalLogFiles; + } + + public void setTotalLogFiles(Long totalLogFiles) { + this.totalLogFiles = totalLogFiles; + } + + public Long getTotalRecordsToBeUpdate() { + return totalRecordsToBeUpdate; + } + + public void setTotalRecordsToBeUpdate(Long totalRecordsToBeUpdate) { + this.totalRecordsToBeUpdate = totalRecordsToBeUpdate; + } @Override public String toString() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java index e2001a2c73ae5..867fa5a10e264 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java @@ -41,7 +41,6 @@ public interface HoodieTimeline extends Serializable { String CLEAN_ACTION = "clean"; String ROLLBACK_ACTION = "rollback"; String SAVEPOINT_ACTION = "savepoint"; - String COMPACTION_ACTION = "compaction"; String INFLIGHT_EXTENSION = ".inflight"; String COMMIT_EXTENSION = "." + COMMIT_ACTION; @@ -49,14 +48,12 @@ public interface HoodieTimeline extends Serializable { String CLEAN_EXTENSION = "." + CLEAN_ACTION; String ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION; String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION; - String COMPACTION_EXTENSION = "." + COMPACTION_ACTION; //this is to preserve backwards compatibility on commit in-flight filenames String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION; String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION; String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION; String INFLIGHT_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + INFLIGHT_EXTENSION; String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION; - String INFLIGHT_COMPACTION_EXTENSION = "." + COMPACTION_ACTION + INFLIGHT_EXTENSION; /** * Filter this timeline to just include the in-flights @@ -197,14 +194,6 @@ static String makeSavePointFileName(String commitTime) { return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION; } - static String makeInflightCompactionFileName(String commitTime) { - return commitTime + HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION; - } - - static String makeCompactionFileName(String commitTime) { - return commitTime + HoodieTimeline.COMPACTION_EXTENSION; - } - static String makeInflightDeltaFileName(String commitTime) { return commitTime + HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index 6848d4a21db73..ae004991c1c7d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -94,8 +94,7 @@ protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] included public HoodieActiveTimeline(FileSystem fs, String metaPath) { this(fs, metaPath, new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, - INFLIGHT_DELTA_COMMIT_EXTENSION, COMPACTION_EXTENSION, - INFLIGHT_COMPACTION_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, + INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); } @@ -119,21 +118,21 @@ private void readObject(java.io.ObjectInputStream in) } /** - * Get all instants (commits, delta commits, compactions) that produce new data, in the active + * Get all instants (commits, delta commits) that produce new data, in the active * timeline * */ - public HoodieTimeline getCommitsAndCompactionsTimeline() { + public HoodieTimeline getCommitsTimeline() { return getTimelineOfActions( - Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)); + Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION)); } /** - * Get all instants (commits, delta commits, compactions, clean, savepoint, rollback) that result + * Get all instants (commits, delta commits, clean, savepoint, rollback) that result * in actions, in the active timeline * */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions( - Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, + Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION)); } @@ -152,14 +151,6 @@ public HoodieTimeline getDeltaCommitTimeline() { (Function> & Serializable) this::getInstantDetails); } - /** - * Get only the commits (inflight and completed) in the compaction timeline - */ - public HoodieTimeline getCompactionTimeline() { - return new HoodieDefaultTimeline(filterInstantsByAction(COMPACTION_ACTION), - (Function> & Serializable) this::getInstantDetails); - } - /** * Get a timeline of a specific set of actions. useful to create a merged timeline of multiple * actions diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java index bf27b7db2c22c..1891c9807b2d1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java @@ -94,10 +94,6 @@ public String getFileName() { return isInflight ? HoodieTimeline.makeInflightSavePointFileName(timestamp) : HoodieTimeline.makeSavePointFileName(timestamp); - } else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightCompactionFileName(timestamp) : - HoodieTimeline.makeCompactionFileName(timestamp); } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { return isInflight ? HoodieTimeline.makeInflightDeltaFileName(timestamp) : diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 1ccca51b26228..9489e57ab3664 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -158,7 +158,7 @@ public static final void createCompactionCommitFiles(String basePath, String... for (String commitTime : commitTimes) { boolean createFile = fs.createNewFile(new Path( basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCompactionFileName(commitTime))); + .makeCommitFileName(commitTime))); if (!createFile) { throw new IOException("cannot create commit file for commit " + commitTime); } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index 03e86d3bd33b3..ee4f5a9544286 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -100,7 +100,7 @@ public FileStatus[] listStatus(JobConf job) throws IOException { String tableName = metadata.getTableConfig().getTableName(); String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName); // Get all commits, delta commits, compactions, as all of them produce a base parquet file today - HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline() + HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata, timeline, statuses); diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index c92f0e59332e9..6add30565753f 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -125,7 +125,6 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { String maxCommitTime = metaClient.getActiveTimeline() .getTimelineOfActions( Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, - HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) .filterCompletedInstants().lastInstant().get().getTimestamp(); rtSplits.add( diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 00d8ac5c9fd0c..dede5e5f5d2d8 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -21,7 +21,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieCommitMetadata; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -35,17 +34,6 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidDatasetException; import com.uber.hoodie.hive.util.SchemaUtil; -import java.io.IOException; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; import org.apache.commons.dbcp.BasicDataSource; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -63,6 +51,18 @@ import parquet.hadoop.metadata.ParquetMetadata; import parquet.schema.MessageType; +import java.io.IOException; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + @SuppressWarnings("ConstantConditions") public class HoodieHiveClient { @@ -111,7 +111,7 @@ public class HoodieHiveClient { e); } - activeTimeline = metaClient.getActiveTimeline().getCommitsAndCompactionsTimeline() + activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); } @@ -323,7 +323,7 @@ public MessageType getDataSchema() { // If this is MOR, depending on whether the latest commit is a delta commit or compaction commit // Get a datafile written and get the schema from that file Optional lastCompactionCommit = metaClient.getActiveTimeline() - .getCompactionTimeline().filterCompletedInstants().lastInstant(); + .getCommitTimeline().filterCompletedInstants().lastInstant(); LOG.info("Found the last compaction commit as " + lastCompactionCommit); Optional lastDeltaCommit; @@ -379,7 +379,7 @@ private MessageType readSchemaFromLastCompaction(Optional lastCom + syncConfig.basePath)); // Read from the compacted file wrote - HoodieCompactionMetadata compactionMetadata = HoodieCompactionMetadata + HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata .fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get()); String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() .stream().findAny() diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java index 2707377aff991..f2eb5e4f1ffe6 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java @@ -16,9 +16,6 @@ package com.uber.hoodie.hive; -import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID; -import static org.junit.Assert.fail; - import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -26,10 +23,8 @@ import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.minicluster.HdfsTestService; import com.uber.hoodie.common.minicluster.ZookeeperTestService; -import com.uber.hoodie.common.model.CompactionWriteStat; import com.uber.hoodie.common.model.HoodieAvroPayload; import com.uber.hoodie.common.model.HoodieCommitMetadata; -import com.uber.hoodie.common.model.HoodieCompactionMetadata; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieDeltaWriteStat; import com.uber.hoodie.common.model.HoodieLogFile; @@ -44,15 +39,6 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.hive.util.HiveTestService; -import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.commons.io.FileUtils; @@ -72,6 +58,19 @@ import org.joda.time.format.DateTimeFormatter; import org.junit.runners.model.InitializationError; +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.UUID; + +import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID; +import static org.junit.Assert.fail; + @SuppressWarnings("SameParameterValue") public class TestUtil { @@ -182,9 +181,9 @@ static void createMORDataset(String commitTime, String deltaCommitTime, int numb createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); - HoodieCompactionMetadata compactionMetadata = new HoodieCompactionMetadata(); + HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream().map(k -> new CompactionWriteStat(k, key, 0, 0, 0)) + .forEach((key, value) -> value.stream() .forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); // Write a delta commit @@ -211,9 +210,9 @@ static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimp createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); - HoodieCompactionMetadata compactionMetadata = new HoodieCompactionMetadata(); + HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream().map(k -> new CompactionWriteStat(k, key, 0, 0, 0)) + .forEach((key, value) -> value.stream() .forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), @@ -342,12 +341,12 @@ private static void createCommitFile( } private static void createCompactionCommitFile( - HoodieCompactionMetadata commitMetadata, String commitTime) + HoodieCommitMetadata commitMetadata, String commitTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCompactionFileName(commitTime)); + .makeCommitFileName(commitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java index fb8db4a94adba..b74d917071ed2 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -70,11 +70,10 @@ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, Strin .getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { return table.getActiveTimeline().getTimelineOfActions( - Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION) - ); + Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, + HoodieActiveTimeline.DELTA_COMMIT_ACTION)); } else { - return table.getCompletedCompactionCommitTimeline(); + return table.getCommitTimeline().filterCompletedInstants(); } } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index 6adab8d843d8d..4aca81f1725b7 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -53,7 +53,7 @@ class IncrementalRelation(val sqlContext: SQLContext, throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") } val hoodieTable = HoodieTable.getHoodieTable(metaClient, null) - val commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline(); + val commitTimeline = hoodieTable.getCommitTimeline.filterCompletedInstants(); if (commitTimeline.empty()) { throw new HoodieException("No instants to incrementally pull") } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index 61aec29fb5657..5bcba8cb8361b 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -294,7 +294,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath); Optional - lastCommit = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline() + lastCommit = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants().lastInstant(); if (lastCommit.isPresent()) { return lastCommit.get().getTimestamp(); @@ -332,14 +332,14 @@ private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException { HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation); - List commitsToSync = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline() + List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants() .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants() .map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (commitsToSync.isEmpty()) { log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata - .getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants() + .getActiveTimeline().getCommitsTimeline().filterCompletedInstants() .getInstants() .collect(Collectors.toList()) + " and from commit time is " + config.fromCommitTime); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index 0a74e20361e0c..3d199e0f52593 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -74,11 +74,11 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir); final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView( tableMetadata, - tableMetadata.getActiveTimeline().getCommitsAndCompactionsTimeline() + tableMetadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants()); // Get the latest commit Optional latestCommit = tableMetadata.getActiveTimeline() - .getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant(); + .getCommitsTimeline().filterCompletedInstants().lastInstant(); if (!latestCommit.isPresent()) { logger.warn("No commits present. Nothing to snapshot"); return; diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index ad8ccd2edf700..2baafd036382f 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -120,7 +120,7 @@ public HoodieDeltaStreamer(Config cfg) throws IOException { if (fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath); this.commitTimelineOpt = Optional - .of(meta.getActiveTimeline().getCommitsAndCompactionsTimeline() + .of(meta.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants()); } else { this.commitTimelineOpt = Optional.empty(); From 03844915f52e309e73c4b1b8d32534673b3a42f5 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Sun, 10 Dec 2017 23:31:54 -0800 Subject: [PATCH 003/374] Multi FS Support - Reviving PR 191, to make FileSystem creation off actual path - Streamline all filesystem access to HoodieTableMetaClient - Hadoop Conf from Spark Context serialized & passed to executor code too - Pick up env vars prefixed with HOODIE_ENV_ into Configuration object - Cleanup usage of FSUtils.getFS, piggybacking off HoodieTableMetaClient.getFS - Adding s3a to supported schemes & support escaping "." in env vars - Tests use HoodieTestUtils.getDefaultHadoopConf --- .../java/com/uber/hoodie/cli/HoodieCLI.java | 3 +- .../cli/commands/ArchivedCommitsCommand.java | 8 +- .../hoodie/cli/commands/CleansCommand.java | 2 +- .../hoodie/cli/commands/CommitsCommand.java | 9 +- .../hoodie/cli/commands/DatasetsCommand.java | 2 +- .../cli/commands/SavepointsCommand.java | 2 +- .../uber/hoodie/cli/commands/SparkMain.java | 4 +- .../com/uber/hoodie/cli/utils/SparkUtil.java | 2 + .../com/uber/hoodie/cli/DedupeSparkJob.scala | 4 +- .../com/uber/hoodie/cli/SparkHelpers.scala | 3 +- .../com/uber/hoodie/HoodieReadClient.java | 8 +- .../com/uber/hoodie/HoodieWriteClient.java | 98 ++++++----- .../uber/hoodie/func/LazyInsertIterable.java | 3 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 11 +- .../bloom/HoodieBloomIndexCheckFunction.java | 19 ++- .../com/uber/hoodie/io/HoodieCleanHelper.java | 3 - .../hoodie/io/HoodieCommitArchiveLog.java | 7 +- .../com/uber/hoodie/io/HoodieIOHandle.java | 7 +- .../compact/HoodieRealtimeTableCompactor.java | 42 ++--- .../io/storage/HoodieParquetWriter.java | 10 +- .../storage/HoodieStorageWriterFactory.java | 8 +- .../io/storage/HoodieWrapperFileSystem.java | 44 +++-- .../hoodie/table/HoodieCopyOnWriteTable.java | 29 ++-- .../hoodie/table/HoodieMergeOnReadTable.java | 22 ++- .../com/uber/hoodie/table/HoodieTable.java | 16 +- .../src/test/java/HoodieClientExample.java | 2 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 140 ++++++++------- .../java/com/uber/hoodie/TestMultiFS.java | 159 ++++++++++++++++++ .../hoodie/common/HoodieClientTestUtils.java | 2 +- .../common/HoodieMergeOnReadTestUtils.java | 11 +- .../common/HoodieTestDataGenerator.java | 5 +- .../hoodie/func/TestUpdateMapFunction.java | 11 +- .../index/bloom/TestHoodieBloomIndex.java | 26 ++- .../hoodie/io/TestHoodieCommitArchiveLog.java | 14 +- .../uber/hoodie/io/TestHoodieCompactor.java | 38 ++--- .../hoodie/table/TestCopyOnWriteTable.java | 27 +-- .../hoodie/table/TestMergeOnReadTable.java | 91 +++++----- hoodie-common/pom.xml | 6 + .../hoodie/avro/MercifulJsonConverter.java | 2 +- .../common/SerializableConfiguration.java | 56 ++++++ .../uber/hoodie/common/model/ActionType.java | 2 +- .../common/table/HoodieTableMetaClient.java | 29 +++- .../log/HoodieCompactedLogRecordScanner.java | 2 +- .../common/table/log/HoodieLogFormat.java | 2 +- .../table/log/block/HoodieLogBlock.java | 2 +- .../table/timeline/HoodieActiveTimeline.java | 50 +++--- .../timeline/HoodieArchivedTimeline.java | 19 +-- .../table/view/HoodieTableFileSystemView.java | 8 +- .../uber/hoodie/common/util/AvroUtils.java | 6 +- .../com/uber/hoodie/common/util/FSUtils.java | 28 ++- .../uber/hoodie/common/util/ParquetUtils.java | 44 +++-- .../common/minicluster/HdfsTestService.java | 5 +- .../hoodie/common/model/HoodieTestUtils.java | 14 +- .../common/table/log/HoodieLogFormatTest.java | 8 +- .../string/HoodieActiveTimelineTest.java | 4 +- .../view/HoodieTableFileSystemViewTest.java | 2 +- .../uber/hoodie/common/util/TestFSUtils.java | 20 +++ .../hoodie/common/util/TestParquetUtils.java | 8 +- .../uber/hoodie/hadoop/HoodieInputFormat.java | 2 +- .../hadoop/HoodieROTablePathFilter.java | 5 +- .../realtime/HoodieRealtimeRecordReader.java | 7 +- .../HoodieRealtimeRecordReaderTest.java | 16 +- .../com/uber/hoodie/hive/HiveSyncTool.java | 3 +- .../uber/hoodie/hive/HoodieHiveClient.java | 2 +- .../hoodie/hive/util/HiveTestService.java | 3 +- .../uber/hoodie/HoodieDataSourceHelpers.java | 2 +- .../com/uber/hoodie/IncrementalRelation.scala | 5 +- .../src/test/scala/DataSourceTest.scala | 2 +- .../hoodie/utilities/HDFSParquetImporter.java | 51 +++--- .../utilities/HiveIncrementalPuller.java | 4 +- .../utilities/HoodieSnapshotCopier.java | 19 ++- .../deltastreamer/HoodieDeltaStreamer.java | 11 +- .../schema/FilebasedSchemaProvider.java | 3 +- .../hoodie/utilities/sources/DFSSource.java | 2 +- .../utilities/sources/HiveIncrPullSource.java | 2 +- .../hoodie/utilities/sources/KafkaSource.java | 7 +- .../utilities/TestHDFSParquetImporter.java | 4 +- .../utilities/TestHoodieSnapshotCopier.java | 27 +-- 78 files changed, 851 insertions(+), 535 deletions(-) create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java index 27f0ab516f10b..15cce03e23e9b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java @@ -17,6 +17,7 @@ package com.uber.hoodie.cli; import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.util.FSUtils; import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -36,7 +37,7 @@ public enum CLIState { public static boolean initConf() { if (HoodieCLI.conf == null) { - HoodieCLI.conf = new Configuration(); + HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration()); return true; } return false; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index e1b03b271406b..d4945c0120aec 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -54,12 +54,14 @@ public String showCommits( System.out .println("===============> Showing only " + limit + " archived commits <==============="); - FileStatus[] fsStatuses = FSUtils.getFs().globStatus( - new Path(HoodieCLI.tableMetadata.getBasePath() + "/.hoodie/.commits_.archive*")); + String basePath = HoodieCLI.tableMetadata.getBasePath(); + FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf) + .globStatus(new Path(basePath + "/.hoodie/.commits_.archive*")); List allCommits = new ArrayList<>(); for (FileStatus fs : fsStatuses) { //read the archived file - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(), + HoodieLogFormat.Reader reader = HoodieLogFormat + .newReader(FSUtils.getFs(basePath, HoodieCLI.conf), new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false); List readRecords = new ArrayList<>(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java index 1b5a9602df995..38638866b647d 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java @@ -77,7 +77,7 @@ public String showCleans() throws IOException { @CliCommand(value = "cleans refresh", help = "Refresh the commits") public String refreshCleans() throws IOException { HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index d6446a2c4dd55..d7b6594d138c4 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -96,7 +96,7 @@ public String showCommits( @CliCommand(value = "commits refresh", help = "Refresh the commits") public String refreshCommits() throws IOException { HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } @@ -224,14 +224,13 @@ public boolean isCompareCommitsAvailable() { public String compareCommits( @CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) throws Exception { - HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path); + + HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); - ; HoodieTableMetaClient source = HoodieCLI.tableMetadata; HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); - ; String targetLatestCommit = targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); @@ -266,7 +265,7 @@ public boolean isSyncCommitsAvailable() { public String syncCommits( @CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) throws Exception { - HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path); + HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieCLI.state = HoodieCLI.CLIState.SYNC; return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java index fc1f22a3a28fc..de59d06694d86 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -33,7 +33,7 @@ public String connect( final String path) throws IOException { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); - HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path)); + HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path)); HoodieCLI.state = HoodieCLI.CLIState.DATASET; return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java index 4f5b2c9a3bfef..db267d6a656e5 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java @@ -138,7 +138,7 @@ public String rollbackToSavepoint( @CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") public String refreshMetaClient() throws IOException { HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index aba2d9da8d452..4f8f2f856b306 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -97,8 +97,8 @@ private static int deduplicatePartitionPath(JavaSparkContext jsc, String repairedOutputPath, String basePath) throws Exception { - DedupeSparkJob job = new DedupeSparkJob(basePath, - duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), FSUtils.getFs()); + DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, + new SQLContext(jsc), FSUtils.getFs(basePath, jsc.hadoopConfiguration())); job.fixDuplicates(true); return 0; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java index 4b4ab2a2d6353..d85626d358bf6 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java @@ -18,6 +18,7 @@ import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.cli.commands.SparkMain; +import com.uber.hoodie.common.util.FSUtils; import java.io.File; import java.net.URISyntaxException; import org.apache.log4j.Logger; @@ -66,6 +67,7 @@ public static JavaSparkContext initJavaSparkConf(String name) { sparkConf = HoodieWriteClient.registerClasses(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf); jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false); + FSUtils.prepareHadoopConf(jsc.hadoopConfiguration()); return jsc; } } diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala index 82c97e0a4b7d7..0058043ed6388 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala @@ -75,7 +75,7 @@ class DedupeSparkJob(basePath: String, val tmpTableName = s"htbl_${System.currentTimeMillis()}" val dedupeTblName = s"${tmpTableName}_dupeKeys" - val metadata = new HoodieTableMetaClient(fs, basePath) + val metadata = new HoodieTableMetaClient(fs.getConf, basePath) val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) @@ -127,7 +127,7 @@ class DedupeSparkJob(basePath: String, def fixDuplicates(dryRun: Boolean = true) = { - val metadata = new HoodieTableMetaClient(fs, basePath) + val metadata = new HoodieTableMetaClient(fs.getConf, basePath) val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala index 3fc18c8e4d5f3..8e1eccfeedb46 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala @@ -38,7 +38,7 @@ import scala.collection.mutable._ object SparkHelpers { @throws[Exception] def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) { - val sourceRecords = ParquetUtils.readAvroRecords(sourceFile) + val sourceRecords = ParquetUtils.readAvroRecords(fs.getConf, sourceFile) val schema: Schema = sourceRecords.get(0).getSchema val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble) val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter) @@ -47,6 +47,7 @@ object SparkHelpers { for (rec <- sourceRecords) { val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString if (!keysToSkip.contains(key)) { + writer.writeAvro(key, rec) } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 6c329c330bef9..eb9ad62e470ec 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -67,10 +67,10 @@ public class HoodieReadClient implements Serializable { */ public HoodieReadClient(JavaSparkContext jsc, String basePath) { this.jsc = jsc; - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); // Create a Hoodie table which encapsulated the commits and files visible this.hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); + .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), null); this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); this.index = new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc); @@ -129,8 +129,8 @@ public Dataset read(JavaRDD hoodieKeys, int parallelism) JavaPairRDD keyRowRDD = originalDF.javaRDD() .mapToPair(row -> { HoodieKey key = new HoodieKey( - row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), - row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); + row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), + row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); return new Tuple2<>(key, row); }); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 0de261c8f706a..cfd20b4e15f21 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -53,18 +53,6 @@ import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner; import com.uber.hoodie.table.WorkloadProfile; import com.uber.hoodie.table.WorkloadStat; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.storage.StorageLevel; -import scala.Option; -import scala.Tuple2; - import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; @@ -76,6 +64,17 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.storage.StorageLevel; +import scala.Option; +import scala.Tuple2; /** * Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient @@ -112,7 +111,7 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) */ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) { - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration()); this.jsc = jsc; this.config = clientConfig; this.index = HoodieIndex.createIndex(config, jsc); @@ -133,8 +132,9 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, */ public JavaRDD> filterExists(JavaRDD> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); @@ -146,8 +146,9 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco public JavaRDD upsert(JavaRDD> records, final String commitTime) { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); try { // De-dupe/merge if needed @@ -180,8 +181,9 @@ public JavaRDD upsert(JavaRDD> records, final Strin public JavaRDD insert(JavaRDD> records, final String commitTime) { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); try { // De-dupe/merge if needed JavaRDD> dedupedRecords = @@ -237,8 +239,9 @@ public JavaRDD bulkInsert(JavaRDD> records, Option bulkInsertPartitioner) { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); try { // De-dupe/merge if needed @@ -306,8 +309,7 @@ private JavaRDD> combineOnCondition(boolean condition, * file instead of using HoodieCommitMetadata */ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, - HoodieTable table, - String commitTime) throws HoodieCommitException { + HoodieTable table, String commitTime) throws HoodieCommitException { try { HoodieCommitMetadata metadata = new HoodieCommitMetadata(); profile.getPartitionPaths().stream().forEach(path -> { @@ -409,8 +411,9 @@ public boolean commit(String commitTime, logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); @@ -485,8 +488,9 @@ public boolean commit(String commitTime, * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); if (table.getCompletedCommitTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -512,8 +516,9 @@ public boolean savepoint(String user, String comment) { * @return true if the savepoint was created successfully */ public boolean savepoint(String commitTime, String user, String comment) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, @@ -575,8 +580,9 @@ public boolean savepoint(String commitTime, String user, String comment) { * @return true if the savepoint was deleted successfully */ public void deleteSavepoint(String savepointTime) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieInstant savePoint = @@ -602,8 +608,9 @@ public void deleteSavepoint(String savepointTime) { * @return true if the savepoint was rollecback to successfully */ public boolean rollbackToSavepoint(String savepointTime) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline commitTimeline = table.getCommitsTimeline(); @@ -653,8 +660,9 @@ private void rollback(List commits) { String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); @@ -761,8 +769,9 @@ private void clean(String startCleanTime) throws HoodieIOException { final Timer.Context context = metrics.getCleanCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), + true), config); List cleanStats = table.clean(jsc); if (cleanStats.isEmpty()) { @@ -810,8 +819,9 @@ public String startCommit() { public void startCommitWithTime(String commitTime) { logger.info("Generate a new commit time " + commitTime); - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = table.getCommitActionType(); activeTimeline.createInflight( @@ -827,8 +837,9 @@ public void startCommitWithTime(String commitTime) { */ private void compact(String compactionCommitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); Optional compactionMetadata = table.compact(jsc, compactionCommitTime); if (compactionMetadata.isPresent()) { logger.info("Compacted successfully on commit " + compactionCommitTime); @@ -876,8 +887,9 @@ private JavaRDD> deduplicateRecords(JavaRDD> rec * Cleanup all inflight commits */ private void rollbackInflightCommits() { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index cc038f21ac981..8b49897e7e783 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -72,7 +72,8 @@ protected List computeNext() { HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, record.getPartitionPath(), - TaskContext.getPartitionId()); + TaskContext.getPartitionId(), + hoodieTable); partitionsCleaned.add(record.getPartitionPath()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 5a0d69002c552..bf452bc8f30f0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -155,7 +155,8 @@ private JavaPairRDD lookupIndex( // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); - return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, + return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo, + partitionRecordKeyPairRDD, parallelism); } @@ -257,7 +258,8 @@ List> loadInvolvedFiles(List partitio .mapToPair(ft -> { try { String[] minMaxKeys = ParquetUtils - .readMinMaxRecordKeys(ft._2().getFileStatus().getPath()); + .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), + ft._2().getFileStatus().getPath()); return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); } catch (MetadataNotFoundException me) { @@ -358,7 +360,7 @@ JavaPairRDD> explodeRecordRDDWithFileCompariso * Make sure the parallelism is atleast the groupby parallelism for tagging location */ @VisibleForTesting - JavaPairRDD findMatchingFilesForRecordKeys( + JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTable, final Map> partitionToFileIndexInfo, JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions) { @@ -372,7 +374,8 @@ JavaPairRDD findMatchingFilesForRecordKeys( .sortByKey(true, joinParallelism); return fileSortedTripletRDD - .mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true) + .mapPartitionsWithIndex( + new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) .flatMap(indexLookupResults -> indexLookupResults.iterator()) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index 0d562ae8680de..cf5a12007de4e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -24,10 +24,12 @@ import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.func.LazyIterableIterator; +import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Set; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -45,20 +47,24 @@ public class HoodieBloomIndexCheckFunction implements private final String basePath; - public HoodieBloomIndexCheckFunction(String basePath) { + private final HoodieTable table; + + public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) { + this.table = table; this.basePath = basePath; } /** * Given a list of row keys and one file, return only row keys existing in that file. */ - public static List checkCandidatesAgainstFile(List candidateRecordKeys, + public static List checkCandidatesAgainstFile(Configuration configuration, + List candidateRecordKeys, Path filePath) throws HoodieIndexException { List foundRecordKeys = new ArrayList<>(); try { // Load all rowKeys from the file, to double-confirm if (!candidateRecordKeys.isEmpty()) { - Set fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath); + Set fileRowKeys = ParquetUtils.readRowKeysFromParquet(configuration, filePath); logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath); if (logger.isDebugEnabled()) { logger.debug("Keys from " + filePath + " => " + fileRowKeys); @@ -107,7 +113,8 @@ protected void start() { private void initState(String fileName, String partitionPath) throws HoodieIndexException { try { Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName); - bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath); + bloomFilter = ParquetUtils + .readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath); candidateRecordKeys = new ArrayList<>(); currentFile = fileName; currentParitionPath = partitionPath; @@ -154,7 +161,7 @@ protected List computeNext() { .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); } ret.add(new IndexLookupResult(currentFile, - checkCandidatesAgainstFile(candidateRecordKeys, filePath))); + checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); initState(fileName, partitionPath); if (bloomFilter.mightContain(recordKey)) { @@ -177,7 +184,7 @@ protected List computeNext() { logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); } ret.add(new IndexLookupResult(currentFile, - checkCandidatesAgainstFile(candidateRecordKeys, filePath))); + checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); } } catch (Throwable e) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index 086b87f89a7d6..85028ba292fcb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -51,14 +50,12 @@ public class HoodieCleanHelper> { private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; private HoodieWriteConfig config; - private FileSystem fs; public HoodieCleanHelper(HoodieTable hoodieTable, HoodieWriteConfig config) { this.hoodieTable = hoodieTable; this.fileSystemView = hoodieTable.getCompletedFileSystemView(); this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); this.config = config; - this.fs = hoodieTable.getFs(); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 3cb697c7a1078..7704517c629e7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -126,7 +126,8 @@ private Stream getInstantsToArchive() { int minCommitsToKeep = config.getMinCommitsToKeep(); HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true), + config); // GroupBy each action and limit each action timeline to maxCommitsToKeep HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() @@ -165,7 +166,7 @@ private Stream getInstantsToArchive() { private boolean deleteArchivedInstants(List archivedInstants) { log.info("Deleting instants " + archivedInstants); HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs, config.getBasePath(), true); + new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { @@ -188,7 +189,7 @@ public void archive(List instants) throws HoodieCommitException { try { HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs, config.getBasePath(), true); + new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true); HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 53ec545fc059b..332c9a2d3ba1f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -47,7 +47,7 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { this.commitTime = commitTime; this.config = config; - this.fs = FSUtils.getFs(); + this.fs = hoodieTable.getMetaClient().getFs(); this.hoodieTable = hoodieTable; this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline(); this.fileSystemView = hoodieTable.getROFileSystemView(); @@ -73,8 +73,9 @@ public Path makeNewPath(String partitionPath, int taskPartitionId, String fileNa public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, String commitTime, String partitionPath, - int taskPartitionId) { - FileSystem fs = FSUtils.getFs(); + int taskPartitionId, + HoodieTable hoodieTable) { + FileSystem fs = hoodieTable.getMetaClient().getFs(); try { FileStatus[] prevFailedFiles = fs.globStatus(new Path(String .format("%s/%s/%s", config.getBasePath(), partitionPath, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 92107c0ccfc9a..fda6b5a268f1d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -16,6 +16,8 @@ package com.uber.hoodie.io.compact; +import static java.util.stream.Collectors.toList; + import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -34,13 +36,6 @@ import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FlatMapFunction; - import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Collection; @@ -49,8 +44,12 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.StreamSupport; - -import static java.util.stream.Collectors.toList; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; /** * HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all @@ -73,7 +72,6 @@ public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf .getTableType().name()); // TODO - rollback any compactions in flight - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); List partitionPaths = @@ -102,15 +100,9 @@ public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf log.info("After filtering, Compacting " + operations + " files"); List updateStatusMap = jsc.parallelize(operations, operations.size()) - .map(s -> executeCompaction(metaClient, config, s, compactionCommitTime)) - .flatMap(new FlatMapFunction, HoodieWriteStat>() { - @Override - public Iterator call( - List hoodieWriteStats) - throws Exception { - return hoodieWriteStats.iterator(); - } - }).collect(); + .map(s -> executeCompaction(hoodieTable, config, s, compactionCommitTime)) + .flatMap(writeStatList -> writeStatList.iterator()) + .collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); for (HoodieWriteStat stat : updateStatusMap) { @@ -134,10 +126,11 @@ private boolean isCompactionSucceeded(HoodieCommitMetadata result) { return true; } - private List executeCompaction(HoodieTableMetaClient metaClient, - HoodieWriteConfig config, CompactionOperation operation, String commitTime) + + private List executeCompaction(HoodieTable hoodieTable, + HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException { - FileSystem fs = FSUtils.getFs(); + FileSystem fs = hoodieTable.getMetaClient().getFs(); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); @@ -147,7 +140,7 @@ private List executeCompaction(HoodieTableMetaClient metaClient // Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover). // Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon. - + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); String maxInstantTime = metaClient.getActiveTimeline() .getTimelineOfActions( Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, @@ -162,8 +155,7 @@ private List executeCompaction(HoodieTableMetaClient metaClient } // Compacting is very similar to applying updates to existing file - HoodieCopyOnWriteTable table = - new HoodieCopyOnWriteTable(config, metaClient); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); Iterator> result = table .handleUpdate(commitTime, operation.getFileId(), scanner.iterator()); Iterable> resultIterable = () -> result; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java index 2e207fd7f12fb..48bdfda59e6c7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java @@ -19,6 +19,7 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; @@ -26,7 +27,6 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.ParquetFileWriter; import org.apache.parquet.hadoop.ParquetWriter; @@ -51,9 +51,9 @@ public class HoodieParquetWriter HoodieSto throws IOException { //TODO - based on the metadata choose the implementation of HoodieStorageWriter // Currently only parquet is supported - return newParquetStorageWriter(commitTime, path, config, schema); + return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable); } private static HoodieStorageWriter newParquetStorageWriter( - String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException { + String commitTime, Path path, HoodieWriteConfig config, Schema schema, + HoodieTable hoodieTable) throws IOException { BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP()); HoodieAvroWriteSupport writeSupport = @@ -50,7 +50,7 @@ private static HoodieSt HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, config.getParquetBlockSize(), config.getParquetPageSize(), - config.getParquetMaxFileSize(), FSUtils.getFs().getConf()); + config.getParquetMaxFileSize(), hoodieTable.getHadoopConf()); return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java index 87a628447a434..a9d2a2298e910 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java @@ -16,7 +16,8 @@ package com.uber.hoodie.io.storage; -import java.io.FileNotFoundException; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -33,7 +34,6 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -41,17 +41,14 @@ import org.apache.hadoop.fs.FsStatus; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Options; -import org.apache.hadoop.fs.ParentNotDirectoryException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Progressable; @@ -70,6 +67,8 @@ public class HoodieWrapperFileSystem extends FileSystem { SUPPORT_SCHEMES.add("file"); SUPPORT_SCHEMES.add("hdfs"); SUPPORT_SCHEMES.add("s3"); + SUPPORT_SCHEMES.add("s3a"); + // Hoodie currently relies on underlying object store being fully // consistent so only regional buckets should be used. @@ -85,7 +84,12 @@ public class HoodieWrapperFileSystem extends FileSystem { @Override public void initialize(URI uri, Configuration conf) throws IOException { // Get the default filesystem to decorate - fileSystem = FileSystem.get(conf); + Path path = new Path(uri); + // Remove 'hoodie-' prefix from path + if (path.toString().startsWith(HOODIE_SCHEME_PREFIX)) { + path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, "")); + } + this.fileSystem = FSUtils.getFs(path.toString(), conf); // Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get // fileSystem.initialize(FileSystem.getDefaultUri(conf), conf); // fileSystem.setConf(conf); @@ -219,7 +223,7 @@ public boolean delete(Path f, boolean recursive) throws IOException { } @Override - public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { + public FileStatus[] listStatus(Path f) throws IOException { return fileSystem.listStatus(convertToDefaultPath(f)); } @@ -415,19 +419,19 @@ public RemoteIterator listCorruptFileBlocks(Path path) throws IOException @Override public FileStatus[] listStatus(Path f, PathFilter filter) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listStatus(convertToDefaultPath(f), filter); } @Override public FileStatus[] listStatus(Path[] files) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listStatus(convertDefaults(files)); } @Override public FileStatus[] listStatus(Path[] files, PathFilter filter) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listStatus(convertDefaults(files), filter); } @@ -444,13 +448,13 @@ public FileStatus[] globStatus(Path pathPattern, PathFilter filter) @Override public RemoteIterator listLocatedStatus(Path f) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listLocatedStatus(convertToDefaultPath(f)); } @Override public RemoteIterator listFiles(Path f, boolean recursive) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listFiles(convertToDefaultPath(f), recursive); } @@ -571,21 +575,21 @@ public short getDefaultReplication(Path path) { @Override public void access(Path path, FsAction mode) - throws AccessControlException, FileNotFoundException, IOException { + throws IOException { fileSystem.access(convertToDefaultPath(path), mode); } @Override public void createSymlink(Path target, Path link, boolean createParent) - throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, - ParentNotDirectoryException, UnsupportedFileSystemException, IOException { + throws + IOException { fileSystem .createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent); } @Override public FileStatus getFileLinkStatus(Path f) - throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException, + throws IOException { return fileSystem.getFileLinkStatus(convertToDefaultPath(f)); } @@ -759,8 +763,12 @@ public Path convertToHoodiePath(Path oldPath) { } public static Path convertToHoodiePath(Path file, Configuration conf) { - String scheme = FileSystem.getDefaultUri(conf).getScheme(); - return convertPathWithScheme(file, getHoodieScheme(scheme)); + try { + String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); + return convertPathWithScheme(file, getHoodieScheme(scheme)); + } catch (HoodieIOException e) { + throw e; + } } private Path convertToDefaultPath(Path oldPath) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index a509411d771cb..3e4b1dc86fcc5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -52,7 +52,6 @@ import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -413,10 +412,10 @@ public Iterator> handleUpdate(String commitTime, String fileLo throw new HoodieUpsertException("Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc); } else { - Configuration conf = FSUtils.getFs().getConf(); - AvroReadSupport.setAvroReadSchema(conf, upsertHandle.getSchema()); + AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); ParquetReader reader = - AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(conf).build(); + AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf()) + .build(); try { IndexedRecord record; while ((record = reader.read()) != null) { @@ -500,8 +499,9 @@ public Optional compact(JavaSparkContext jsc, String commi @Override public List clean(JavaSparkContext jsc) { try { + FileSystem fs = getMetaClient().getFs(); List partitionsToClean = - FSUtils.getAllPartitionPaths(getFs(), getMetaClient().getBasePath(), + FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()); logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config .getCleanerPolicy()); @@ -522,7 +522,7 @@ public List clean(JavaSparkContext jsc) { protected Map deleteCleanedFiles(String partitionPath, List commits) throws IOException { logger.info("Cleaning path " + partitionPath); - FileSystem fs = FSUtils.getFs(); + FileSystem fs = getMetaClient().getFs(); FileStatus[] toBeDeleted = fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> { if (!path.toString().contains(".parquet")) { @@ -558,7 +558,7 @@ public List rollback(JavaSparkContext jsc, List comm // delete all the data files for all these commits logger.info("Clean out all parquet files generated for commits: " + commits); List stats = jsc.parallelize( - FSUtils.getAllPartitionPaths(FSUtils.getFs(), this.getMetaClient().getBasePath(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) .map((Function) partitionPath -> { // Scan all partitions files with this commit time @@ -618,7 +618,7 @@ private List cleanPartitionPaths(List partitionsToClean .parallelize(partitionsToClean, cleanerParallelism) .flatMapToPair(getFilesToDeleteFunc(this, config)) .repartition(cleanerParallelism) // repartition to remove skews - .mapPartitionsToPair(deleteFilesFunc(this, config)) + .mapPartitionsToPair(deleteFilesFunc(this)) .reduceByKey( // merge partition level clean stats below (Function2) (e1, e2) -> e1 @@ -646,17 +646,17 @@ private List cleanPartitionPaths(List partitionsToClean }).collect(Collectors.toList()); } - private PairFlatMapFunction>, String, PartitionCleanStat> deleteFilesFunc( - HoodieTable table, HoodieWriteConfig config) { + private static PairFlatMapFunction>, String, PartitionCleanStat> deleteFilesFunc( + HoodieTable table) { return (PairFlatMapFunction>, String, PartitionCleanStat>) iter -> { - HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config); Map partitionCleanStatMap = new HashMap<>(); + FileSystem fs = table.getMetaClient().getFs(); while (iter.hasNext()) { Tuple2 partitionDelFileTuple = iter.next(); String partitionPath = partitionDelFileTuple._1(); String deletePathStr = partitionDelFileTuple._2(); - Boolean deletedFileResult = deleteFileAndGetResult(deletePathStr); + Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); if (!partitionCleanStatMap.containsKey(partitionPath)) { partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath)); @@ -682,10 +682,11 @@ private static PairFlatMapFunction getFilesToDeleteFunc( }; } - private Boolean deleteFileAndGetResult(String deletePathStr) throws IOException { + private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) + throws IOException { Path deletePath = new Path(deletePathStr); logger.debug("Working on delete path :" + deletePath); - boolean deleteResult = getFs().delete(deletePath, false); + boolean deleteResult = fs.delete(deletePath, false); if (deleteResult) { logger.debug("Cleaned file at path :" + deletePath); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 19461b025368d..839aa1840ebeb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -32,19 +32,11 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - import java.io.IOException; import java.io.UncheckedIOException; import java.util.Arrays; @@ -55,6 +47,12 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; /** @@ -74,8 +72,7 @@ public class HoodieMergeOnReadTable extends private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class); - public HoodieMergeOnReadTable(HoodieWriteConfig config, - HoodieTableMetaClient metaClient) { + public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { super(config, metaClient); } @@ -195,7 +192,7 @@ public List rollback(JavaSparkContext jsc, List comm .onParentPath( new Path(this.getMetaClient().getBasePath(), partitionPath)) .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) - .withFs(FSUtils.getFs()) + .withFs(getMetaClient().getFs()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; // generate metadata @@ -215,7 +212,8 @@ public List rollback(JavaSparkContext jsc, List comm numRollbackBlocks++; } filesToNumBlocksRollback - .put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), + .put(getMetaClient().getFs() + .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); } catch (IOException | InterruptedException io) { throw new HoodieRollbackException( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 76ab92d99ca21..3140e34eec286 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -34,12 +34,6 @@ import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieSavepointException; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaSparkContext; - import java.io.IOException; import java.io.Serializable; import java.util.Iterator; @@ -47,6 +41,11 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaSparkContext; /** * Abstract implementation of a HoodieTable @@ -55,6 +54,7 @@ public abstract class HoodieTable implements Seri protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; + private static Logger logger = LogManager.getLogger(HoodieTable.class); protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { @@ -87,8 +87,8 @@ public HoodieTableMetaClient getMetaClient() { return metaClient; } - public FileSystem getFs() { - return metaClient.getFs(); + public Configuration getHadoopConf() { + return metaClient.getHadoopConf(); } /** diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index ef31fea1cf084..f1fc056e1dd10 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -81,7 +81,7 @@ public void run() throws Exception { // initialize the table, if not done already Path path = new Path(tablePath); - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration()); if (!fs.exists(path)) { HoodieTableMetaClient .initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName, diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index f6995762d3e05..e3adf8d9c8393 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -84,6 +84,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { private transient JavaSparkContext jsc = null; private transient SQLContext sqlContext; + private transient FileSystem fs; private String basePath = null; private transient HoodieTestDataGenerator dataGen = null; private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"}; @@ -100,6 +101,7 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration()); HoodieTestUtils.init(basePath); dataGen = new HoodieTestDataGenerator(); } @@ -212,7 +214,6 @@ public void testUpserts() throws Exception { HoodieWriteConfig cfg = getConfig(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - FileSystem fs = FSUtils.getFs(); /** * Write 1 (only inserts) @@ -230,8 +231,9 @@ public void testUpserts() throws Exception { assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) .getCommitTimeline(); assertEquals("Expecting a single commit.", 1, @@ -242,7 +244,8 @@ public void testUpserts() throws Exception { records.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable + .getHoodieTable(metaClient, getConfig()); List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) .collect(); @@ -268,13 +271,13 @@ public void testUpserts() throws Exception { assertNoWriteErrors(statuses); // verify there are now 2 commits - timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline(); + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, getConfig()); // Index should be able to locate all updates in correct locations. @@ -305,7 +308,6 @@ public void testDeletes() throws Exception { HoodieWriteConfig cfg = getConfig(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - FileSystem fs = FSUtils.getFs(); /** * Write 1 (inserts and deletes) @@ -326,8 +328,9 @@ public void testDeletes() throws Exception { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) .getCommitTimeline(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); @@ -336,7 +339,8 @@ public void testDeletes() throws Exception { assertEquals("Must contain 200 records", fewRecordsForInsert.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable + .getHoodieTable(metaClient, getConfig()); List taggedRecords = index .tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); @@ -359,7 +363,7 @@ public void testDeletes() throws Exception { assertNoWriteErrors(statuses); // verify there are now 2 commits - timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline(); + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), @@ -393,7 +397,6 @@ public void testCreateSavepoint() throws Exception { .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - FileSystem fs = FSUtils.getFs(); HoodieTestDataGenerator .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); @@ -451,8 +454,10 @@ public void testCreateSavepoint() throws Exception { List partitionPaths = FSUtils .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + HoodieTable table = HoodieTable + .getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); List dataFiles = partitionPaths.stream().flatMap(s -> { return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); @@ -472,7 +477,7 @@ public void testCreateSavepoint() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); dataFiles = partitionPaths.stream().flatMap(s -> { @@ -490,7 +495,6 @@ public void testRollbackToSavepoint() throws Exception { .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - FileSystem fs = FSUtils.getFs(); HoodieTestDataGenerator .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); @@ -531,7 +535,8 @@ public void testRollbackToSavepoint() throws Exception { assertNoWriteErrors(statuses); List partitionPaths = FSUtils .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); @@ -551,7 +556,7 @@ public void testRollbackToSavepoint() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); @@ -573,7 +578,7 @@ public void testRollbackToSavepoint() throws Exception { table.getCompletedSavepointTimeline().getInstants().findFirst().get(); client.rollbackToSavepoint(savepoint.getTimestamp()); - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); dataFiles = partitionPaths.stream().flatMap(s -> { @@ -602,7 +607,6 @@ public void testInsertAndCleanByVersions() throws Exception { .retainFileVersions(maxVersions).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - FileSystem fs = FSUtils.getFs(); /** * do a big insert @@ -618,8 +622,9 @@ public void testInsertAndCleanByVersions() throws Exception { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) .getCommitTimeline(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); @@ -647,7 +652,8 @@ public void testInsertAndCleanByVersions() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); table = HoodieTable.getHoodieTable(metadata, getConfig()); timeline = table.getCommitsTimeline(); @@ -702,7 +708,6 @@ public void testInsertAndCleanByCommits() throws Exception { .retainCommits(maxCommits).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - FileSystem fs = FSUtils.getFs(); /** * do a big insert @@ -718,13 +723,15 @@ public void testInsertAndCleanByCommits() throws Exception { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) .getCommitTimeline(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable + .getHoodieTable(metaClient, getConfig()); assertFalse(table.getCompletedCommitTimeline().empty()); String commitTime = @@ -747,7 +754,8 @@ public void testInsertAndCleanByCommits() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); Optional @@ -788,8 +796,8 @@ public void testRollbackCommit() throws Exception { String commitTime2 = "20160502020601"; String commitTime3 = "20160506030611"; new File(basePath + "/.hoodie").mkdirs(); - HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(), - new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, + HoodieTestDataGenerator + .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // Only first two have commit files @@ -878,9 +886,9 @@ public void testAutoRollbackCommit() throws Exception { String commitTime2 = "20160502020601"; String commitTime3 = "20160506030611"; new File(basePath + "/.hoodie").mkdirs(); - HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(), - new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + HoodieTestDataGenerator + .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, + basePath); // One good commit HoodieTestUtils.createCommitFiles(basePath, commitTime1); @@ -956,8 +964,6 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { @Test public void testSmallInsertHandlingForUpserts() throws Exception { - - FileSystem fs = FSUtils.getFs(); final String TEST_PARTITION_PATH = "2016/09/26"; final int INSERT_SPLIT_LIMIT = 100; // setup the small file handling params @@ -982,7 +988,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); assertEquals("file should contain 100 records", - ParquetUtils.readRowKeysFromParquet(new Path(basePath, + ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100); @@ -1006,9 +1012,9 @@ public void testSmallInsertHandlingForUpserts() throws Exception { Path newFile = new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); assertEquals("file should contain 140 records", - ParquetUtils.readRowKeysFromParquet(newFile).size(), 140); + ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140); - List records = ParquetUtils.readAvroRecords(newFile); + List records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile); for (GenericRecord record : records) { String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); assertEquals("only expect commit2", commitTime2, @@ -1030,7 +1036,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { assertNoWriteErrors(statuses); assertEquals("2 files needs to be committed.", 2, statuses.size()); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView(); List files = fileSystemView @@ -1040,7 +1046,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { for (HoodieDataFile file : files) { if (file.getFileName().contains(file1)) { assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime()); - records = ParquetUtils.readAvroRecords(new Path(file.getPath())); + records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); for (GenericRecord record : records) { String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); @@ -1057,7 +1063,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { keys2.size()); } else { assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime()); - records = ParquetUtils.readAvroRecords(new Path(file.getPath())); + records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); for (GenericRecord record : records) { String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); assertEquals("only expect commit3", commitTime3, @@ -1091,12 +1097,12 @@ public void testSmallInsertHandlingForInserts() throws Exception { List statuses = client.insert(insertRecordsRDD1, commitTime1).collect(); assertNoWriteErrors(statuses); - assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, FSUtils.getFs()); + assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, fs); assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); assertEquals("file should contain 100 records", - ParquetUtils.readRowKeysFromParquet(new Path(basePath, + ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100); @@ -1116,9 +1122,9 @@ public void testSmallInsertHandlingForInserts() throws Exception { Path newFile = new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); assertEquals("file should contain 140 records", - ParquetUtils.readRowKeysFromParquet(newFile).size(), 140); + ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140); - List records = ParquetUtils.readAvroRecords(newFile); + List records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile); for (GenericRecord record : records) { String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); @@ -1137,8 +1143,8 @@ public void testSmallInsertHandlingForInserts() throws Exception { assertNoWriteErrors(statuses); assertEquals("2 files needs to be committed.", 2, statuses.size()); - FileSystem fs = FSUtils.getFs(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); List files = table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3) @@ -1148,7 +1154,7 @@ public void testSmallInsertHandlingForInserts() throws Exception { int totalInserts = 0; for (HoodieDataFile file : files) { assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime()); - records = ParquetUtils.readAvroRecords(new Path(file.getPath())); + records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); totalInserts += records.size(); } assertEquals("Total number of records must add up", totalInserts, @@ -1169,7 +1175,8 @@ public void testKeepLatestFileVersions() throws IOException { String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); @@ -1183,7 +1190,8 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config); String file2P0C1 = HoodieTestUtils @@ -1206,7 +1214,8 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update @@ -1260,8 +1269,7 @@ public void testKeepLatestFileVersionsMOR() throws IOException { // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(basePath, "001"); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); List hoodieCleanStats = table.clean(jsc); assertEquals("Must clean three files, one parquet and 2 log files", 3, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); @@ -1286,9 +1294,8 @@ public void testKeepLatestCommits() throws IOException { String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -1300,9 +1307,8 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); - table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), - config); + table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); String file2P0C1 = HoodieTestUtils .createNewDataFile(basePath, partitionPaths[0], "001"); // insert @@ -1324,7 +1330,8 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update @@ -1341,7 +1348,8 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "003"); table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update @@ -1384,7 +1392,8 @@ public void testCleaningWithZeroPartitonPaths() throws IOException { HoodieTestUtils.createCommitFiles(basePath, "000"); HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); @@ -1449,7 +1458,8 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003"); HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), + .getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStats = table.clean(jsc); @@ -1476,8 +1486,8 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - FileSystem fs = FSUtils.getFs(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); String commitTime = "000"; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java new file mode 100644 index 0000000000000..1bdc15d25a623 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java @@ -0,0 +1,159 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie; + + +import static org.junit.Assert.assertEquals; + +import com.uber.hoodie.common.HoodieClientTestUtils; +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.minicluster.HdfsTestService; +import com.uber.hoodie.common.model.HoodieAvroPayload; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.HoodieIndex; +import java.io.Serializable; +import java.util.List; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +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.Before; +import org.junit.Test; + +public class TestMultiFS implements Serializable { + + private static String dfsBasePath; + private static HdfsTestService hdfsTestService; + private static MiniDFSCluster dfsCluster; + private static DistributedFileSystem dfs; + private static Logger logger = LogManager.getLogger(TestMultiFS.class); + private String tablePath = "file:///tmp/hoodie/sample-table"; + private String tableName = "hoodie_rt"; + private String tableType = HoodieTableType.COPY_ON_WRITE.name(); + private static JavaSparkContext jsc; + private static SQLContext sqlContext; + + @Before + public void initClass() throws Exception { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + + // Create a temp folder as the base path + dfs = dfsCluster.getFileSystem(); + dfsBasePath = dfs.getWorkingDirectory().toString(); + dfs.mkdirs(new Path(dfsBasePath)); + + SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example"); + sparkConf.setMaster("local[1]"); + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + sparkConf.set("spark.kryoserializer.buffer.max", "512m"); + jsc = new JavaSparkContext(sparkConf); + sqlContext = new SQLContext(jsc); + } + + @After + public void cleanupClass() throws Exception { + if (hdfsTestService != null) { + hdfsTestService.stop(); + } + if (jsc != null) { + jsc.stop(); + } + FSUtils.setFs(null); + } + + @Test + public void readLocalWriteHDFS() throws Exception { + + // Generator of some records to be loaded in. + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + + // Initialize table and filesystem + FileSystem hdfs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration()); + HoodieTableMetaClient + .initTableType(hdfs, dfsBasePath, HoodieTableType.valueOf(tableType), tableName, + HoodieAvroPayload.class.getName()); + + //Create write client to write some records in + HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .forTable(tableName).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .build(); + HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg); + + // Write generated data to hdfs (only inserts) + String readCommitTime = hdfsWriteClient.startCommit(); + logger.info("Starting commit " + readCommitTime); + List records = dataGen.generateInserts(readCommitTime, 100); + JavaRDD writeRecords = jsc.parallelize(records, 1); + hdfsWriteClient.upsert(writeRecords, readCommitTime); + + // Read from hdfs + FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) + .getCommitTimeline(); + Dataset readRecords = HoodieClientTestUtils + .readCommit(dfsBasePath, sqlContext, timeline, readCommitTime); + assertEquals("Should contain 100 records", readRecords.count(), records.size()); + + // Write to local + FileSystem local = FSUtils.getFs(tablePath, jsc.hadoopConfiguration()); + HoodieTableMetaClient + .initTableType(local, tablePath, HoodieTableType.valueOf(tableType), tableName, + HoodieAvroPayload.class.getName()); + HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .forTable(tableName).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .build(); + HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig); + + String writeCommitTime = localWriteClient.startCommit(); + logger.info("Starting write commit " + writeCommitTime); + List localRecords = dataGen.generateInserts(writeCommitTime, 100); + JavaRDD localWriteRecords = jsc.parallelize(localRecords, 1); + logger.info("Writing to path: " + tablePath); + localWriteClient.upsert(localWriteRecords, writeCommitTime); + + logger.info("Reading from path: " + tablePath); + fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf()); + metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath); + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + Dataset localReadRecords = HoodieClientTestUtils + .readCommit(tablePath, sqlContext, timeline, writeCommitTime); + assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size()); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index c2db12d5e86e1..295d415259ed4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -174,7 +174,7 @@ public static Dataset read(String basePath, List filteredPaths = new ArrayList<>(); try { HoodieTable hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); + .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); for (String path : paths) { TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( hoodieTable.getMetaClient(), diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java index ec3d5728cf5be..c7e3e915618aa 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java @@ -18,7 +18,7 @@ import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; -import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat; import java.io.IOException; @@ -42,12 +42,13 @@ */ public class HoodieMergeOnReadTestUtils { - public static List getRecordsUsingInputFormat(List inputPaths) + public static List getRecordsUsingInputFormat(List inputPaths, + String basePath) throws IOException { JobConf jobConf = new JobConf(); Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA)); HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat(); - setPropsForInputFormat(inputFormat, jobConf, schema); + setPropsForInputFormat(inputFormat, jobConf, schema, basePath); return inputPaths.stream().map(path -> { setInputPath(jobConf, path); List records = new ArrayList<>(); @@ -76,12 +77,12 @@ public static List getRecordsUsingInputFormat(List inputP } private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, - Schema schema) { + Schema schema, String basePath) { List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); String postions = fields.stream().map(f -> String.valueOf(f.pos())) .collect(Collectors.joining(",")); - Configuration conf = FSUtils.getFs().getConf(); + Configuration conf = HoodieTestUtils.getDefaultHadoopConf(); jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); jobConf.set("partition_columns", "datestr"); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index c197e6b5167b6..beb76df651ab7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.util.FSUtils; @@ -193,7 +194,7 @@ public static void createCommitFile(String basePath, String commitTime) throws I Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline .makeCommitFileName(commitTime)); - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); FSDataOutputStream os = fs.create(commitFile, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); try { @@ -209,7 +210,7 @@ public static void createSavepointFile(String basePath, String commitTime) throw Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline .makeSavePointFileName(commitTime)); - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); FSDataOutputStream os = fs.create(commitFile, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); try { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index 8433c03661da9..ef4a86833f0e6 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -55,8 +55,9 @@ public void init() throws Exception { public void testSchemaEvolutionOnUpdate() throws Exception { // Create a bunch of records with a old version of schema HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient( + HoodieTestUtils.getDefaultHadoopConf(), basePath); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; @@ -80,16 +81,16 @@ public void testSchemaEvolutionOnUpdate() throws Exception { Iterator> insertResult = table.handleInsert("100", records.iterator()); Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); - FSUtils.getFs().create(commitFile); + FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile); // Now try an update with an evolved schema // Evolved schema does not have guarantee on preserving the original field ordering config = makeHoodieClientConfig("/exampleEvolvedSchema.txt"); - metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath); String fileId = insertResult.next().get(0).getFileId(); System.out.println(fileId); - table = new HoodieCopyOnWriteTable(config, metadata); + table = new HoodieCopyOnWriteTable(config, metaClient); // New content with values for the newly added field recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 8d4cc2558380b..92dcae96b1feb 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -54,7 +54,6 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroSchemaConverter; @@ -73,12 +72,11 @@ public class TestHoodieBloomIndex { private JavaSparkContext jsc = null; private String basePath = null; - private transient final FileSystem fs; + private transient FileSystem fs; private String schemaStr; private Schema schema; public TestHoodieBloomIndex() throws Exception { - fs = FSUtils.getFs(); } @Before @@ -89,6 +87,7 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); HoodieTestUtils.init(basePath); // We have some records to be tagged (two different partitions) schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); @@ -120,8 +119,6 @@ public void testLoadUUIDsInMemory() throws IOException { .parallelize(Arrays.asList(record1, record2, record3, record4)); // Load to memory - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - Map> map = recordRDD .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())) .groupByKey().collectAsMap(); @@ -174,7 +171,7 @@ public void testLoadInvolvedFiles() throws IOException { Arrays.asList(record2, record3, record4), schema, null, false); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); List> filesList = index.loadInvolvedFiles(partitions, table); // Still 0, as no valid commit @@ -291,7 +288,8 @@ public void testCheckUUIDsAgainstOneFile() List uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey()); - List results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids, + List results = HoodieBloomIndexCheckFunction + .checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids, new Path(basePath + "/2016/01/31/" + filename)); assertEquals(results.size(), 2); assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") @@ -308,7 +306,7 @@ public void testTagLocationWithEmptyRDD() throws Exception { // We have some records to be tagged (two different partitions) JavaRDD recordRDD = jsc.emptyRDD(); // Also create the metadata and config - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); @@ -348,7 +346,7 @@ public void testTagLocation() throws Exception { .parallelize(Arrays.asList(record1, record2, record3, record4)); // Also create the metadata and config - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); @@ -367,7 +365,7 @@ public void testTagLocation() throws Exception { String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true); // We do the tag again - metadata = new HoodieTableMetaClient(fs, basePath); + metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metadata, config); taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); @@ -409,7 +407,7 @@ public void testCheckExists() throws Exception { JavaRDD keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4)); // Also create the metadata and config - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); @@ -429,7 +427,7 @@ public void testCheckExists() throws Exception { String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true); // We do the tag again - metadata = new HoodieTableMetaClient(fs, basePath); + metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metadata, config); taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); @@ -476,7 +474,7 @@ public void testBloomFilterFalseError() throws IOException, InterruptedException // We do the tag JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2)); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); @@ -515,7 +513,7 @@ private String writeParquetFile(String partitionPath, String filename, List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); newCommitTime = "101"; @@ -161,7 +159,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { updatedRecords); // Verify that all data file has one log file - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = @@ -174,14 +172,14 @@ public void testLogFileCountsAfterCompaction() throws Exception { } // Do a compaction - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); HoodieCommitMetadata result = compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); // Verify that recently written compacted data file has no log file - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index a98b76838316b..61303c3fb7ce4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -89,7 +89,8 @@ public void testMakeNewPath() throws Exception { String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieWriteConfig config = makeHoodieClientConfig(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath); @@ -115,7 +116,7 @@ public void testUpdateRecords() throws Exception { // Prepare the AvroParquetIO HoodieWriteConfig config = makeHoodieClientConfig(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); String partitionPath = "/2016/01/31"; HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); @@ -154,7 +155,8 @@ public void testUpdateRecords() throws Exception { // Read out the bloom filter and make sure filter can answer record exist or not Path parquetFilePath = new Path(parquetFile.getAbsolutePath()); - BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(parquetFilePath); + BloomFilter filter = ParquetUtils + .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath); for (HoodieRecord record : records) { assertTrue(filter.mightContain(record.getRecordKey())); } @@ -163,7 +165,8 @@ public void testUpdateRecords() throws Exception { + FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile(); // Read the parquet file, check the record content - List fileRecords = ParquetUtils.readAvroRecords(parquetFilePath); + List fileRecords = ParquetUtils + .readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath); GenericRecord newRecord; int index = 0; for (GenericRecord record : fileRecords) { @@ -188,7 +191,7 @@ public void testUpdateRecords() throws Exception { Thread.sleep(1000); String newCommitTime = HoodieTestUtils.makeNewCommitTime(); - metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = new HoodieCopyOnWriteTable(config, metadata); Iterator> iter = table .handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), @@ -211,7 +214,7 @@ public void testUpdateRecords() throws Exception { // Check whether the record has been updated Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath()); BloomFilter updatedFilter = ParquetUtils - .readBloomFilterFromParquetMetadata(updatedParquetFilePath); + .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath); for (HoodieRecord record : records) { // No change to the _row_key assertTrue(updatedFilter.mightContain(record.getRecordKey())); @@ -261,7 +264,7 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { HoodieWriteConfig config = makeHoodieClientConfigBuilder() .withWriteStatusClass(MetadataMergeWriteStatus.class).build(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); @@ -298,8 +301,8 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { public void testInsertWithPartialFailures() throws Exception { HoodieWriteConfig config = makeHoodieClientConfig(); String commitTime = HoodieTestUtils.makeNewCommitTime(); - FileSystem fs = FSUtils.getFs(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); + FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); // Write a few records, and get atleast one file @@ -340,7 +343,7 @@ public void testInsertWithPartialFailures() throws Exception { public void testInsertRecords() throws Exception { HoodieWriteConfig config = makeHoodieClientConfig(); String commitTime = HoodieTestUtils.makeNewCommitTime(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); // Case 1: @@ -389,7 +392,7 @@ public void testFileSizeUpsertRecords() throws Exception { HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024) .parquetPageSize(64 * 1024).build()).build(); String commitTime = HoodieTestUtils.makeNewCommitTime(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); List records = new ArrayList<>(); @@ -437,7 +440,7 @@ private List testUpsertPartitioner(int smal HoodieClientTestUtils.fakeCommitFile(basePath, "001"); HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator( diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 42649159c86f3..5465879f030ed 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -19,6 +19,11 @@ package com.uber.hoodie.table; +import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieClientTestUtils; @@ -45,6 +50,14 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.compact.HoodieCompactor; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -61,25 +74,11 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - public class TestMergeOnReadTable { private transient JavaSparkContext jsc = null; private transient SQLContext sqlContext; - private String basePath = null; + private static String basePath = null; private HoodieCompactor compactor; private FileSystem fs; @@ -94,12 +93,11 @@ public static void cleanUp() throws Exception { if (hdfsTestService != null) { hdfsTestService.stop(); dfsCluster.shutdown(); - ; } FSUtils.setFs(null); // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM FileSystem.closeAll(); - HoodieTestUtils.resetFS(); + HoodieTestUtils.resetFS(basePath); } @BeforeClass @@ -113,30 +111,28 @@ public static void setUpDFS() throws IOException { dfs = dfsCluster.getFileSystem(); } FSUtils.setFs(dfs); - HoodieTestUtils.resetFS(); + HoodieTestUtils.resetFS(basePath); } @Before public void init() throws IOException { - this.fs = FSUtils.getFs(); - // Initialize a local spark env jsc = new JavaSparkContext( HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable")); - jsc.hadoopConfiguration().addResource(FSUtils.getFs().getConf()); // Create a temp folder as the base path TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + jsc.hadoopConfiguration().addResource(fs.getConf()); + dfs.mkdirs(new Path(basePath)); FSUtils.setFs(dfs); HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + sqlContext = new SQLContext(jsc); // SQLContext stuff compactor = new HoodieRealtimeTableCompactor(); - - //SQLContext stuff - sqlContext = new SQLContext(jsc); } @After @@ -167,7 +163,8 @@ public void testSimpleInsertAndUpdate() throws Exception { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); Optional deltaCommit = @@ -209,7 +206,7 @@ public void testSimpleInsertAndUpdate() throws Exception { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp()); @@ -229,8 +226,9 @@ public void testSimpleInsertAndUpdate() throws Exception { assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit - table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false)); + table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), + getConfig(false)); HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); @@ -284,7 +282,8 @@ public void testSimpleInsertAndDelete() throws Exception { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); Optional deltaCommit = @@ -332,7 +331,7 @@ public void testSimpleInsertAndDelete() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp()); @@ -349,7 +348,7 @@ public void testSimpleInsertAndDelete() throws Exception { List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) .collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles); + .getRecordsUsingInputFormat(dataFiles, basePath); //Wrote 40 records and deleted 20 records, so remaining 40-20 = 20 assertEquals("Must contain 20 records", 20, recordsRead.size()); } @@ -377,9 +376,8 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { //verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + cfg.getBasePath()); Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertTrue(commit.isPresent()); @@ -403,8 +401,9 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { //rollback a COW commit when TableType is MOR client.rollback(newCommitTime); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable + .getHoodieTable(metaClient, cfg); FileStatus[] allFiles = HoodieTestUtils .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, @@ -440,7 +439,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); Optional deltaCommit = @@ -476,7 +476,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp()); @@ -487,19 +487,19 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) .collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles); + .getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); // Test delta commit rollback client.rollback(newCommitTime); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); - recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles); + recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -516,13 +516,14 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { assertNoWriteErrors(statuses); HoodieCompactor compactor = new HoodieRealtimeTableCompactor(); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true)); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable table = HoodieTable + .getHoodieTable(metaClient, getConfig(true)); compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); @@ -541,7 +542,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { client.rollback(compactedCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index db973dfa7f37f..181fed895818d 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -140,5 +140,11 @@ + + com.github.stefanbirkner + system-rules + 1.16.0 + test + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java index b775d906876e2..b39d641746f8b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java @@ -77,7 +77,7 @@ private Object typeConvert(Object value, String name, Schema schema) throws IOEx switch (schema.getType()) { case BOOLEAN: if (value instanceof Boolean) { - return (Boolean) value; + return value; } break; case DOUBLE: diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java b/hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java new file mode 100644 index 0000000000000..01a1a6c09a862 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java @@ -0,0 +1,56 @@ +/* + * 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.common; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; + +public class SerializableConfiguration implements Serializable { + + private transient Configuration configuration; + + public SerializableConfiguration(Configuration configuration) { + this.configuration = configuration; + } + + public Configuration get() { + return configuration; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + configuration.write(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + configuration = new Configuration(false); + configuration.readFields(in); + } + + @Override + public String toString() { + StringBuilder str = new StringBuilder(); + configuration.iterator().forEachRemaining(e -> + str.append(String.format("%s => %s \n", e.getKey(), e.getValue()))); + return configuration.toString(); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java index 4a44276966412..96e5b97d0b063 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java @@ -17,5 +17,5 @@ package com.uber.hoodie.common.model; public enum ActionType { - commit, savepoint, compaction, clean, rollback; + commit, savepoint, compaction, clean, rollback } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index d012d9799f690..758d7b5dbf0db 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -16,6 +16,7 @@ package com.uber.hoodie.common.table; +import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; @@ -26,6 +27,7 @@ import java.io.Serializable; import java.util.Objects; import java.util.Properties; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,24 +54,28 @@ public class HoodieTableMetaClient implements Serializable { private String basePath; private transient FileSystem fs; private String metaPath; + private SerializableConfiguration hadoopConf; private HoodieTableType tableType; private HoodieTableConfig tableConfig; private HoodieActiveTimeline activeTimeline; private HoodieArchivedTimeline archivedTimeline; - public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException { + public HoodieTableMetaClient(Configuration conf, String basePath) + throws DatasetNotFoundException { // Do not load any timeline by default - this(fs, basePath, false); + this(conf, basePath, false); } - public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad) + public HoodieTableMetaClient(Configuration conf, String basePath, + boolean loadActiveTimelineOnLoad) throws DatasetNotFoundException { log.info("Loading HoodieTableMetaClient from " + basePath); this.basePath = basePath; - this.fs = fs; + this.hadoopConf = new SerializableConfiguration(conf); Path basePathDir = new Path(this.basePath); this.metaPath = basePath + File.separator + METAFOLDER_NAME; Path metaPathDir = new Path(this.metaPath); + this.fs = getFs(); DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir); this.tableConfig = new HoodieTableConfig(fs, metaPath); this.tableType = tableConfig.getTableType(); @@ -96,7 +102,7 @@ public HoodieTableMetaClient() { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); + fs = null; // will be lazily inited } private void writeObject(java.io.ObjectOutputStream out) @@ -136,9 +142,16 @@ public HoodieTableConfig getTableConfig() { * Get the FS implementation for this table */ public FileSystem getFs() { + if (fs == null) { + fs = FSUtils.getFs(metaPath, hadoopConf.get()); + } return fs; } + public Configuration getHadoopConf() { + return hadoopConf.get(); + } + /** * Get the active instants as a timeline * @@ -146,7 +159,7 @@ public FileSystem getFs() { */ public synchronized HoodieActiveTimeline getActiveTimeline() { if (activeTimeline == null) { - activeTimeline = new HoodieActiveTimeline(fs, metaPath); + activeTimeline = new HoodieActiveTimeline(this); } return activeTimeline; } @@ -159,7 +172,7 @@ public synchronized HoodieActiveTimeline getActiveTimeline() { */ public synchronized HoodieArchivedTimeline getArchivedTimeline() { if (archivedTimeline == null) { - archivedTimeline = new HoodieArchivedTimeline(fs, metaPath); + archivedTimeline = new HoodieArchivedTimeline(this); } return archivedTimeline; } @@ -196,7 +209,7 @@ public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs, fs.mkdirs(metaPathDir); } HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() + " from " + basePath); return metaClient; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 6671992337d8b..140d5d4fa9098 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -83,7 +83,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logMetadata) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index ae004991c1c7d..4c9f5c5232fc9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -37,7 +37,6 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -54,11 +53,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); - private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); - private String metaPath; - private transient FileSystem fs; - + private HoodieTableMetaClient metaClient; /** * Returns next commit time in the {@link #COMMIT_FORMATTER} format. @@ -67,12 +63,14 @@ public static String createNewCommitTime() { return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); } - protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) { + protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includedExtensions) { // Filter all the filter in the metapath and include only the extensions passed and // convert them into HoodieInstant try { this.instants = - Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> { + Arrays.stream( + HoodieTableMetaClient + .scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> { // Include only the meta files with extensions that needs to be included String extension = FSUtils.getFileExtension(path.getName()); return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); @@ -85,14 +83,13 @@ protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] included } catch (IOException e) { throw new HoodieIOException("Failed to scan metadata", e); } - this.fs = fs; - this.metaPath = metaPath; + this.metaClient = metaClient; // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function> & Serializable) this::getInstantDetails; } - public HoodieActiveTimeline(FileSystem fs, String metaPath) { - this(fs, metaPath, + public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { + this(metaClient, new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); @@ -114,7 +111,6 @@ public HoodieActiveTimeline() { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); } /** @@ -214,9 +210,9 @@ public void revertToInflight(HoodieInstant instant) { public void deleteInflight(HoodieInstant instant) { log.info("Deleting in-flight " + instant); - Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName()); + Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName()); try { - boolean result = fs.delete(inFlightCommitFilePath, false); + boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false); if (result) { log.info("Removed in-flight " + instant); } else { @@ -230,18 +226,18 @@ public void deleteInflight(HoodieInstant instant) { @Override public Optional getInstantDetails(HoodieInstant instant) { - Path detailPath = new Path(metaPath, instant.getFileName()); + Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName()); return readDataFromPath(detailPath); } protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed, Optional data) { - Path commitFilePath = new Path(metaPath, completed.getFileName()); + Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); try { // open a new file and write the commit metadata in - Path inflightCommitFile = new Path(metaPath, inflight.getFileName()); + Path inflightCommitFile = new Path(metaClient.getMetaPath(), inflight.getFileName()); createFileInMetaPath(inflight.getFileName(), data); - boolean success = fs.rename(inflightCommitFile, commitFilePath); + boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath); if (!success) { throw new HoodieIOException( "Could not rename " + inflightCommitFile + " to " + commitFilePath); @@ -252,11 +248,11 @@ protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant comp } protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { - Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName()); + Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName()); try { - if (!fs.exists(inFlightCommitFilePath)) { - Path commitFilePath = new Path(metaPath, completed.getFileName()); - boolean success = fs.rename(commitFilePath, inFlightCommitFilePath); + if (!metaClient.getFs().exists(inFlightCommitFilePath)) { + Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); + boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath); if (!success) { throw new HoodieIOException( "Could not rename " + commitFilePath + " to " + inFlightCommitFilePath); @@ -272,15 +268,15 @@ public void saveToInflight(HoodieInstant instant, Optional content) { } protected void createFileInMetaPath(String filename, Optional content) { - Path fullPath = new Path(metaPath, filename); + Path fullPath = new Path(metaClient.getMetaPath(), filename); try { if (!content.isPresent()) { - if (fs.createNewFile(fullPath)) { + if (metaClient.getFs().createNewFile(fullPath)) { log.info("Created a new file in meta path: " + fullPath); return; } } else { - FSDataOutputStream fsout = fs.create(fullPath, true); + FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true); fsout.write(content.get()); fsout.close(); return; @@ -292,7 +288,7 @@ protected void createFileInMetaPath(String filename, Optional content) { } protected Optional readDataFromPath(Path detailPath) { - try (FSDataInputStream is = fs.open(detailPath)) { + try (FSDataInputStream is = metaClient.getFs().open(detailPath)) { return Optional.of(IOUtils.toByteArray(is)); } catch (IOException e) { throw new HoodieIOException("Could not read commit details from " + detailPath, e); @@ -300,6 +296,6 @@ protected Optional readDataFromPath(Path detailPath) { } public HoodieActiveTimeline reload() { - return new HoodieActiveTimeline(fs, metaPath); + return new HoodieActiveTimeline(metaClient); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java index 37e5e94145341..bc04873afa6d0 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java @@ -16,8 +16,8 @@ package com.uber.hoodie.common.table.timeline; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; @@ -27,7 +27,6 @@ import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; @@ -44,17 +43,17 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits"; - private transient FileSystem fs; - private String metaPath; + private HoodieTableMetaClient metaClient; private Map readCommits = new HashMap<>(); private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); - public HoodieArchivedTimeline(FileSystem fs, String metaPath) { + public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { // Read back the commits to make sure - Path archiveLogPath = getArchiveLogPath(metaPath); + Path archiveLogPath = getArchiveLogPath(metaClient.getMetaPath()); try (SequenceFile.Reader reader = - new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath))) { + new SequenceFile.Reader(metaClient.getHadoopConf(), + SequenceFile.Reader.file(archiveLogPath))) { Text key = new Text(); Text val = new Text(); while (reader.next(key, val)) { @@ -71,8 +70,7 @@ public HoodieArchivedTimeline(FileSystem fs, String metaPath) { } // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function> & Serializable) this::getInstantDetails; - this.fs = fs; - this.metaPath = metaPath; + this.metaClient = metaClient; } /** @@ -91,7 +89,6 @@ public HoodieArchivedTimeline() { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); } @@ -105,7 +102,7 @@ public Optional getInstantDetails(HoodieInstant instant) { } public HoodieArchivedTimeline reload() { - return new HoodieArchivedTimeline(fs, metaPath); + return new HoodieArchivedTimeline(metaClient); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index afd2c89dc36fb..60df7f4715b38 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -40,7 +40,6 @@ import java.util.stream.Stream; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; /** @@ -57,7 +56,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.RealtimeView, Serializable { protected HoodieTableMetaClient metaClient; - protected transient FileSystem fs; // This is the commits that will be visible for all views extending this view protected HoodieTimeline visibleActiveTimeline; @@ -72,7 +70,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView, public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; - this.fs = metaClient.getFs(); this.visibleActiveTimeline = visibleActiveTimeline; this.fileGroupMap = new HashMap<>(); this.partitionToFileGroupsMap = new HashMap<>(); @@ -98,7 +95,6 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); } private void writeObject(java.io.ObjectOutputStream out) @@ -255,8 +251,8 @@ public Stream getAllFileGroups(String partitionPathStr) { try { // Create the path if it does not exist already Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); - FSUtils.createPathIfNotExists(fs, partitionPath); - FileStatus[] statuses = fs.listStatus(partitionPath); + FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); + FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); List fileGroups = addFilesToView(statuses); return fileGroups.stream(); } catch (IOException e) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java index ae0dbd3f004a3..633960d0a3a2a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java @@ -94,7 +94,7 @@ public static List> loadFromFile(FileSystem fs, public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime, Optional durationInMs, List cleanStats) { ImmutableMap.Builder partitionMetadataBuilder = - ImmutableMap.builder(); + ImmutableMap.builder(); int totalDeleted = 0; String earliestCommitToRetain = null; for (HoodieCleanStat stat : cleanStats) { @@ -116,7 +116,7 @@ public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime, public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime, Optional durationInMs, List commits, List stats) { ImmutableMap.Builder partitionMetadataBuilder = - ImmutableMap.builder(); + ImmutableMap.builder(); int totalDeleted = 0; for (HoodieRollbackStat stat : stats) { HoodieRollbackPartitionMetadata metadata = @@ -132,7 +132,7 @@ public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbac public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment, Map> latestFiles) { ImmutableMap.Builder partitionMetadataBuilder = - ImmutableMap.builder(); + ImmutableMap.builder(); for (Map.Entry> stat : latestFiles.entrySet()) { HoodieSavepointPartitionMetadata metadata = new HoodieSavepointPartitionMetadata(stat.getKey(), stat.getValue()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index daecf6237df63..d788cdd4435ca 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -29,6 +29,7 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; +import java.util.Map.Entry; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -55,6 +56,7 @@ public class FSUtils { private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; private static final long MIN_CLEAN_TO_KEEP = 10; private static final long MIN_ROLLBACK_TO_KEEP = 10; + private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_"; private static FileSystem fs; /** @@ -65,17 +67,32 @@ public static void setFs(FileSystem fs) { FSUtils.fs = fs; } + public static Configuration prepareHadoopConf(Configuration conf) { + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + + // look for all properties, prefixed to be picked up + for (Entry prop : System.getenv().entrySet()) { + if (prop.getKey().startsWith(HOODIE_ENV_PROPS_PREFIX)) { + LOG.info("Picking up value for hoodie env var :" + prop.getKey()); + conf.set(prop.getKey() + .replace(HOODIE_ENV_PROPS_PREFIX, "") + .replaceAll("_DOT_", "."), + prop.getValue()); + } + } + return conf; + } + - public static FileSystem getFs() { + public static FileSystem getFs(String path, Configuration conf) { if (fs != null) { return fs; } - Configuration conf = new Configuration(); - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); FileSystem fs; + conf = prepareHadoopConf(conf); try { - fs = FileSystem.get(conf); + fs = new Path(path).getFileSystem(conf); } catch (IOException e) { throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(), e); @@ -83,7 +100,6 @@ public static FileSystem getFs() { LOG.info( String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]", conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString())); - return fs; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index a4a683350e7a6..ecbaad1c96980 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -51,10 +51,11 @@ public class ParquetUtils { * Read the rowKey list from the given parquet file. * * @param filePath The parquet file path. + * @param configuration configuration to build fs object */ - public static Set readRowKeysFromParquet(Path filePath) { - Configuration conf = new Configuration(); - conf.addResource(getFs().getConf()); + public static Set readRowKeysFromParquet(Configuration configuration, Path filePath) { + Configuration conf = new Configuration(configuration); + conf.addResource(getFs(filePath.toString(), conf).getConf()); Schema readSchema = HoodieAvroUtils.getRecordKeySchema(); AvroReadSupport.setAvroReadSchema(conf, readSchema); AvroReadSupport.setRequestedProjection(conf, readSchema); @@ -84,19 +85,12 @@ public static Set readRowKeysFromParquet(Path filePath) { return rowKeys; } - - /** - * Read the metadata from a parquet file - */ - public static ParquetMetadata readMetadata(Path parquetFilePath) { - return readMetadata(new Configuration(), parquetFilePath); - } - public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) { ParquetMetadata footer; try { // TODO(vc): Should we use the parallel reading version here? - footer = ParquetFileReader.readFooter(getFs().getConf(), parquetFilePath); + footer = ParquetFileReader + .readFooter(getFs(parquetFilePath.toString(), conf).getConf(), parquetFilePath); } catch (IOException e) { throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); @@ -108,14 +102,15 @@ public static ParquetMetadata readMetadata(Configuration conf, Path parquetFileP /** * Get the schema of the given parquet file. */ - public static MessageType readSchema(Path parquetFilePath) { - return readMetadata(parquetFilePath).getFileMetaData().getSchema(); + public static MessageType readSchema(Configuration configuration, Path parquetFilePath) { + return readMetadata(configuration, parquetFilePath).getFileMetaData().getSchema(); } - private static List readParquetFooter(Path parquetFilePath, String... footerNames) { + private static List readParquetFooter(Configuration configuration, Path parquetFilePath, + String... footerNames) { List footerVals = new ArrayList<>(); - ParquetMetadata footer = readMetadata(parquetFilePath); + ParquetMetadata footer = readMetadata(configuration, parquetFilePath); Map metadata = footer.getFileMetaData().getKeyValueMetaData(); for (String footerName : footerNames) { if (metadata.containsKey(footerName)) { @@ -128,21 +123,22 @@ private static List readParquetFooter(Path parquetFilePath, String... fo return footerVals; } - public static Schema readAvroSchema(Path parquetFilePath) { - return new AvroSchemaConverter().convert(readSchema(parquetFilePath)); + public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) { + return new AvroSchemaConverter().convert(readSchema(configuration, parquetFilePath)); } /** * Read out the bloom filter from the parquet file meta data. */ - public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) { - String footerVal = readParquetFooter(parquetFilePath, + public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, + Path parquetFilePath) { + String footerVal = readParquetFooter(configuration, parquetFilePath, HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0); return new BloomFilter(footerVal); } - public static String[] readMinMaxRecordKeys(Path parquetFilePath) { - List minMaxKeys = readParquetFooter(parquetFilePath, + public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) { + List minMaxKeys = readParquetFooter(configuration, parquetFilePath, HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER, HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER); if (minMaxKeys.size() != 2) { @@ -156,11 +152,11 @@ public static String[] readMinMaxRecordKeys(Path parquetFilePath) { /** * NOTE: This literally reads the entire file contents, thus should be used with caution. */ - public static List readAvroRecords(Path filePath) { + public static List readAvroRecords(Configuration configuration, Path filePath) { ParquetReader reader = null; List records = new ArrayList<>(); try { - reader = AvroParquetReader.builder(filePath).build(); + reader = AvroParquetReader.builder(filePath).withConf(configuration).build(); Object obj = reader.read(); while (obj != null) { if (obj instanceof GenericRecord) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java index 0b3a6bdebfee0..74cc1104b079d 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import com.google.common.io.Files; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.io.File; import java.io.IOException; import org.apache.commons.io.FileUtils; @@ -54,7 +55,7 @@ public class HdfsTestService { private MiniDFSCluster miniDfsCluster; public HdfsTestService() { - hadoopConf = new Configuration(); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); workDir = Files.createTempDir().getAbsolutePath(); } @@ -67,7 +68,7 @@ public MiniDFSCluster start(boolean format) throws IOException { .checkState(workDir != null, "The work dir must be set before starting cluster."); if (hadoopConf == null) { - hadoopConf = new Configuration(); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); } // If clean, then remove the work dir so we can start fresh. diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 9489e57ab3664..7017bbf91b2bd 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -58,6 +58,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -69,18 +70,23 @@ public class HoodieTestUtils { - public static FileSystem fs = FSUtils.getFs(); + public static FileSystem fs; public static final String TEST_EXTENSION = ".test"; public static final String RAW_TRIPS_TEST_NAME = "raw_trips"; public static final int DEFAULT_TASK_PARTITIONID = 1; public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; private static Random rand = new Random(46474747); - public static void resetFS() { - HoodieTestUtils.fs = FSUtils.getFs(); + public static void resetFS(String basePath) { + HoodieTestUtils.fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + } + + public static Configuration getDefaultHadoopConf() { + return new Configuration(); } public static HoodieTableMetaClient init(String basePath) throws IOException { + fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); return initTableType(basePath, HoodieTableType.COPY_ON_WRITE); } @@ -211,7 +217,7 @@ public static void createCleanFiles(String basePath, String commitTime) throws I Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline .makeCleanerFileName(commitTime)); - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); FSDataOutputStream os = fs.create(commitFile, true); try { HoodieCleanStat cleanStats = new HoodieCleanStat( diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 8fc7fb46f8a64..084cc1f121a81 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -67,7 +67,7 @@ public class HoodieLogFormatTest { private FileSystem fs; private Path partitionPath; - private String basePath; + private static String basePath; @BeforeClass public static void setUpClass() throws IOException, InterruptedException { @@ -78,7 +78,7 @@ public static void setUpClass() throws IOException, InterruptedException { @AfterClass public static void tearDownClass() { MiniClusterUtil.shutdown(); - HoodieTestUtils.resetFS(); + HoodieTestUtils.resetFS(basePath); } @Before @@ -343,7 +343,7 @@ public void testAppendAndReadOnCorruptedLog() writer.close(); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) - fs = FileSystem.get(fs.getConf()); + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); @@ -533,7 +533,7 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() writer.close(); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) - fs = FileSystem.get(fs.getConf()); + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 22285a6c51b14..189be698dcda0 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -74,7 +74,7 @@ public void testLoadingInstantsFromFiles() throws IOException { HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); - timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath()); + timeline = new HoodieActiveTimeline(metaClient); timeline.saveAsComplete(instant1, Optional.empty()); timeline.saveAsComplete(instant2, Optional.empty()); timeline.saveAsComplete(instant3, Optional.empty()); @@ -98,7 +98,7 @@ public void testLoadingInstantsFromFiles() throws IOException { @Test public void testTimelineOperationsBasic() throws Exception { - timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath()); + timeline = new HoodieActiveTimeline(metaClient); assertTrue(timeline.empty()); assertEquals("", 0, timeline.countInstants()); assertEquals("", Optional.empty(), timeline.firstInstant()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 992b864162b6d..68d32215a6ebd 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -69,7 +69,7 @@ public void init() throws IOException { } private void refreshFsView(FileStatus[] statuses) { - metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true); + metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs.getConf(), basePath, true); if (statuses != null) { fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java index edcc1509b25c3..43b0ebdf32ea2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java @@ -16,15 +16,24 @@ package com.uber.hoodie.common.util; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.text.SimpleDateFormat; import java.util.Date; import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.junit.Rule; import org.junit.Test; +import org.junit.contrib.java.lang.system.EnvironmentVariables; public class TestFSUtils { + @Rule + public final EnvironmentVariables environmentVariables + = new EnvironmentVariables(); + @Test public void testMakeDataFileName() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); @@ -59,4 +68,15 @@ public void testGetFileNameWithoutMeta() { String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName); assertTrue(FSUtils.getFileId(fullFileName).equals(fileName)); } + + @Test + public void testEnvVarVariablesPickedup() { + environmentVariables.set("HOODIE_ENV_fs_DOT_key1", "value1"); + Configuration conf = FSUtils.prepareHadoopConf(HoodieTestUtils.getDefaultHadoopConf()); + assertEquals("value1", conf.get("fs.key1")); + conf.set("fs.key1", "value11"); + conf.set("fs.key2", "value2"); + assertEquals("value11", conf.get("fs.key1")); + assertEquals("value2", conf.get("fs.key2")); + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java index 266cb1158cf3d..fd0a4475a13b1 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java @@ -22,6 +22,7 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -79,12 +80,15 @@ public void testHoodieWriteSupport() throws Exception { // Read and verify List rowKeysInFile = new ArrayList<>( - ParquetUtils.readRowKeysFromParquet(new Path(filePath))); + ParquetUtils + .readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); Collections.sort(rowKeysInFile); Collections.sort(rowKeys); assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile); - BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath)); + BloomFilter filterInFile = ParquetUtils + .readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), + new Path(filePath)); for (String rowKey : rowKeys) { assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index ee4f5a9544286..302373e1beb92 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -295,6 +295,6 @@ protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path da } Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels); LOG.info("Reading hoodie metadata from path " + baseDir.toString()); - return new HoodieTableMetaClient(fs, baseDir.toString()); + return new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index c8ffbcc9a6b2a..2c01ed666a220 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -19,6 +19,7 @@ import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieException; import java.io.Serializable; @@ -86,7 +87,7 @@ public boolean accept(Path path) { } Path folder = null; try { - FileSystem fs = path.getFileSystem(new Configuration()); + FileSystem fs = path.getFileSystem(FSUtils.prepareHadoopConf(new Configuration())); if (fs.isDirectory(path)) { return true; } @@ -123,7 +124,7 @@ public boolean accept(Path path) { if (baseDir != null) { try { HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs, baseDir.toString()); + new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline() .filterCompletedInstants(), diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 00ef57e42d287..35ef39eda8ca4 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -84,7 +84,7 @@ public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); try { baseFileSchema = readSchema(jobConf, split.getPath()); - readAndCompactLog(); + readAndCompactLog(jobConf); } catch (IOException e) { throw new HoodieIOException( "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); @@ -110,7 +110,7 @@ private static MessageType readSchema(Configuration conf, Path parquetFilePath) * Goes through the log files and populates a map with latest version of each key logged, since * the base split was written. */ - private void readAndCompactLog() throws IOException { + private void readAndCompactLog(JobConf jobConf) throws IOException { Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema); List projectionFields = orderFields( jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), @@ -123,7 +123,8 @@ private void readAndCompactLog() throws IOException { String.format("About to read compacted logs %s for base split %s, projecting cols %s", split.getDeltaFilePaths(), split.getPath(), projectionFields)); HoodieCompactedLogRecordScanner compactedLogRecordScanner = - new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getBasePath(), + new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), + split.getBasePath(), split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime()); // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 84f02b8688fd0..85e4eaca3a877 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -41,6 +41,7 @@ import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; @@ -64,10 +65,13 @@ public class HoodieRealtimeRecordReaderTest { private JobConf jobConf; + private FileSystem fs; @Before public void setUp() { jobConf = new JobConf(); + fs = FSUtils + .getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf()); } @Rule @@ -79,7 +83,9 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) - .overBaseCommit(baseCommit).withFs(FSUtils.getFs()).build(); + .overBaseCommit(baseCommit) + .withFs(fs) + .build(); List records = new ArrayList<>(); for (int i = 0; i < numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); @@ -123,8 +129,8 @@ public void testReader() throws Exception { //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, - FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null); + getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), + (String[]) null), jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); @@ -183,8 +189,8 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, - FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null); + getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), + (String[]) null), jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java index 088e24a9c962b..472001ede9e69 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Partition; @@ -183,7 +184,7 @@ public static void main(String[] args) throws Exception { cmd.usage(); System.exit(1); } - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); HiveConf hiveConf = new HiveConf(); hiveConf.addResource(fs.getConf()); new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index dede5e5f5d2d8..95d1d58212a0c 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -91,7 +91,7 @@ public class HoodieHiveClient { HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { this.syncConfig = cfg; this.fs = fs; - this.metaClient = new HoodieTableMetaClient(fs, cfg.basePath, true); + this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true); this.tableType = metaClient.getTableType(); LOG.info("Creating hive connection " + cfg.jdbcUrl); diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java index 26ed1b0f95e50..7b9172ead70bd 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.io.Files; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; @@ -87,7 +88,7 @@ public HiveServer2 start() throws IOException { .checkState(workDir != null, "The work dir must be set before starting cluster."); if (hadoopConf == null) { - hadoopConf = new Configuration(); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); } String localHiveLocation = getHiveLocation(workDir); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java index b74d917071ed2..996786a7d4bed 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -67,7 +67,7 @@ public static String latestCommit(FileSystem fs, String basePath) { */ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); + .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { return table.getActiveTimeline().getTimelineOfActions( Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index 4aca81f1725b7..cf0a2c1aa8082 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -47,7 +47,7 @@ class IncrementalRelation(val sqlContext: SQLContext, private val log = LogManager.getLogger(classOf[IncrementalRelation]) val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - val metaClient = new HoodieTableMetaClient(fs, basePath, true) + val metaClient = new HoodieTableMetaClient(sqlContext.sparkContext.hadoopConfiguration, basePath, true) // MOR datasets not supported yet if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") @@ -72,7 +72,8 @@ class IncrementalRelation(val sqlContext: SQLContext, val latestMeta = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get) val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next() - AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema(new Path(metaFilePath))) + AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema( + sqlContext.sparkContext.hadoopConfiguration, new Path(metaFilePath))) } override def schema: StructType = latestSchema diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index b9fed1cfe7db4..1bdc92777c1a1 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -57,7 +57,7 @@ class DataSourceTest extends AssertionsForJUnit { val folder = new TemporaryFolder folder.create basePath = folder.getRoot.getAbsolutePath - fs = FSUtils.getFs + fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration) } @Test def testCopyOnWriteStorage() { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 27f2649740b06..1d3a2b1c10eae 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -56,21 +56,18 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; import scala.Tuple2; public class HDFSParquetImporter implements Serializable { private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class); private final Config cfg; - private final transient FileSystem fs; + private transient FileSystem fs; public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd"); public HDFSParquetImporter( Config cfg) throws IOException { this.cfg = cfg; - fs = FSUtils.getFs(); } public static class FormatValidator implements IValueValidator { @@ -203,6 +200,7 @@ private String getSchema() throws Exception { } public int dataImport(JavaSparkContext jsc, int retry) throws Exception { + this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration()); int ret = -1; try { // Verify that targetPath is not present. @@ -251,43 +249,36 @@ protected int dataImport(JavaSparkContext jsc) throws IOException { GenericRecord.class, job.getConfiguration()) // To reduce large number of tasks. .coalesce(16 * cfg.parallelism) - .map(new Function, HoodieRecord>() { - @Override - public HoodieRecord call(Tuple2 entry) - throws Exception { - GenericRecord genericRecord = entry._2(); - Object partitionField = genericRecord.get(cfg.partitionKey); - if (partitionField == null) { - throw new HoodieIOException( - "partition key is missing. :" + cfg.partitionKey); - } - Object rowField = genericRecord.get(cfg.rowKey); - if (rowField == null) { - throw new HoodieIOException( - "row field is missing. :" + cfg.rowKey); - } - long ts = (long) ((Double) partitionField * 1000l); - String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); - return new HoodieRecord( - new HoodieKey((String) rowField, partitionPath), - new HoodieJsonPayload(genericRecord.toString())); - } - } + .map(entry -> { + GenericRecord genericRecord = ((Tuple2) entry)._2(); + Object partitionField = genericRecord.get(cfg.partitionKey); + if (partitionField == null) { + throw new HoodieIOException( + "partition key is missing. :" + cfg.partitionKey); + } + Object rowField = genericRecord.get(cfg.rowKey); + if (rowField == null) { + throw new HoodieIOException( + "row field is missing. :" + cfg.rowKey); + } + long ts = (long) ((Double) partitionField * 1000l); + String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); + return new HoodieRecord<>( + new HoodieKey((String) rowField, partitionPath), + new HoodieJsonPayload(genericRecord.toString())); + } ); // Get commit time. String commitTime = client.startCommit(); JavaRDD writeResponse = client.bulkInsert(hoodieRecords, commitTime); Accumulator errors = jsc.accumulator(0); - writeResponse.foreach(new VoidFunction() { - @Override - public void call(WriteStatus writeStatus) throws Exception { + writeResponse.foreach(writeStatus -> { if (writeStatus.hasErrors()) { errors.add(1); logger.error(String.format("Error processing records :writeStatus:%s", writeStatus.getStat().toString())); } - } }); if (errors.value() == 0) { logger.info(String diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index 5bcba8cb8361b..a6fbc3f363cfb 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -291,7 +291,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) { return "0"; } - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath); Optional lastCommit = metadata.getActiveTimeline().getCommitsTimeline() @@ -331,7 +331,7 @@ private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException { - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants() .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants() diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index 3d199e0f52593..41ef772769288 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -20,6 +20,7 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; +import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.table.HoodieTableConfig; @@ -70,8 +71,10 @@ static class Config implements Serializable { public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, final boolean shouldAssumeDatePartitioning) throws IOException { - FileSystem fs = FSUtils.getFs(); - final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir); + FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration()); + final SerializableConfiguration serConf = new SerializableConfiguration( + jsc.hadoopConfiguration()); + final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView( tableMetadata, tableMetadata.getActiveTimeline().getCommitsTimeline() @@ -104,7 +107,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi jsc.parallelize(partitions, partitions.size()) .flatMap(partition -> { // Only take latest version files <= latestCommit. - FileSystem fs1 = FSUtils.getFs(); + FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get()); List> filePaths = new ArrayList<>(); Stream dataFiles = fsView .getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp); @@ -123,13 +126,13 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi String partition = tuple._1(); Path sourceFilePath = new Path(tuple._2()); Path toPartitionPath = new Path(outputDir, partition); - FileSystem fs1 = FSUtils.getFs(); + FileSystem ifs = FSUtils.getFs(baseDir, serConf.get()); - if (!fs1.exists(toPartitionPath)) { - fs1.mkdirs(toPartitionPath); + if (!ifs.exists(toPartitionPath)) { + ifs.mkdirs(toPartitionPath); } - FileUtil.copy(fs1, sourceFilePath, fs1, - new Path(toPartitionPath, sourceFilePath.getName()), false, fs1.getConf()); + FileUtil.copy(ifs, sourceFilePath, ifs, + new Path(toPartitionPath, sourceFilePath.getName()), false, ifs.getConf()); }); // Also copy the .commit files diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index 2baafd036382f..6a63c33fc84ed 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -115,10 +115,11 @@ public class HoodieDeltaStreamer implements Serializable { public HoodieDeltaStreamer(Config cfg) throws IOException { this.cfg = cfg; - this.fs = FSUtils.getFs(); + this.jssc = getSparkContext(); + this.fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); if (fs.exists(new Path(cfg.targetBasePath))) { - HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath); + HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), cfg.targetBasePath); this.commitTimelineOpt = Optional .of(meta.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants()); @@ -129,8 +130,6 @@ public HoodieDeltaStreamer(Config cfg) throws IOException { //TODO(vc) Should these be passed from outside? initSchemaProvider(); initKeyGenerator(); - this.jssc = getSparkContext(); - initSource(); } @@ -203,7 +202,9 @@ private void sync() throws Exception { Properties properties = new Properties(); properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName); HoodieTableMetaClient - .initializePathAsHoodieDataset(FSUtils.getFs(), cfg.targetBasePath, properties); + .initializePathAsHoodieDataset( + FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), cfg.targetBasePath, + properties); } log.info("Checkpoint to resume from : " + resumeCheckpointStr); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java index f6ea67f0155d6..44745b093fd2e 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java @@ -25,6 +25,7 @@ import java.util.Arrays; import org.apache.avro.Schema; import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -50,7 +51,7 @@ static class Config { public FilebasedSchemaProvider(PropertiesConfiguration config) { super(config); - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(config.getBasePath(), new Configuration()); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP)); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java index 128a449a43e98..f9c9a9a1dbd63 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java @@ -65,7 +65,7 @@ static class Config { public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) { super(config, sparkContext, dataFormat, schemaProvider); - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(config.getBasePath(), sparkContext.hadoopConfiguration()); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index 08c9193663576..aeecb9db014ca 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -72,7 +72,7 @@ static class Config { public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) { super(config, sparkContext, dataFormat, schemaProvider); - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(config.getBasePath(), sparkContext.hadoopConfiguration()); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java index 2f2941e5d675b..36c43c5968988 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java @@ -48,7 +48,6 @@ import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.OffsetRange; import scala.Predef; -import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.immutable.Map; import scala.collection.immutable.Set; @@ -134,16 +133,16 @@ static class ScalaHelpers { public static Map toScalaMap(HashMap m) { return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap( - Predef.>conforms() + Predef.conforms() ); } public static Set toScalaSet(HashSet s) { - return JavaConverters.asScalaSetConverter(s).asScala().toSet(); + return JavaConverters.asScalaSetConverter(s).asScala().toSet(); } public static java.util.Map toJavaMap(Map m) { - return JavaConverters.mapAsJavaMapConverter(m).asJava(); + return JavaConverters.mapAsJavaMapConverter(m).asJava(); } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java index af0a5233078a3..c1eaa86aae2f4 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java @@ -24,6 +24,7 @@ import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.minicluster.HdfsTestService; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.FSUtils; @@ -38,7 +39,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -174,7 +174,7 @@ private void createRecords(Path srcFolder) throws ParseException, IOException { ParquetWriter writer = AvroParquetWriter .builder(srcFile) .withSchema(HoodieTestDataGenerator.avroSchema) - .withConf(new Configuration()) + .withConf(HoodieTestUtils.getDefaultHadoopConf()) .build(); for (GenericRecord record : records) { writer.write(record); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java index 6f9acc4892fc6..db670673f6047 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java @@ -44,17 +44,22 @@ public class TestHoodieSnapshotCopier { @Before public void init() throws IOException { - // Prepare directories - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - rootPath = folder.getRoot().getAbsolutePath(); - basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME; - HoodieTestUtils.init(basePath); - outputPath = rootPath + "/output"; - fs = FSUtils.getFs(); - // Start a local Spark job - SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); - jsc = new JavaSparkContext(conf); + try { + // Prepare directories + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + rootPath = "file://" + folder.getRoot().getAbsolutePath(); + basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME; + outputPath = rootPath + "/output"; + + fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + HoodieTestUtils.init(basePath); + // Start a local Spark job + SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); + jsc = new JavaSparkContext(conf); + } catch (Exception e) { + e.printStackTrace(); + } } @Test From dd23f564930bb48ce70d101e741110ed414dff74 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Wed, 3 Jan 2018 04:32:21 -0800 Subject: [PATCH 004/374] Nicer handling of timeline archival for Cloud storage - When append() is not supported, rollover to new file always (instead of failing) - Provide way to configure archive log folder (avoids small files inside .hoodie) - Datasets written via Spark datasource archive to .hoodie/archived - HoodieClientExample will now retain only 2,3 commits to exercise archival path during dev cycles - Few tweaks to code structure around CommitArchiveLog --- .../com/uber/hoodie/HoodieWriteClient.java | 4 +- .../hoodie/io/HoodieCommitArchiveLog.java | 27 +++------- .../src/test/java/HoodieClientExample.java | 6 ++- .../hoodie/io/TestHoodieCommitArchiveLog.java | 54 +++++++++---------- .../common/table/HoodieTableConfig.java | 11 ++++ .../common/table/HoodieTableMetaClient.java | 24 +++++++++ .../table/log/HoodieLogFormatWriter.java | 11 +++- .../timeline/HoodieArchivedTimeline.java | 6 +-- .../table/HoodieTableMetaClientTest.java | 2 +- .../common/table/log/HoodieLogFormatTest.java | 30 +++++++++++ .../HoodieRealtimeRecordReaderTest.java | 1 + .../scala/com/uber/hoodie/DefaultSource.scala | 1 + 12 files changed, 121 insertions(+), 56 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index cfd20b4e15f21..6b0adf67fbb74 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -91,7 +91,6 @@ public class HoodieWriteClient implements Seriali private final HoodieWriteConfig config; private transient final HoodieMetrics metrics; private transient final HoodieIndex index; - private transient final HoodieCommitArchiveLog archiveLog; private transient Timer.Context writeContext = null; /** @@ -116,7 +115,6 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, this.config = clientConfig; this.index = HoodieIndex.createIndex(config, jsc); this.metrics = new HoodieMetrics(config, config.getTableName()); - this.archiveLog = new HoodieCommitArchiveLog(clientConfig, fs); if (rollbackInFlight) { rollbackInflightCommits(); @@ -446,6 +444,8 @@ public boolean commit(String commitTime, } // We cannot have unbounded commit files. Archive commits if we have to archive + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); archiveLog.archiveIfRequired(); if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 7704517c629e7..bb295b64d2b51 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -40,7 +40,6 @@ import com.uber.hoodie.table.HoodieTable; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -60,15 +59,14 @@ public class HoodieCommitArchiveLog { private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class); private final Path archiveFilePath; - private final FileSystem fs; + private final HoodieTableMetaClient metaClient; private final HoodieWriteConfig config; private HoodieLogFormat.Writer writer; - public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) { - this.fs = fs; + public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { this.config = config; - this.archiveFilePath = HoodieArchivedTimeline - .getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME); + this.metaClient = metaClient; + this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); } private HoodieLogFormat.Writer openWriter() { @@ -78,7 +76,7 @@ private HoodieLogFormat.Writer openWriter() { .onParentPath(archiveFilePath.getParent()) .withFileId(archiveFilePath.getName()) .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) - .withFs(fs) + .withFs(metaClient.getFs()) .overBaseCommit("").build(); } else { return this.writer; @@ -125,9 +123,7 @@ private Stream getInstantsToArchive() { int maxCommitsToKeep = config.getMaxCommitsToKeep(); int minCommitsToKeep = config.getMinCommitsToKeep(); - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); // GroupBy each action and limit each action timeline to maxCommitsToKeep HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() @@ -165,16 +161,13 @@ private Stream getInstantsToArchive() { private boolean deleteArchivedInstants(List archivedInstants) { log.info("Deleting instants " + archivedInstants); - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true); - boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { - if (fs.exists(commitFile)) { - success &= fs.delete(commitFile, false); + if (metaClient.getFs().exists(commitFile)) { + success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { @@ -186,13 +179,9 @@ private boolean deleteArchivedInstants(List archivedInstants) { } public void archive(List instants) throws HoodieCommitException { - try { - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true); HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); - Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); log.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index f1fc056e1dd10..14c1e6eac239f 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -24,9 +24,10 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.index.HoodieIndex.IndexType; import java.util.List; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -92,7 +93,8 @@ public void run() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()) .build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index 430a0a591177a..5d8362688cd4b 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -63,7 +63,8 @@ public void testArchiveEmptyDataset() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").build(); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, + new HoodieTableMetaClient(fs.getConf(), cfg.getBasePath(), true)); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); } @@ -82,9 +83,9 @@ public void testArchiveDatasetWithArchival() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); @@ -96,18 +97,19 @@ public void testArchiveDatasetWithArchival() throws IOException { HoodieTestUtils.createCleanFiles(basePath, "105"); //reload the timeline and get all the commmits before archive - timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline() + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() .filterCompletedInstants(); List originalCommits = timeline.getInstants().collect(Collectors.toList()); assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants()); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, + new HoodieTableMetaClient(fs.getConf(), basePath, true)); assertTrue(archiveLog.archiveIfRequired()); //reload the timeline and remove the remaining commits - timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline() + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() .filterCompletedInstants(); originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); @@ -147,22 +149,20 @@ public void testArchiveDatasetWithNoArchival() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "102"); HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = - metadata.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); } @@ -173,8 +173,8 @@ public void testArchiveCommitSafety() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "102"); @@ -182,14 +182,13 @@ public void testArchiveCommitSafety() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = - metadata.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() + .filterCompletedInstants(); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); assertTrue("Archived commits should always be safe", @@ -206,8 +205,8 @@ public void testArchiveCommitSavepointNoHole() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createSavepointFile(basePath, "101"); @@ -216,14 +215,13 @@ public void testArchiveCommitSavepointNoHole() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = - metadata.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() + .filterCompletedInstants(); assertEquals( "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)", 5, timeline.countInstants()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 8cc6c18c6dd5a..577d7cf4e346c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -52,11 +52,13 @@ public class HoodieTableConfig implements Serializable { public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format"; public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class"; + public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder"; public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE; public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET; public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG; public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName(); + public static final String DEFAULT_ARCHIVELOG_FOLDER = ""; private Properties props; public HoodieTableConfig(FileSystem fs, String metaPath) { @@ -105,6 +107,9 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, && !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) { properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS); } + if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) { + properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); + } properties .store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } finally { @@ -161,4 +166,10 @@ public HoodieFileFormat getRTFileFormat() { return DEFAULT_RT_FILE_FORMAT; } + /** + * Get the relative path of archive log folder under metafolder, for this dataset + */ + public String getArchivelogFolder() { + return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 758d7b5dbf0db..841cf47ab5e2f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -131,6 +131,18 @@ public String getMetaPath() { return metaPath; } + /** + * @return path where archived timeline is stored + */ + public String getArchivePath() { + String archiveFolder = tableConfig.getArchivelogFolder(); + if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) { + return getMetaPath(); + } else { + return getMetaPath() + "/" + archiveFolder; + } + } + /** * @return Table Config */ @@ -208,6 +220,18 @@ public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs, if (!fs.exists(metaPathDir)) { fs.mkdirs(metaPathDir); } + + // if anything other than default archive log folder is specified, create that too + String archiveLogPropVal = props + .getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, + HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER); + if (!archiveLogPropVal.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) { + Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal); + if (!fs.exists(archiveLogDir)) { + fs.mkdirs(archiveLogDir); + } + } + HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index 26a0845e25124..f326295710959 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -40,7 +40,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { private final static Logger log = LogManager.getLogger(HoodieLogFormatWriter.class); - private final HoodieLogFile logFile; + private HoodieLogFile logFile; private final FileSystem fs; private final long sizeThreshold; private final Integer bufferSize; @@ -83,6 +83,15 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { throw new HoodieException(e); } } + } catch (IOException ioe) { + if (ioe.getMessage().equalsIgnoreCase("Not supported")) { + log.info("Append not supported. Opening a new log file.."); + this.logFile = logFile.rollOver(fs); + this.output = fs.create(this.logFile.getPath(), false, bufferSize, replication, + WriterBuilder.DEFAULT_SIZE_THRESHOLD, null); + } else { + throw ioe; + } } } else { log.info(logFile + " does not exist. Create a new file"); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java index bc04873afa6d0..793d9d9961778 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java @@ -50,7 +50,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { // Read back the commits to make sure - Path archiveLogPath = getArchiveLogPath(metaClient.getMetaPath()); + Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); try (SequenceFile.Reader reader = new SequenceFile.Reader(metaClient.getHadoopConf(), SequenceFile.Reader.file(archiveLogPath))) { @@ -92,8 +92,8 @@ private void readObject(java.io.ObjectInputStream in) } - public static Path getArchiveLogPath(String metaPath) { - return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE); + public static Path getArchiveLogPath(String archiveFolder) { + return new Path(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE); } @Override diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index d771b8236e9d9..fcfb9b7f658ec 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -107,7 +107,7 @@ public void checkCommitTimeline() throws IOException { @Test public void checkArchiveCommitTimeline() throws IOException { - Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath()); + Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); SequenceFile.Writer writer = SequenceFile .createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath), SequenceFile.Writer.keyClass(Text.class), diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 084cc1f121a81..1e8b021914a34 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Maps; import com.uber.hoodie.common.minicluster.MiniClusterUtil; +import com.uber.hoodie.common.model.HoodieArchivedLogFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; @@ -53,6 +54,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.After; @@ -241,6 +243,34 @@ public void testLeaseRecovery() throws IOException, URISyntaxException, Interrup writer.close(); } + @Test + public void testAppendNotSupported() + throws IOException, URISyntaxException, InterruptedException { + // Use some fs like LocalFileSystem, that does not support appends + Path localPartitionPath = new Path("file://" + partitionPath); + FileSystem localFs = FSUtils + .getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); + Path testPath = new Path(localPartitionPath, "append_test"); + localFs.mkdirs(testPath); + + // Some data & append two times. + List records = SchemaTestUtil.generateTestRecords(0, 100); + Map metadata = Maps.newHashMap(); + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, + getSimpleSchema(), metadata); + + for (int i = 0; i < 2; i++) { + HoodieLogFormat.newWriterBuilder().onParentPath(testPath) + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") + .overBaseCommit("").withFs(localFs).build().appendBlock(dataBlock).close(); + } + + // ensure there are two log file versions, with same data. + FileStatus[] statuses = localFs.listStatus(testPath); + assertEquals(2, statuses.length); + } + @SuppressWarnings("unchecked") @Test public void testBasicWriteAndScan() diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 85e4eaca3a877..d9d24caa10661 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -72,6 +72,7 @@ public void setUp() { jobConf = new JobConf(); fs = FSUtils .getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf()); + HoodieTestUtils.fs = fs; } @Rule diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 26579630d7815..285ba87b1b021 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -189,6 +189,7 @@ class DefaultSource extends RelationProvider val properties = new Properties(); properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tblName.get); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, storageType); + properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived"); HoodieTableMetaClient.initializePathAsHoodieDataset(fs, path.get, properties); } From ac6b4c2afda7ea7f16d042952d6bb13c1b73cc04 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Wed, 3 Jan 2018 16:05:30 -0800 Subject: [PATCH 005/374] Remove stateful fs member from HoodieTestUtils & FSUtils --- .../TestHoodieClientOnCopyOnWriteStorage.java | 18 +++++++-------- .../java/com/uber/hoodie/TestMultiFS.java | 23 +++++++++++-------- .../hoodie/func/TestUpdateMapFunction.java | 2 +- .../index/bloom/TestHoodieBloomIndex.java | 2 +- .../hoodie/io/TestHoodieCommitArchiveLog.java | 4 ++-- .../uber/hoodie/io/TestHoodieCompactor.java | 10 +++++--- .../hoodie/table/TestCopyOnWriteTable.java | 2 +- .../hoodie/table/TestMergeOnReadTable.java | 20 +++++----------- .../com/uber/hoodie/common/util/FSUtils.java | 13 ----------- .../common/minicluster/HdfsTestService.java | 8 ++----- .../hoodie/common/model/HoodieTestUtils.java | 23 ++++++++----------- .../table/HoodieTableMetaClientTest.java | 9 +++----- .../common/table/log/HoodieLogFormatTest.java | 4 +--- .../string/HoodieActiveTimelineTest.java | 2 +- .../view/HoodieTableFileSystemViewTest.java | 19 +++++++-------- .../hoodie/hadoop/InputFormatTestUtil.java | 8 +++++-- .../HoodieRealtimeRecordReaderTest.java | 5 ++-- .../utilities/TestHDFSParquetImporter.java | 3 --- .../utilities/TestHoodieSnapshotCopier.java | 2 +- 19 files changed, 74 insertions(+), 103 deletions(-) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index e3adf8d9c8393..184ea82e086e3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -102,7 +102,7 @@ public void init() throws IOException { folder.create(); basePath = folder.getRoot().getAbsolutePath(); fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration()); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(fs, basePath); dataGen = new HoodieTestDataGenerator(); } @@ -1247,27 +1247,27 @@ public void testKeepLatestFileVersionsMOR() throws IOException { .retainFileVersions(1).build()).build(); HoodieTableMetaClient metaClient = HoodieTestUtils - .initTableType(basePath, HoodieTableType.MERGE_ON_READ); + .initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "000", file1P0, Optional.empty()); + .createNewLogFile(fs, basePath, partitionPaths[0], "000", file1P0, Optional.empty()); String file2P0L1 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "000", file1P0, Optional.of(2)); + .createNewLogFile(fs, basePath, partitionPaths[0], "000", file1P0, Optional.of(2)); // make 1 compaction commit - HoodieTestUtils.createCompactionCommitFiles(basePath, "000"); + HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000"); // Make 4 files, one base file and 3 log files associated with base file HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0); file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.empty()); + .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); + .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); + .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); // make 1 compaction commit - HoodieTestUtils.createCompactionCommitFiles(basePath, "001"); + HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); List hoodieCleanStats = table.clean(jsc); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java index 1bdc15d25a623..676b970d9bf9c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java @@ -47,8 +47,8 @@ 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.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; public class TestMultiFS implements Serializable { @@ -64,8 +64,8 @@ public class TestMultiFS implements Serializable { private static JavaSparkContext jsc; private static SQLContext sqlContext; - @Before - public void initClass() throws Exception { + @BeforeClass + public static void initClass() throws Exception { hdfsTestService = new HdfsTestService(); dfsCluster = hdfsTestService.start(true); @@ -82,15 +82,18 @@ public void initClass() throws Exception { sqlContext = new SQLContext(jsc); } - @After - public void cleanupClass() throws Exception { - if (hdfsTestService != null) { - hdfsTestService.stop(); - } + @AfterClass + public static void cleanupClass() throws Exception { if (jsc != null) { jsc.stop(); } - FSUtils.setFs(null); + + if (hdfsTestService != null) { + hdfsTestService.stop(); + dfsCluster.shutdown(); + } + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + FileSystem.closeAll(); } @Test diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index ef4a86833f0e6..352a0036b85dd 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -48,7 +48,7 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()), basePath); } @Test diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 92dcae96b1feb..c3ace401eff67 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -88,7 +88,7 @@ public void init() throws IOException { folder.create(); basePath = folder.getRoot().getAbsolutePath(); fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(fs, basePath); // We have some records to be tagged (two different partitions) schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index 5d8362688cd4b..d0333d077470f 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -54,8 +54,8 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + HoodieTestUtils.init(fs, basePath); } @Test @@ -75,7 +75,7 @@ public void testArchiveDatasetWithArchival() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build()) .forTable("test-trip-table").build(); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(fs, basePath); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "102"); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index a6e385f2a2250..e05e73364d859 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -31,6 +31,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; @@ -44,6 +45,7 @@ import java.io.IOException; import java.util.List; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileSystem; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.junit.After; @@ -57,6 +59,7 @@ public class TestHoodieCompactor { private String basePath = null; private HoodieCompactor compactor; private transient HoodieTestDataGenerator dataGen = null; + private transient FileSystem fs; @Before public void init() throws IOException { @@ -67,7 +70,8 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); dataGen = new HoodieTestDataGenerator(); compactor = new HoodieRealtimeTableCompactor(); @@ -100,7 +104,7 @@ private HoodieWriteConfig.Builder getConfigBuilder() { @Test(expected = IllegalArgumentException.class) public void testCompactionOnCopyOnWriteFail() throws Exception { - HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE); + HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.COPY_ON_WRITE); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); @@ -155,7 +159,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, + .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, updatedRecords); // Verify that all data file has one log file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index 61303c3fb7ce4..9432be72d01c2 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -76,7 +76,7 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(FSUtils.getFs(basePath, jsc.hadoopConfiguration()), basePath); } @Test diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 5465879f030ed..a9ed76379f0fe 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -42,7 +42,6 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; @@ -80,7 +79,6 @@ public class TestMergeOnReadTable { private transient SQLContext sqlContext; private static String basePath = null; private HoodieCompactor compactor; - private FileSystem fs; //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) //The implementation and gurantees of many API's differ, for example check rename(src,dst) @@ -94,10 +92,8 @@ public static void cleanUp() throws Exception { hdfsTestService.stop(); dfsCluster.shutdown(); } - FSUtils.setFs(null); // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM FileSystem.closeAll(); - HoodieTestUtils.resetFS(basePath); } @BeforeClass @@ -110,8 +106,6 @@ public static void setUpDFS() throws IOException { // Create a temp folder as the base path dfs = dfsCluster.getFileSystem(); } - FSUtils.setFs(dfs); - HoodieTestUtils.resetFS(basePath); } @Before @@ -124,12 +118,10 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); - jsc.hadoopConfiguration().addResource(fs.getConf()); + jsc.hadoopConfiguration().addResource(dfs.getConf()); dfs.mkdirs(new Path(basePath)); - FSUtils.setFs(dfs); - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); sqlContext = new SQLContext(jsc); // SQLContext stuff compactor = new HoodieRealtimeTableCompactor(); @@ -219,7 +211,7 @@ public void testSimpleInsertAndUpdate() throws Exception { compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime()); - allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath()); + allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); @@ -339,7 +331,7 @@ public void testSimpleInsertAndDelete() throws Exception { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath()); + allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); @@ -357,7 +349,7 @@ public void testSimpleInsertAndDelete() throws Exception { public void testCOWToMORConvertedDatasetRollback() throws Exception { //Set TableType to COW - HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE); + HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.COPY_ON_WRITE); HoodieWriteConfig cfg = getConfig(true); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); @@ -396,7 +388,7 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { assertNoWriteErrors(statuses); //Set TableType to MOR - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); //rollback a COW commit when TableType is MOR client.rollback(newCommitTime); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index d788cdd4435ca..5417f01c9b328 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.util; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; @@ -57,15 +56,6 @@ public class FSUtils { private static final long MIN_CLEAN_TO_KEEP = 10; private static final long MIN_ROLLBACK_TO_KEEP = 10; private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_"; - private static FileSystem fs; - - /** - * Only to be used for testing. - */ - @VisibleForTesting - public static void setFs(FileSystem fs) { - FSUtils.fs = fs; - } public static Configuration prepareHadoopConf(Configuration conf) { conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); @@ -86,9 +76,6 @@ public static Configuration prepareHadoopConf(Configuration conf) { public static FileSystem getFs(String path, Configuration conf) { - if (fs != null) { - return fs; - } FileSystem fs; conf = prepareHadoopConf(conf); try { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java index 74cc1104b079d..a90739f75c4b4 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java @@ -55,7 +55,6 @@ public class HdfsTestService { private MiniDFSCluster miniDfsCluster; public HdfsTestService() { - hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); workDir = Files.createTempDir().getAbsolutePath(); } @@ -66,10 +65,7 @@ public Configuration getHadoopConf() { public MiniDFSCluster start(boolean format) throws IOException { Preconditions .checkState(workDir != null, "The work dir must be set before starting cluster."); - - if (hadoopConf == null) { - hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); - } + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); // If clean, then remove the work dir so we can start fresh. String localDFSLocation = getDFSLocation(workDir); @@ -91,8 +87,8 @@ public MiniDFSCluster start(boolean format) throws IOException { } public void stop() throws IOException { + logger.info("HDFS Minicluster service being shut down."); miniDfsCluster.shutdown(); - logger.info("HDFS Minicluster service shut down."); miniDfsCluster = null; hadoopConf = null; } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 7017bbf91b2bd..74103f8fa6ac2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -70,27 +70,22 @@ public class HoodieTestUtils { - public static FileSystem fs; public static final String TEST_EXTENSION = ".test"; public static final String RAW_TRIPS_TEST_NAME = "raw_trips"; public static final int DEFAULT_TASK_PARTITIONID = 1; public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; private static Random rand = new Random(46474747); - public static void resetFS(String basePath) { - HoodieTestUtils.fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - } - public static Configuration getDefaultHadoopConf() { return new Configuration(); } - public static HoodieTableMetaClient init(String basePath) throws IOException { - fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - return initTableType(basePath, HoodieTableType.COPY_ON_WRITE); + public static HoodieTableMetaClient init(FileSystem fs, String basePath) throws IOException { + return initTableType(fs, basePath, HoodieTableType.COPY_ON_WRITE); } - public static HoodieTableMetaClient initTableType(String basePath, HoodieTableType tableType) + public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, + HoodieTableType tableType) throws IOException { Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); @@ -105,7 +100,8 @@ public static HoodieTableMetaClient initOnTemp() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); String basePath = folder.getRoot().getAbsolutePath(); - return HoodieTestUtils.init(basePath); + return HoodieTestUtils + .init(FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()), basePath); } public static String makeNewCommitTime() { @@ -143,7 +139,7 @@ public static final String createDataFile(String basePath, String partitionPath, return fileID; } - public static final String createNewLogFile(String basePath, String partitionPath, + public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime, String fileID, Optional version) throws IOException { String folderPath = basePath + "/" + partitionPath + "/"; boolean makeDir = fs.mkdirs(new Path(folderPath)); @@ -159,7 +155,8 @@ public static final String createNewLogFile(String basePath, String partitionPat return fileID; } - public static final void createCompactionCommitFiles(String basePath, String... commitTimes) + public static final void createCompactionCommitFiles(FileSystem fs, String basePath, + String... commitTimes) throws IOException { for (String commitTime : commitTimes) { boolean createFile = fs.createNewFile(new Path( @@ -268,7 +265,7 @@ public static T serializeDeserialize(T object, Class return deseralizedObject; } - public static void writeRecordsToLogFiles(String basePath, Schema schema, + public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema schema, List updatedRecords) { Map> groupedUpdated = updatedRecords.stream() .collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index fcfb9b7f658ec..bd710d1a99e52 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -36,7 +36,6 @@ import org.apache.hadoop.io.Text; import org.junit.Before; import org.junit.Test; -import org.junit.rules.TemporaryFolder; public class HoodieTableMetaClientTest { @@ -45,10 +44,8 @@ public class HoodieTableMetaClientTest { @Before public void init() throws IOException { - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - this.basePath = folder.getRoot().getAbsolutePath(); - metaClient = HoodieTestUtils.init(basePath); + metaClient = HoodieTestUtils.initOnTemp(); + basePath = metaClient.getBasePath(); } @Test @@ -109,7 +106,7 @@ public void checkCommitTimeline() throws IOException { public void checkArchiveCommitTimeline() throws IOException { Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); SequenceFile.Writer writer = SequenceFile - .createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath), + .createWriter(metaClient.getHadoopConf(), SequenceFile.Writer.file(archiveLogPath), SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class)); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 1e8b021914a34..e9f3a3a877487 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -80,7 +80,6 @@ public static void setUpClass() throws IOException, InterruptedException { @AfterClass public static void tearDownClass() { MiniClusterUtil.shutdown(); - HoodieTestUtils.resetFS(basePath); } @Before @@ -91,8 +90,7 @@ public void setUp() throws IOException, InterruptedException { assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath()))); this.partitionPath = new Path(folder.getRoot().getPath()); this.basePath = folder.getRoot().getParent(); - HoodieTestUtils.fs = fs; - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); } @After diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 189be698dcda0..096c75d779320 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -49,7 +49,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true); + metaClient.getFs().delete(new Path(this.metaClient.getBasePath()), true); } @Test diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 68d32215a6ebd..fab8e371459f4 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -45,7 +45,6 @@ import org.apache.hadoop.fs.Path; import org.junit.Before; import org.junit.Test; -import org.junit.rules.TemporaryFolder; @SuppressWarnings("ResultOfMethodCallIgnored") public class HoodieTableFileSystemViewTest { @@ -58,10 +57,8 @@ public class HoodieTableFileSystemViewTest { @Before public void init() throws IOException { - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - this.basePath = folder.getRoot().getAbsolutePath(); - metaClient = HoodieTestUtils.init(basePath); + metaClient = HoodieTestUtils.initOnTemp(); + basePath = metaClient.getBasePath(); fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); roView = (TableFileSystemView.ReadOptimizedView) fsView; @@ -69,7 +66,7 @@ public void init() throws IOException { } private void refreshFsView(FileStatus[] statuses) { - metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs.getConf(), basePath, true); + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); if (statuses != null) { fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), @@ -184,7 +181,7 @@ public void testStreamLatestVersionInPartition() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(11, statuses.length); refreshFsView(null); @@ -285,7 +282,7 @@ public void testStreamEveryVersionInPartition() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(7, statuses.length); refreshFsView(null); @@ -359,7 +356,7 @@ public void streamLatestVersionInRange() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(9, statuses.length); refreshFsView(statuses); @@ -430,7 +427,7 @@ public void streamLatestVersionsBefore() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(7, statuses.length); refreshFsView(null); @@ -492,7 +489,7 @@ public void streamLatestVersions() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(10, statuses.length); refreshFsView(statuses); diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java index ac14e64842f9a..4b060a132bd26 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java @@ -39,7 +39,9 @@ public class InputFormatTestUtil { public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, String commitNumber) throws IOException { basePath.create(); - HoodieTestUtils.init(basePath.getRoot().toString()); + HoodieTestUtils + .init(FSUtils.getFs(basePath.getRoot().toString(), HoodieTestUtils.getDefaultHadoopConf()), + basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); for (int i = 0; i < numberOfFiles; i++) { File dataFile = @@ -99,7 +101,9 @@ public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { basePath.create(); - HoodieTestUtils.init(basePath.getRoot().toString()); + HoodieTestUtils + .init(FSUtils.getFs(basePath.getRoot().toString(), HoodieTestUtils.getDefaultHadoopConf()), + basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index d9d24caa10661..2bef7780a1ed9 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -72,7 +72,6 @@ public void setUp() { jobConf = new JobConf(); fs = FSUtils .getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf()); - HoodieTestUtils.fs = fs; } @Rule @@ -105,7 +104,7 @@ public void testReader() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); HoodieTestUtils - .initTableType(basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + .initTableType(fs, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; File partitionDir = InputFormatTestUtil .prepareParquetDataset(basePath, schema, 1, 100, commitTime); @@ -163,7 +162,7 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema()); HoodieTestUtils - .initTableType(basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + .initTableType(fs, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java index c1eaa86aae2f4..c1e58153fb4c5 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java @@ -27,7 +27,6 @@ import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; -import com.uber.hoodie.common.util.FSUtils; import java.io.IOException; import java.io.Serializable; import java.text.ParseException; @@ -71,7 +70,6 @@ public static void initClass() throws Exception { dfs = dfsCluster.getFileSystem(); dfsBasePath = dfs.getWorkingDirectory().toString(); dfs.mkdirs(new Path(dfsBasePath)); - FSUtils.setFs(dfs); } @AfterClass @@ -79,7 +77,6 @@ public static void cleanupClass() throws Exception { if (hdfsTestService != null) { hdfsTestService.stop(); } - FSUtils.setFs(null); } /** diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java index db670673f6047..326894dfeab0f 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java @@ -53,7 +53,7 @@ public void init() throws IOException { outputPath = rootPath + "/output"; fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(fs, basePath); // Start a local Spark job SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); jsc = new JavaSparkContext(conf); From 581e755eaff65c43f75ea46d930a00df3e4c1e5b Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Thu, 18 Jan 2018 00:05:22 -0800 Subject: [PATCH 006/374] Update Gemfile.lock --- docs/Gemfile.lock | 56 ++++++++++++++++++++++++++--------------------- 1 file changed, 31 insertions(+), 25 deletions(-) diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index dd93f97ddd3e9..e49bc0e79a9f5 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -11,14 +11,15 @@ GEM coffee-script (2.4.1) coffee-script-source execjs - coffee-script-source (1.11.1) + coffee-script-source (1.12.2) colorator (1.1.0) - ethon (0.10.1) + concurrent-ruby (1.0.5) + ethon (0.11.0) ffi (>= 1.3.0) execjs (2.7.0) - faraday (0.10.0) + faraday (0.13.1) multipart-post (>= 1.2, < 3) - ffi (1.9.14) + ffi (1.9.18) forwardable-extended (2.6.0) gemoji (2.1.0) github-pages (106) @@ -52,10 +53,11 @@ GEM octokit (~> 4.0) public_suffix (~> 1.4) typhoeus (~> 0.7) - html-pipeline (2.4.2) + html-pipeline (2.7.1) activesupport (>= 2) nokogiri (>= 1.4) - i18n (0.7.0) + i18n (0.9.1) + concurrent-ruby (~> 1.0) jekyll (3.3.1) addressable (~> 2.4) colorator (~> 1.0) @@ -94,49 +96,53 @@ GEM jekyll-sitemap (0.12.0) jekyll (~> 3.3) jekyll-swiss (0.4.0) - jekyll-watch (1.5.0) - listen (~> 3.0, < 3.1) + jekyll-watch (1.5.1) + listen (~> 3.0) jemoji (0.7.0) activesupport (~> 4.0) gemoji (~> 2.0) html-pipeline (~> 2.2) jekyll (>= 3.0) - json (1.8.3) + json (1.8.6) kramdown (1.11.1) liquid (3.0.6) listen (3.0.6) rb-fsevent (>= 0.9.3) rb-inotify (>= 0.9.7) mercenary (0.3.6) - mini_portile2 (2.1.0) + mini_portile2 (2.3.0) minima (2.0.0) - minitest (5.10.1) + minitest (5.11.1) multipart-post (2.0.0) net-dns (0.8.0) - nokogiri (1.6.8.1) - mini_portile2 (~> 2.1.0) - octokit (4.6.2) + nokogiri (1.8.1) + mini_portile2 (~> 2.3.0) + octokit (4.8.0) sawyer (~> 0.8.0, >= 0.5.3) - pathutil (0.14.0) + pathutil (0.16.1) forwardable-extended (~> 2.6) public_suffix (1.5.3) - rb-fsevent (0.9.8) - rb-inotify (0.9.7) - ffi (>= 0.5.0) + rb-fsevent (0.10.2) + rb-inotify (0.9.10) + ffi (>= 0.5.0, < 2) rouge (1.11.1) safe_yaml (1.0.4) - sass (3.4.22) + sass (3.5.5) + sass-listen (~> 4.0.0) + sass-listen (4.0.0) + rb-fsevent (~> 0.9, >= 0.9.4) + rb-inotify (~> 0.9, >= 0.9.7) sawyer (0.8.1) addressable (>= 2.3.5, < 2.6) faraday (~> 0.8, < 1.0) - terminal-table (1.7.3) - unicode-display_width (~> 1.1.1) - thread_safe (0.3.5) + terminal-table (1.8.0) + unicode-display_width (~> 1.1, >= 1.1.1) + thread_safe (0.3.6) typhoeus (0.8.0) ethon (>= 0.8.0) - tzinfo (1.2.2) + tzinfo (1.2.4) thread_safe (~> 0.1) - unicode-display_width (1.1.1) + unicode-display_width (1.3.0) PLATFORMS ruby @@ -147,4 +153,4 @@ DEPENDENCIES jekyll-feed (~> 0.6) BUNDLED WITH - 1.13.7 + 1.14.3 From cd796782a4dfdb45b67ab19fb4d6aa28b58e97f9 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 15 Dec 2017 14:03:06 -0800 Subject: [PATCH 007/374] Reducing memory footprint required in HoodieAvroDataBlock and HoodieAppendHandle --- .../uber/hoodie/io/HoodieAppendHandle.java | 7 +- .../storage/SizeAwareDataInputStream.java | 63 +++++++++++++++ .../table/log/block/HoodieAvroDataBlock.java | 56 +++++++------ .../table/log/block/HoodieCommandBlock.java | 4 +- .../table/log/block/HoodieCorruptBlock.java | 4 +- .../table/log/block/HoodieDeleteBlock.java | 4 +- .../table/log/block/HoodieLogBlock.java | 3 +- .../common/table/log/HoodieLogFormatTest.java | 81 ++++++++++++++----- 8 files changed, 174 insertions(+), 48 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 683c6a75ea4fc..5866276b90990 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -160,14 +160,17 @@ public void doAppend() { List keysToDelete = new ArrayList<>(); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime); - records.stream().forEach(record -> { + Iterator> recordsItr = records.iterator(); + while (recordsItr.hasNext()) { + HoodieRecord record = recordsItr.next(); Optional indexedRecord = getIndexedRecord(record); if (indexedRecord.isPresent()) { recordList.add(indexedRecord.get()); } else { keysToDelete.add(record.getRecordKey()); } - }); + recordsItr.remove(); //remove entries when IndexedRecord added to new list + } try { if (recordList.size() > 0) { writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata)); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java new file mode 100644 index 0000000000000..dc9e04043c878 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java @@ -0,0 +1,63 @@ +/* + * 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.storage; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Keeps track of how many bytes were read from a DataInputStream + */ +public class SizeAwareDataInputStream { + + private final DataInputStream dis; + private final AtomicInteger numberOfBytesRead; + + public SizeAwareDataInputStream(DataInputStream dis) { + this.dis = dis; + this.numberOfBytesRead = new AtomicInteger(0); + } + + public int readInt() throws IOException { + numberOfBytesRead.addAndGet(Integer.BYTES); + return dis.readInt(); + } + + public void readFully(byte b[], int off, int len) throws IOException { + numberOfBytesRead.addAndGet(len); + dis.readFully(b, off, len); + } + + public void readFully(byte b[]) throws IOException { + numberOfBytesRead.addAndGet(b.length); + dis.readFully(b); + } + + public int skipBytes(int n) throws IOException { + numberOfBytesRead.addAndGet(n); + return dis.skipBytes(n); + } + + public void close() throws IOException { + dis.close(); + } + + public Integer getNumberOfBytesRead() { + return numberOfBytesRead.get(); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java index d2f73ef1b42a3..ebb72bb4bdc91 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java @@ -16,16 +16,9 @@ package com.uber.hoodie.common.table.log.block; +import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.exception.HoodieIOException; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; import org.apache.avro.Schema; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; @@ -35,18 +28,31 @@ import org.apache.avro.io.Encoder; import org.apache.avro.io.EncoderFactory; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + /** - * DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Compressed - * Writer Schema length 2. Compressed Writer Schema content 3. Total number of records in the block - * 4. Size of a record 5. Actual avro serialized content of the record + * DataBlock contains a list of records serialized using Avro. + * The Datablock contains + * 1. Compressed Writer Schema length + * 2. Compressed Writer Schema content + * 3. Total number of records in the block + * 4. Size of a record + * 5. Actual avro serialized content of the record */ public class HoodieAvroDataBlock extends HoodieLogBlock { private List records; private Schema schema; - public HoodieAvroDataBlock(List records, Schema schema, - Map metadata) { + public HoodieAvroDataBlock(List records, Schema schema, Map metadata) { super(metadata); this.records = records; this.schema = schema; @@ -56,6 +62,7 @@ public HoodieAvroDataBlock(List records, Schema schema) { this(records, schema, null); } + //TODO : (na) lazily create IndexedRecords only when required public List getRecords() { return records; } @@ -85,7 +92,9 @@ public byte[] getBytes() throws IOException { output.writeInt(records.size()); // 4. Write the records - records.forEach(s -> { + Iterator itr = records.iterator(); + while (itr.hasNext()) { + IndexedRecord s = itr.next(); ByteArrayOutputStream temp = new ByteArrayOutputStream(); Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null); try { @@ -99,10 +108,11 @@ public byte[] getBytes() throws IOException { output.writeInt(size); // Write the content output.write(temp.toByteArray()); + itr.remove(); } catch (IOException e) { throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e); } - }); + } output.close(); return baos.toByteArray(); @@ -113,10 +123,10 @@ public HoodieLogBlockType getBlockType() { return HoodieLogBlockType.AVRO_DATA_BLOCK; } - public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) - throws IOException { + //TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used + public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) throws IOException { - DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content)); + SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); Map metadata = null; // 1. Read the metadata written out, if applicable if (readMetadata) { @@ -132,22 +142,20 @@ public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, bool readerSchema = writerSchema; } + //TODO : (na) lazily create IndexedRecords only when required GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema); // 2. Get the total records int totalRecords = dis.readInt(); List records = new ArrayList<>(totalRecords); // 3. Read the content - for (int i = 0; i < totalRecords; i++) { - // TODO - avoid bytes copy + for (int i=0;i metadata = null; if (readMetadata) { metadata = HoodieLogBlock.getLogMetadata(dis); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java index 3858ae54ef655..5819c99e1d9e8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java @@ -16,6 +16,8 @@ package com.uber.hoodie.common.table.log.block; +import com.uber.hoodie.common.storage.SizeAwareDataInputStream; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -62,7 +64,7 @@ public byte[] getCorruptedBytes() { public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata) throws IOException { - DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content)); + SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); Map metadata = null; int bytesRemaining = blockSize; if (readMetadata) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index 485bfdcc0691b..3751124fec96a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.nio.charset.Charset; import java.util.Map; + +import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import org.apache.commons.lang3.StringUtils; /** @@ -64,7 +66,7 @@ public HoodieLogBlockType getBlockType() { } public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException { - DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content)); + SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); Map metadata = null; if (readMetadata) { metadata = HoodieLogBlock.getLogMetadata(dis); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index 39049b25bb22c..d21332f2fd18f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -17,6 +17,7 @@ package com.uber.hoodie.common.table.log.block; import com.google.common.collect.Maps; +import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import com.uber.hoodie.exception.HoodieException; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -90,7 +91,7 @@ public static byte[] getLogMetadataBytes(Map metadata) /** * Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes} */ - public static Map getLogMetadata(DataInputStream dis) + public static Map getLogMetadata(SizeAwareDataInputStream dis) throws IOException { Map metadata = Maps.newHashMap(); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index e9f3a3a877487..00fd9ca5e4eac 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -276,7 +276,11 @@ public void testBasicWriteAndScan() Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); + Schema schema = getSimpleSchema(); List records = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords = records.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, @@ -292,8 +296,8 @@ public void testBasicWriteAndScan() nextBlock.getBlockType()); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", - records.size(), dataBlockRead.getRecords().size()); - assertEquals("Both records lists should be the same. (ordering guaranteed)", records, + copyOfRecords.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords, dataBlockRead.getRecords()); } @@ -305,6 +309,10 @@ public void testBasicAppendAndRead() .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); + Schema schema = getSimpleSchema(); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, @@ -316,6 +324,9 @@ public void testBasicAppendAndRead() .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords2 = records2.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, getSimpleSchema(), metadata); writer = writer.appendBlock(dataBlock); @@ -326,6 +337,9 @@ public void testBasicAppendAndRead() .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords3 = records3.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, getSimpleSchema(), metadata); writer = writer.appendBlock(dataBlock); @@ -337,22 +351,22 @@ public void testBasicAppendAndRead() HoodieLogBlock nextBlock = reader.next(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", - records1.size(), dataBlockRead.getRecords().size()); - assertEquals("Both records lists should be the same. (ordering guaranteed)", records1, + copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", - records2.size(), dataBlockRead.getRecords().size()); - assertEquals("Both records lists should be the same. (ordering guaranteed)", records2, + copyOfRecords2.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, dataBlockRead.getRecords()); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", - records3.size(), dataBlockRead.getRecords().size()); - assertEquals("Both records lists should be the same. (ordering guaranteed)", records3, + copyOfRecords3.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, dataBlockRead.getRecords()); } @@ -450,6 +464,9 @@ public void testAvroLogRecordReaderBasic() .overBaseCommit("100").withFs(fs).withSizeThreshold(500).build(); // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); @@ -459,6 +476,9 @@ public void testAvroLogRecordReaderBasic() // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords2 = records2.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); writer = writer.appendBlock(dataBlock); writer.close(); @@ -475,8 +495,8 @@ public void testAvroLogRecordReaderBasic() Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); - records1.addAll(records2); - Set originalKeys = records1.stream() + copyOfRecords1.addAll(copyOfRecords2); + Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect( Collectors.toSet()); @@ -495,6 +515,9 @@ public void testAvroLogRecordReaderWithRollbackTombstone() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); @@ -515,6 +538,9 @@ public void testAvroLogRecordReaderWithRollbackTombstone() // Write 3 List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords3 = records3.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, schema, metadata); writer = writer.appendBlock(dataBlock); writer.close(); @@ -532,8 +558,8 @@ public void testAvroLogRecordReaderWithRollbackTombstone() Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); - records1.addAll(records3); - Set originalKeys = records1.stream() + copyOfRecords1.addAll(copyOfRecords3); + Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect( Collectors.toSet()); @@ -552,6 +578,9 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); @@ -585,6 +614,10 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() // Write 3 List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords3 = records3.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + dataBlock = new HoodieAvroDataBlock(records3, schema, metadata); writer = writer.appendBlock(dataBlock); writer.close(); @@ -602,8 +635,8 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); - records1.addAll(records3); - Set originalKeys = records1.stream() + copyOfRecords1.addAll(copyOfRecords3); + Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect( Collectors.toSet()); @@ -622,6 +655,9 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); @@ -631,11 +667,14 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords2 = records2.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); writer = writer.appendBlock(dataBlock); - records1.addAll(records2); - List originalKeys = records1.stream() + copyOfRecords1.addAll(copyOfRecords2); + List originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect( Collectors.toList()); @@ -691,9 +730,12 @@ public void testAvroLogRecordReaderWithFailedRollbacks() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema, metadata); writer = writer.appendBlock(dataBlock); @@ -703,7 +745,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks() dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); writer = writer.appendBlock(dataBlock); - List originalKeys = records1.stream() + List originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect( Collectors.toList()); @@ -757,6 +799,9 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); @@ -764,7 +809,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() schema, metadata); writer = writer.appendBlock(dataBlock); - List originalKeys = records1.stream() + List originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect( Collectors.toList()); From 6f737bd9b0bc01070db3b2c6517a55715b920176 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 23 Jan 2018 15:10:43 -0800 Subject: [PATCH 008/374] Reducing list status calls from listing logfile versions, some associated refactoring --- .../com/uber/hoodie/io/HoodieAppendHandle.java | 16 +++++++++++----- .../java/com/uber/hoodie/io/HoodieIOHandle.java | 2 -- .../com/uber/hoodie/io/HoodieMergeHandle.java | 3 +++ .../io/compact/HoodieRealtimeTableCompactor.java | 5 +++-- .../uber/hoodie/common/model/HoodieLogFile.java | 1 + .../com/uber/hoodie/common/util/FSUtils.java | 4 ++-- 6 files changed, 20 insertions(+), 11 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 5866276b90990..fb107444d4543 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -19,11 +19,13 @@ import com.beust.jcommander.internal.Maps; import com.clearspring.analytics.util.Lists; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDeltaWriteStat; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; @@ -58,6 +60,7 @@ public class HoodieAppendHandle extends HoodieIOH private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class); private static AtomicLong recordIndex = new AtomicLong(1); + private TableFileSystemView.RealtimeView fileSystemView; private final WriteStatus writeStatus; private final String fileId; private String partitionPath; @@ -77,6 +80,7 @@ public HoodieAppendHandle(HoodieWriteConfig config, writeStatus.setStat(new HoodieDeltaWriteStat()); this.writeStatus = writeStatus; this.fileId = fileId; + this.fileSystemView = hoodieTable.getRTFileSystemView(); init(recordItr); } @@ -87,11 +91,11 @@ private void init(Iterator> recordItr) { // extract some information from the first record if (partitionPath == null) { partitionPath = record.getPartitionPath(); + FileSlice fileSlice = fileSystemView.getLatestFileSlices(record.getPartitionPath()) + .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)) + .findFirst().get(); // HACK(vc) This also assumes a base file. It will break, if appending without one. - String latestValidFilePath = - fileSystemView.getLatestDataFiles(record.getPartitionPath()) - .filter(dataFile -> dataFile.getFileId().equals(fileId)) - .findFirst().get().getFileName(); + String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); writeStatus.getStat().setPrevCommit(baseCommitTime); writeStatus.setFileId(fileId); @@ -101,7 +105,9 @@ private void init(Iterator> recordItr) { try { this.writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) - .withFileId(fileId).overBaseCommit(baseCommitTime) + .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles() + .max(HoodieLogFile.getLogVersionComparator()::compare) + .map(logFile -> logFile.getLogVersion()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) .withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); this.currentLogFile = writer.getLogFile(); ((HoodieDeltaWriteStat) writeStatus.getStat()) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 332c9a2d3ba1f..a02df5213e3c1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -40,7 +40,6 @@ public abstract class HoodieIOHandle { protected final FileSystem fs; protected final HoodieTable hoodieTable; protected HoodieTimeline hoodieTimeline; - protected TableFileSystemView.ReadOptimizedView fileSystemView; protected final Schema schema; public HoodieIOHandle(HoodieWriteConfig config, String commitTime, @@ -50,7 +49,6 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, this.fs = hoodieTable.getMetaClient().getFs(); this.hoodieTable = hoodieTable; this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline(); - this.fileSystemView = hoodieTable.getROFileSystemView(); this.schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index b61b9d9e83283..e6ce314682317 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -22,6 +22,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; +import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -48,6 +49,7 @@ public class HoodieMergeHandle extends HoodieIOHa private WriteStatus writeStatus; private HashMap> keyToNewRecords; private HoodieStorageWriter storageWriter; + private TableFileSystemView.ReadOptimizedView fileSystemView; private Path newFilePath; private Path oldFilePath; private long recordsWritten = 0; @@ -60,6 +62,7 @@ public HoodieMergeHandle(HoodieWriteConfig config, Iterator> recordItr, String fileId) { super(config, commitTime, hoodieTable); + this.fileSystemView = hoodieTable.getROFileSystemView(); init(fileId, recordItr); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index fda6b5a268f1d..501e452c41bb8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; @@ -78,11 +79,11 @@ public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); + TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) - .flatMap((FlatMapFunction) partitionPath -> hoodieTable - .getRTFileSystemView() + .flatMap((FlatMapFunction) partitionPath -> fileSystemView .getLatestFileSlices(partitionPath) .map(s -> new CompactionOperation(s.getDataFile().get(), partitionPath, s.getLogFiles().collect(Collectors.toList()), config)) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java index 4e09f5f339e70..c4a5279144f14 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java @@ -36,6 +36,7 @@ public class HoodieLogFile implements Serializable { public static final String DELTA_EXTENSION = ".log"; + public static final Integer LOGFILE_BASE_VERSION = 1; private final Path path; private Optional fileStatus; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index 5417f01c9b328..f0173f0770a52 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -284,7 +284,7 @@ public static int getCurrentLogVersion(FileSystem fs, Path partitionPath, Optional currentVersion = getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime); // handle potential overflow - return (currentVersion.isPresent()) ? currentVersion.get() : 1; + return (currentVersion.isPresent()) ? currentVersion.get() : HoodieLogFile.LOGFILE_BASE_VERSION; } /** @@ -295,7 +295,7 @@ public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final Optional currentVersion = getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime); // handle potential overflow - return (currentVersion.isPresent()) ? currentVersion.get() + 1 : 1; + return (currentVersion.isPresent()) ? currentVersion.get() + 1 : HoodieLogFile.LOGFILE_BASE_VERSION; } public static int getDefaultBufferSize(final FileSystem fs) { From 77a6aeb39bf32e88d360ef8d20c420d6584de117 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Wed, 29 Nov 2017 16:59:28 -0800 Subject: [PATCH 009/374] Add FinalizeWrite in HoodieCreateHandle for COW tables --- .../com/uber/hoodie/HoodieWriteClient.java | 69 +++++++++++++- .../uber/hoodie/config/HoodieWriteConfig.java | 26 +++++ .../uber/hoodie/io/HoodieCreateHandle.java | 27 +++++- .../com/uber/hoodie/io/HoodieIOHandle.java | 7 ++ .../uber/hoodie/metrics/HoodieMetrics.java | 19 ++++ .../hoodie/table/HoodieCopyOnWriteTable.java | 36 +++++++ .../hoodie/table/HoodieMergeOnReadTable.java | 5 + .../com/uber/hoodie/table/HoodieTable.java | 10 ++ .../TestHoodieClientOnCopyOnWriteStorage.java | 94 +++++++++++++++++++ .../hoodie/common/model/HoodieWriteStat.java | 15 +++ .../common/table/HoodieTableMetaClient.java | 1 + .../com/uber/hoodie/common/util/FSUtils.java | 4 + .../uber/hoodie/common/util/TestFSUtils.java | 12 +++ 13 files changed, 318 insertions(+), 7 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 6b0adf67fbb74..6ce23a62d2843 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -57,6 +57,7 @@ import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.text.ParseException; +import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashMap; @@ -64,7 +65,9 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileStatus; 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.Partitioner; @@ -416,14 +419,30 @@ public boolean commit(String commitTime, HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); List> stats = writeStatuses - .mapToPair((PairFunction) writeStatus -> - new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) - .collect(); + .mapToPair((PairFunction) writeStatus -> + new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) + .collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); for (Tuple2 stat : stats) { metadata.addWriteStat(stat._1(), stat._2()); } + + // Finalize write + final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); + final Optional result = table.finalizeWrite(jsc, stats); + if (finalizeCtx != null && result.isPresent()) { + Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); + durationInMs.ifPresent(duration -> { + logger.info("Finalize write elapsed time (Seconds): " + duration / 1000); + metrics.updateFinalizeWriteMetrics(duration, result.get()); + } + ); + } + + // Clean temp files + cleanTemporaryDataFiles(); + // add in extra metadata if (extraMetadata.isPresent()) { extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v)); @@ -679,6 +698,9 @@ private void rollback(List commits) { } }); + // clean data files in temporary folder + cleanTemporaryDataFiles(); + try { if (commitTimeline.empty() && inflightTimeline.empty()) { // nothing to rollback @@ -741,6 +763,35 @@ private void rollback(List commits) { } } + private void cleanTemporaryDataFiles() { + if (!config.shouldFinalizeWrite()) { + return; + } + + final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); + try { + if (!fs.exists(temporaryFolder)) { + return; + } + List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); + List> results = jsc.parallelize(fileStatusesList, config.getFinalizeParallelism()) + .map(fileStatus -> { + FileSystem fs1 = FSUtils.getFs(); + boolean success = fs1.delete(fileStatus.getPath(), false); + logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + success); + return new Tuple2<>(fileStatus.getPath().toString(), success); + }).collect(); + + for (Tuple2 result : results) { + if (!result._2()) { + logger.info("Failed to delete file: " + result._1()); + throw new HoodieIOException("Failed to delete file in temporary folder: " + result._1()); + } + } + } catch (IOException e) { + throw new HoodieIOException("Failed to clean data files in temporary folder: " + temporaryFolder); + } + } /** * Releases any resources used by the client. */ @@ -826,6 +877,18 @@ public void startCommitWithTime(String commitTime) { String commitActionType = table.getCommitActionType(); activeTimeline.createInflight( new HoodieInstant(true, commitActionType, commitTime)); + + // create temporary folder if needed + if (config.shouldFinalizeWrite()) { + final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); + try { + if (!fs.exists(temporaryFolder)) { + fs.mkdirs(temporaryFolder); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to create temporary folder: " + temporaryFolder); + } + } } /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index b2efc8254c798..4c9d1fb2c3198 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -58,6 +58,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class"; private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); + private static final String HOODIE_FINALIZE_WRITE_BEFORE_COMMIT = "hoodie.finalize.write.before.commit"; + private static final String DEFAULT_HOODIE_FINALIZE_WRITE_BEFORE_COMMIT = "false"; + private static final String FINALIZE_PARALLELISM = "hoodie.finalize.parallelism"; + private static final String DEFAULT_FINALIZE_PARALLELISM = "5"; private HoodieWriteConfig(Properties props) { super(props); @@ -114,6 +118,14 @@ public String getWriteStatusClassName() { return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP); } + public boolean shouldFinalizeWrite() { + return Boolean.parseBoolean(props.getProperty(HOODIE_FINALIZE_WRITE_BEFORE_COMMIT)); + } + + public int getFinalizeParallelism() { + return Integer.parseInt(props.getProperty(FINALIZE_PARALLELISM)); + } + /** * compaction properties **/ @@ -385,6 +397,16 @@ public Builder withWriteStatusClass(Class writeStatusClas return this; } + public Builder withFinalizeWrite(boolean shouldFinalizeWrite) { + props.setProperty(HOODIE_FINALIZE_WRITE_BEFORE_COMMIT, String.valueOf(shouldFinalizeWrite)); + return this; + } + + public Builder withFinalizeParallelism(int parallelism) { + props.setProperty(FINALIZE_PARALLELISM, String.valueOf(parallelism)); + return this; + } + public HoodieWriteConfig build() { HoodieWriteConfig config = new HoodieWriteConfig(props); // Check for mandatory properties @@ -408,6 +430,10 @@ public HoodieWriteConfig build() { HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING); setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); + setDefaultOnCondition(props, !props.containsKey(HOODIE_FINALIZE_WRITE_BEFORE_COMMIT), + HOODIE_FINALIZE_WRITE_BEFORE_COMMIT, DEFAULT_HOODIE_FINALIZE_WRITE_BEFORE_COMMIT); + setDefaultOnCondition(props, !props.containsKey(FINALIZE_PARALLELISM), + FINALIZE_PARALLELISM, DEFAULT_FINALIZE_PARALLELISM); // Make sure the props is propagated setDefaultOnCondition(props, !isIndexConfigSet, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 629869e182103..18f5be319dd30 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -45,6 +45,7 @@ public class HoodieCreateHandle extends HoodieIOH private final WriteStatus status; private final HoodieStorageWriter storageWriter; private final Path path; + private final Path tempPath; private long recordsWritten = 0; private long recordsDeleted = 0; @@ -55,7 +56,14 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, status.setFileId(UUID.randomUUID().toString()); status.setPartitionPath(partitionPath); - this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId()); + final int sparkPartitionId = TaskContext.getPartitionId(); + this.path = makeNewPath(partitionPath, sparkPartitionId, status.getFileId()); + if (config.shouldFinalizeWrite()) { + this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(), TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); + } else { + this.tempPath = null; + } + try { HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, @@ -64,10 +72,10 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, partitionMetadata.trySave(TaskContext.getPartitionId()); this.storageWriter = HoodieStorageWriterFactory - .getStorageWriter(commitTime, path, hoodieTable, config, schema); + .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); } catch (IOException e) { throw new HoodieInsertException( - "Failed to initialize HoodieStorageWriter for path " + path, e); + "Failed to initialize HoodieStorageWriter for path " + getStorageWriterPath(), e); } logger.info("New InsertHandle for partition :" + partitionPath); } @@ -126,7 +134,10 @@ public WriteStatus close() { stat.setFileId(status.getFileId()); String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", ""); stat.setPath(relativePath); - stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path)); + if (tempPath != null) { + stat.setTempPath(tempPath.toString().replace(new Path(config.getBasePath()) + "/", "")); + } + stat.setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath())); stat.setTotalWriteErrors(status.getFailedRecords().size()); status.setStat(stat); @@ -136,4 +147,12 @@ public WriteStatus close() { e); } } + + private Path getStorageWriterPath() { + // Use tempPath for storage writer if possible + if (this.tempPath != null) { + return this.tempPath; + } + return this.path; + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index a02df5213e3c1..f207ea41a7cca 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -17,6 +17,7 @@ package com.uber.hoodie.io; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.util.FSUtils; @@ -65,6 +66,12 @@ public Path makeNewPath(String partitionPath, int taskPartitionId, String fileNa FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)); } + public Path makeTempPath(String partitionPath, int taskPartitionId, String fileName, int stageId, long taskAttemptId) { + Path path = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); + return new Path(path.toString(), + FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId)); + } + /** * Deletes any new tmp files written during the current commit, into the partition */ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index f6c79bb10b5c3..fe275e1597395 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -37,9 +37,11 @@ public class HoodieMetrics { public String rollbackTimerName = null; public String cleanTimerName = null; public String commitTimerName = null; + public String finalizeTimerName = null; private Timer rollbackTimer = null; private Timer cleanTimer = null; private Timer commitTimer = null; + private Timer finalizeTimer = null; public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.config = config; @@ -49,6 +51,7 @@ public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.rollbackTimerName = getMetricsName("timer", "rollback"); this.cleanTimerName = getMetricsName("timer", "clean"); this.commitTimerName = getMetricsName("timer", "commit"); + this.finalizeTimerName = getMetricsName("timer", "finalize"); } } @@ -77,6 +80,13 @@ public Timer.Context getCommitCtx() { return commitTimer == null ? null : commitTimer.time(); } + public Timer.Context getFinalizeCtx() { + if (config.isMetricsOn() && finalizeTimer == null) { + finalizeTimer = createTimer(finalizeTimerName); + } + return finalizeTimer == null ? null : finalizeTimer.time(); + } + public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, HoodieCommitMetadata metadata) { if (config.isMetricsOn()) { @@ -119,6 +129,15 @@ public void updateCleanMetrics(long durationInMs, int numFilesDeleted) { } } + public void updateFinalizeWriteMetrics(long durationInMs, int numFilesFinalized) { + if (config.isMetricsOn()) { + logger.info(String.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", + durationInMs, numFilesFinalized)); + registerGauge(getMetricsName("finalize", "duration"), durationInMs); + registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized); + } + } + @VisibleForTesting String getMetricsName(String action, String metric) { return config == null ? null : diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 3e4b1dc86fcc5..699d772d0aef4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; @@ -574,6 +575,41 @@ public List rollback(JavaSparkContext jsc, List comm return stats; } + @Override + @SuppressWarnings("unchecked") + public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { + if (!config.shouldFinalizeWrite()) { + return Optional.empty(); + } + + List> results = jsc.parallelize(writeStatuses, config.getFinalizeParallelism()) + .map(writeStatus -> { + Tuple2 writeStatTuple2 = (Tuple2) writeStatus; + HoodieWriteStat writeStat = writeStatTuple2._2(); + final FileSystem fs = FSUtils.getFs(); + final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); + + if (writeStat.getTempPath() != null) { + final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); + boolean success; + try { + logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); + success = fs.rename(tempPath, finalPath); + } catch (IOException e) { + throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath); + } + + if (!success) { + throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath); + } + } + + return new Tuple2<>(writeStat.getPath(), true); + }).collect(); + + return Optional.of(results.size()); + } + private static class PartitionCleanStat implements Serializable { private final String partitionPath; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 839aa1840ebeb..1e507c6453429 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -250,4 +250,9 @@ public List rollback(JavaSparkContext jsc, List comm return allRollbackStats; } + @Override + public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { + // do nothing for MOR tables + return Optional.empty(); + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 3140e34eec286..c3b9d1f702298 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -23,6 +23,7 @@ import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; @@ -46,6 +47,7 @@ import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; /** * Abstract implementation of a HoodieTable @@ -270,4 +272,12 @@ public abstract Optional compact(JavaSparkContext jsc, */ public abstract List rollback(JavaSparkContext jsc, List commits) throws IOException; + + /** + * Finalize the written data files + * + * @param writeStatuses List of WriteStatus + * @return number of files finalized + */ + public abstract Optional finalizeWrite(JavaSparkContext jsc, List> writeStatuses); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 184ea82e086e3..10773db0f1d7a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -302,6 +302,100 @@ public void testUpserts() throws Exception { HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); } + @Test + public void testUpsertsWithFinalizeWrite() throws Exception { + HoodieWriteConfig cfg = getConfigBuilder() + .withFinalizeWrite(true) + .build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); + FileSystem fs = FSUtils.getFs(); + + /** + * Write 1 (only inserts) + */ + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + List statuses = client.upsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + + // check the partition metadata is written out + assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); + + // verify that there is a commit + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) + .getCommitTimeline(); + + assertEquals("Expecting a single commit.", 1, + timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be 001", newCommitTime, + timeline.lastInstant().get().getTimestamp()); + assertEquals("Must contain 200 records", + records.size(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); + // Should have 100 records in table (check using Index), all in locations marked at commit + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) + .collect(); + checkTaggedRecords(taggedRecords, "001"); + + /** + * Write 2 (updates) + */ + newCommitTime = "004"; + client.startCommitWithTime(newCommitTime); + + records = dataGen.generateUpdates(newCommitTime, 100); + LinkedHashMap recordsMap = new LinkedHashMap<>(); + for (HoodieRecord rec : records) { + if (!recordsMap.containsKey(rec.getKey())) { + recordsMap.put(rec.getKey(), rec); + } + } + List dedupedRecords = new ArrayList<>(recordsMap.values()); + + statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + // verify there are now 2 commits + timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline(); + assertEquals("Expecting two commits.", + timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); + assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), + newCommitTime); + + metaClient = new HoodieTableMetaClient(fs, basePath); + table = HoodieTable.getHoodieTable(metaClient, getConfig()); + + // Index should be able to locate all updates in correct locations. + taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), table).collect(); + checkTaggedRecords(taggedRecords, "004"); + + // Check the entire dataset has 100 records still + String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; + for (int i = 0; i < fullPartitionPaths.length; i++) { + fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); + } + assertEquals("Must contain 200 records", + 200, + HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); + + // Check that the incremental consumption from time 000 + assertEquals("Incremental consumption from time 002, should give all records in commit 004", + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "002").count()); + assertEquals("Incremental consumption from time 001, should give all records in commit 004", + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); + } + @Test public void testDeletes() throws Exception { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index f1a58f740187c..c8f241062f4c3 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -70,6 +70,12 @@ public class HoodieWriteStat implements Serializable { */ private long totalWriteErrors; + /** + * Relative path to the temporary file from the base path. + */ + @Nullable + private String tempPath; + /** * Following properties are associated only with the result of a Compaction Operation */ @@ -198,11 +204,20 @@ public void setTotalRecordsToBeUpdate(Long totalRecordsToBeUpdate) { this.totalRecordsToBeUpdate = totalRecordsToBeUpdate; } + public void setTempPath(String tempPath) { + this.tempPath = tempPath; + } + + public String getTempPath() { + return this.tempPath; + } + @Override public String toString() { return new StringBuilder() .append("HoodieWriteStat {") .append("path=" + path) + .append(", tempPath=" + tempPath) .append(", prevCommit='" + prevCommit + '\'') .append(", numWrites=" + numWrites) .append(", numDeletes=" + numDeletes) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 841cf47ab5e2f..c776abf381f77 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -50,6 +50,7 @@ public class HoodieTableMetaClient implements Serializable { private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class); public static String METAFOLDER_NAME = ".hoodie"; + public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; private String basePath; private transient FileSystem fs; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index f0173f0770a52..31fac73558b1f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -94,6 +94,10 @@ public static String makeDataFileName(String commitTime, int taskPartitionId, St return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime); } + public static String makeTempDataFileName(String partitionPath, String commitTime, int taskPartitionId, String fileId, int stageId, long taskAttemptId) { + return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId, taskPartitionId, commitTime, stageId, taskAttemptId); + } + public static String maskWithoutFileId(String commitTime, int taskPartitionId) { return String.format("*_%s_%s.parquet", taskPartitionId, commitTime); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java index 43b0ebdf32ea2..3e05158b2c8b5 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java @@ -43,6 +43,18 @@ public void testMakeDataFileName() { .equals(fileName + "_" + taskPartitionId + "_" + commitTime + ".parquet")); } + @Test + public void testMakeTempDataFileName() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + String partitionPath = "2017/12/31"; + int taskPartitionId = Integer.MAX_VALUE; + int stageId = Integer.MAX_VALUE; + long taskAttemptId = Long.MAX_VALUE; + String fileName = UUID.randomUUID().toString(); + assertTrue(FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId) + .equals(partitionPath.replace("/", "-") + "_" + fileName + "_" + taskPartitionId + "_" + commitTime + "_" + stageId + "_" + taskAttemptId + ".parquet")); + } + @Test public void testMaskFileName() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); From a3d4b5396ee9020d22958006aac8531e3da0d2b1 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Wed, 6 Dec 2017 14:35:44 -0800 Subject: [PATCH 010/374] Incorporating code review feedback for finalizeWrite for COW --- .../com/uber/hoodie/HoodieWriteClient.java | 54 +++------------- .../uber/hoodie/config/HoodieWriteConfig.java | 32 +++++----- .../uber/hoodie/io/HoodieCreateHandle.java | 6 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 62 ++++++++++++++++++- .../hoodie/table/HoodieMergeOnReadTable.java | 10 +++ .../com/uber/hoodie/table/HoodieTable.java | 10 +++ .../TestHoodieClientOnCopyOnWriteStorage.java | 2 +- 7 files changed, 106 insertions(+), 70 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 6ce23a62d2843..ddde0b9642858 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -419,9 +419,9 @@ public boolean commit(String commitTime, HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); List> stats = writeStatuses - .mapToPair((PairFunction) writeStatus -> - new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) - .collect(); + .mapToPair((PairFunction) writeStatus -> + new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) + .collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); for (Tuple2 stat : stats) { @@ -434,14 +434,14 @@ public boolean commit(String commitTime, if (finalizeCtx != null && result.isPresent()) { Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); durationInMs.ifPresent(duration -> { - logger.info("Finalize write elapsed time (Seconds): " + duration / 1000); + logger.info("Finalize write elapsed time (milliseconds): " + duration); metrics.updateFinalizeWriteMetrics(duration, result.get()); } ); } // Clean temp files - cleanTemporaryDataFiles(); + table.cleanTemporaryDataFiles(jsc); // add in extra metadata if (extraMetadata.isPresent()) { @@ -699,7 +699,7 @@ private void rollback(List commits) { }); // clean data files in temporary folder - cleanTemporaryDataFiles(); + table.cleanTemporaryDataFiles(jsc); try { if (commitTimeline.empty() && inflightTimeline.empty()) { @@ -763,35 +763,6 @@ private void rollback(List commits) { } } - private void cleanTemporaryDataFiles() { - if (!config.shouldFinalizeWrite()) { - return; - } - - final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); - try { - if (!fs.exists(temporaryFolder)) { - return; - } - List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); - List> results = jsc.parallelize(fileStatusesList, config.getFinalizeParallelism()) - .map(fileStatus -> { - FileSystem fs1 = FSUtils.getFs(); - boolean success = fs1.delete(fileStatus.getPath(), false); - logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + success); - return new Tuple2<>(fileStatus.getPath().toString(), success); - }).collect(); - - for (Tuple2 result : results) { - if (!result._2()) { - logger.info("Failed to delete file: " + result._1()); - throw new HoodieIOException("Failed to delete file in temporary folder: " + result._1()); - } - } - } catch (IOException e) { - throw new HoodieIOException("Failed to clean data files in temporary folder: " + temporaryFolder); - } - } /** * Releases any resources used by the client. */ @@ -877,18 +848,7 @@ public void startCommitWithTime(String commitTime) { String commitActionType = table.getCommitActionType(); activeTimeline.createInflight( new HoodieInstant(true, commitActionType, commitTime)); - - // create temporary folder if needed - if (config.shouldFinalizeWrite()) { - final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); - try { - if (!fs.exists(temporaryFolder)) { - fs.mkdirs(temporaryFolder); - } - } catch (IOException e) { - throw new HoodieIOException("Failed to create temporary folder: " + temporaryFolder); - } - } + table.initializeFinalizeWrite(); } /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 4c9d1fb2c3198..0677f4500a82c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -58,10 +58,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class"; private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); - private static final String HOODIE_FINALIZE_WRITE_BEFORE_COMMIT = "hoodie.finalize.write.before.commit"; - private static final String DEFAULT_HOODIE_FINALIZE_WRITE_BEFORE_COMMIT = "false"; - private static final String FINALIZE_PARALLELISM = "hoodie.finalize.parallelism"; - private static final String DEFAULT_FINALIZE_PARALLELISM = "5"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "hoodie.copyonwrite.use.temp.folder"; + private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "false"; + private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; + private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = "5"; private HoodieWriteConfig(Properties props) { super(props); @@ -118,12 +118,12 @@ public String getWriteStatusClassName() { return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP); } - public boolean shouldFinalizeWrite() { - return Boolean.parseBoolean(props.getProperty(HOODIE_FINALIZE_WRITE_BEFORE_COMMIT)); + public boolean shouldUseTempFolderForCopyOnWrite() { + return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER)); } - public int getFinalizeParallelism() { - return Integer.parseInt(props.getProperty(FINALIZE_PARALLELISM)); + public int getFinalizeWriteParallelism() { + return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM)); } /** @@ -397,13 +397,13 @@ public Builder withWriteStatusClass(Class writeStatusClas return this; } - public Builder withFinalizeWrite(boolean shouldFinalizeWrite) { - props.setProperty(HOODIE_FINALIZE_WRITE_BEFORE_COMMIT, String.valueOf(shouldFinalizeWrite)); + public Builder withUseTempFolderCopyOnWrite(boolean shouldUseTempFolderCopyOnWrite) { + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER, String.valueOf(shouldUseTempFolderCopyOnWrite)); return this; } - public Builder withFinalizeParallelism(int parallelism) { - props.setProperty(FINALIZE_PARALLELISM, String.valueOf(parallelism)); + public Builder withFinalizeWriteParallelism(int parallelism) { + props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism)); return this; } @@ -430,10 +430,10 @@ public HoodieWriteConfig build() { HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING); setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); - setDefaultOnCondition(props, !props.containsKey(HOODIE_FINALIZE_WRITE_BEFORE_COMMIT), - HOODIE_FINALIZE_WRITE_BEFORE_COMMIT, DEFAULT_HOODIE_FINALIZE_WRITE_BEFORE_COMMIT); - setDefaultOnCondition(props, !props.containsKey(FINALIZE_PARALLELISM), - FINALIZE_PARALLELISM, DEFAULT_FINALIZE_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER), + HOODIE_COPYONWRITE_USE_TEMP_FOLDER, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER); + setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), + FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM); // Make sure the props is propagated setDefaultOnCondition(props, !isIndexConfigSet, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 18f5be319dd30..d4bb198ae8652 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -45,7 +45,7 @@ public class HoodieCreateHandle extends HoodieIOH private final WriteStatus status; private final HoodieStorageWriter storageWriter; private final Path path; - private final Path tempPath; + private Path tempPath = null; private long recordsWritten = 0; private long recordsDeleted = 0; @@ -58,10 +58,8 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, final int sparkPartitionId = TaskContext.getPartitionId(); this.path = makeNewPath(partitionPath, sparkPartitionId, status.getFileId()); - if (config.shouldFinalizeWrite()) { + if (config.shouldUseTempFolderForCopyOnWrite()) { this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(), TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); - } else { - this.tempPath = null; } try { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 699d772d0aef4..c03f75f789c39 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -43,6 +43,7 @@ import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -575,14 +576,33 @@ public List rollback(JavaSparkContext jsc, List comm return stats; } + @Override + public void initializeFinalizeWrite() { + if (!config.shouldUseTempFolderForCopyOnWrite()) { + return; + } + + // create temporary folder if needed + final FileSystem fs = FSUtils.getFs(); + final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); + try { + if (!fs.exists(temporaryFolder)) { + fs.mkdirs(temporaryFolder); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to create temporary folder: " + temporaryFolder); + } + } + @Override @SuppressWarnings("unchecked") public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { - if (!config.shouldFinalizeWrite()) { + if (!config.shouldUseTempFolderForCopyOnWrite()) { return Optional.empty(); } - List> results = jsc.parallelize(writeStatuses, config.getFinalizeParallelism()) + // This is to rename each data file from temporary path to its final location + List> results = jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism()) .map(writeStatus -> { Tuple2 writeStatTuple2 = (Tuple2) writeStatus; HoodieWriteStat writeStat = writeStatTuple2._2(); @@ -610,6 +630,44 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) return Optional.of(results.size()); } + @Override + public void cleanTemporaryDataFiles(JavaSparkContext jsc) { + if (!config.shouldUseTempFolderForCopyOnWrite()) { + return; + } + + final FileSystem fs = FSUtils.getFs(); + final Path temporaryFolder = new Path(config.getBasePath(), + HoodieTableMetaClient.TEMPFOLDER_NAME); + try { + if (!fs.exists(temporaryFolder)) { + logger.info("Temporary folder does not exist: " + temporaryFolder); + return; + } + List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); + List> results = jsc + .parallelize(fileStatusesList, config.getFinalizeWriteParallelism()) + .map(fileStatus -> { + FileSystem fs1 = FSUtils.getFs(); + boolean success = fs1.delete(fileStatus.getPath(), false); + logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + + success); + return new Tuple2<>(fileStatus.getPath().toString(), success); + }).collect(); + + for (Tuple2 result : results) { + if (!result._2()) { + logger.info("Failed to delete file: " + result._1()); + throw new HoodieIOException( + "Failed to delete file in temporary folder: " + result._1()); + } + } + } catch (IOException e) { + throw new HoodieIOException( + "Failed to clean data files in temporary folder: " + temporaryFolder); + } + } + private static class PartitionCleanStat implements Serializable { private final String partitionPath; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 1e507c6453429..e10dde6694e75 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -250,9 +250,19 @@ public List rollback(JavaSparkContext jsc, List comm return allRollbackStats; } + @Override + public void initializeFinalizeWrite() { + // do nothing for MOR tables + } + @Override public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { // do nothing for MOR tables return Optional.empty(); } + + @Override + public void cleanTemporaryDataFiles(JavaSparkContext jsc) { + // do nothing for MOR tables + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index c3b9d1f702298..ca9f3776246ad 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -273,6 +273,11 @@ public abstract Optional compact(JavaSparkContext jsc, public abstract List rollback(JavaSparkContext jsc, List commits) throws IOException; + /** + * Initialize resources needed for finalize write. + */ + public abstract void initializeFinalizeWrite(); + /** * Finalize the written data files * @@ -280,4 +285,9 @@ public abstract List rollback(JavaSparkContext jsc, List finalizeWrite(JavaSparkContext jsc, List> writeStatuses); + + /** + * Clean temporary data files after data files are finalized or commit is rolled back. + */ + public abstract void cleanTemporaryDataFiles(JavaSparkContext jsc); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 10773db0f1d7a..286dcb7ba0f8d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -305,7 +305,7 @@ public void testUpserts() throws Exception { @Test public void testUpsertsWithFinalizeWrite() throws Exception { HoodieWriteConfig cfg = getConfigBuilder() - .withFinalizeWrite(true) + .withUseTempFolderCopyOnWrite(true) .build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); From 13a4a53a178405b16277ab2b01ccb9f5f37a51e2 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 14 Dec 2017 13:36:32 -0800 Subject: [PATCH 011/374] Incorporating code review feedback for finalizeWrite for COW #2 --- .../com/uber/hoodie/HoodieWriteClient.java | 1 - .../uber/hoodie/io/HoodieCreateHandle.java | 6 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 18 ------ .../hoodie/table/HoodieMergeOnReadTable.java | 5 -- .../com/uber/hoodie/table/HoodieTable.java | 5 -- .../hoodie/common/model/HoodieWriteStat.java | 11 ++++ .../common/table/HoodieTableMetaClient.java | 5 ++ .../common/model/TestHoodieWriteStat.java | 62 +++++++++++++++++++ 8 files changed, 79 insertions(+), 34 deletions(-) create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index ddde0b9642858..230a60324a19b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -848,7 +848,6 @@ public void startCommitWithTime(String commitTime) { String commitActionType = table.getCommitActionType(); activeTimeline.createInflight( new HoodieInstant(true, commitActionType, commitTime)); - table.initializeFinalizeWrite(); } /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index d4bb198ae8652..8d933d40cdfd0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -130,11 +130,7 @@ public WriteStatus close() { stat.setNumDeletes(recordsDeleted); stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); stat.setFileId(status.getFileId()); - String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", ""); - stat.setPath(relativePath); - if (tempPath != null) { - stat.setTempPath(tempPath.toString().replace(new Path(config.getBasePath()) + "/", "")); - } + stat.setPaths(new Path(config.getBasePath()), path, tempPath); stat.setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath())); stat.setTotalWriteErrors(status.getFailedRecords().size()); status.setStat(stat); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index c03f75f789c39..511c867fadcd2 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -576,24 +576,6 @@ public List rollback(JavaSparkContext jsc, List comm return stats; } - @Override - public void initializeFinalizeWrite() { - if (!config.shouldUseTempFolderForCopyOnWrite()) { - return; - } - - // create temporary folder if needed - final FileSystem fs = FSUtils.getFs(); - final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); - try { - if (!fs.exists(temporaryFolder)) { - fs.mkdirs(temporaryFolder); - } - } catch (IOException e) { - throw new HoodieIOException("Failed to create temporary folder: " + temporaryFolder); - } - } - @Override @SuppressWarnings("unchecked") public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index e10dde6694e75..9cab45bc7b36c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -250,11 +250,6 @@ public List rollback(JavaSparkContext jsc, List comm return allRollbackStats; } - @Override - public void initializeFinalizeWrite() { - // do nothing for MOR tables - } - @Override public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { // do nothing for MOR tables diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index ca9f3776246ad..c395450bb66b5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -273,11 +273,6 @@ public abstract Optional compact(JavaSparkContext jsc, public abstract List rollback(JavaSparkContext jsc, List commits) throws IOException; - /** - * Initialize resources needed for finalize write. - */ - public abstract void initializeFinalizeWrite(); - /** * Finalize the written data files * diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index c8f241062f4c3..dccfd3105e079 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -20,6 +20,7 @@ import javax.annotation.Nullable; import java.io.Serializable; +import org.apache.hadoop.fs.Path; /** * Statistics about a single Hoodie write operation. @@ -212,6 +213,16 @@ public String getTempPath() { return this.tempPath; } + /** + * Set path and tempPath relative to the given basePath. + */ + public void setPaths(Path basePath, Path path, Path tempPath) { + this.path = path.toString().replace(basePath + "/", ""); + if (tempPath != null) { + this.tempPath = tempPath.toString().replace(basePath + "/", ""); + } + } + @Override public String toString() { return new StringBuilder() diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index c776abf381f77..4c5168f60dd60 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -233,6 +233,11 @@ public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs, } } + // Always create temporaryFolder which is needed for finalizeWrite for Hoodie tables + final Path temporaryFolder = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME); + if (!fs.exists(temporaryFolder)) { + fs.mkdirs(temporaryFolder); + } HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java new file mode 100644 index 0000000000000..77c7d21def0bc --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java @@ -0,0 +1,62 @@ +/* + * 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.common.model; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.util.FSUtils; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.UUID; +import org.apache.hadoop.fs.Path; +import org.junit.Test; + +public class TestHoodieWriteStat { + + @Test + public void testSetPaths() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + String basePathString = "/data/tables/some-hoodie-table"; + String partitionPathString = "2017/12/31"; + String fileName = UUID.randomUUID().toString(); + int taskPartitionId = Integer.MAX_VALUE; + int stageId = Integer.MAX_VALUE; + long taskAttemptId = Long.MAX_VALUE; + + Path basePath = new Path(basePathString); + Path partitionPath = new Path(basePath, partitionPathString); + Path tempPath = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME); + + Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, + taskPartitionId, fileName)); + Path tempFilePath = new Path(tempPath, FSUtils.makeTempDataFileName(partitionPathString, + commitTime, taskPartitionId, fileName, stageId, taskAttemptId)); + + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPaths(basePath, finalizeFilePath, tempFilePath); + assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath())); + assertEquals(tempFilePath, new Path(basePath, writeStat.getTempPath())); + + // test for null tempFilePath + writeStat = new HoodieWriteStat(); + writeStat.setPaths(basePath, finalizeFilePath, null); + assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath())); + assertNull(writeStat.getTempPath()); + } +} From 5c69ecf0d519728494bbe1a173c6ec11825d1b0c Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 21 Dec 2017 09:58:51 -0800 Subject: [PATCH 012/374] Incorporating code review feedback for finalizeWrite for COW #3 --- .../src/main/java/com/uber/hoodie/HoodieWriteClient.java | 6 ------ .../java/com/uber/hoodie/config/HoodieWriteConfig.java | 2 +- .../com/uber/hoodie/table/HoodieCopyOnWriteTable.java | 9 +++++++-- .../src/main/java/com/uber/hoodie/table/HoodieTable.java | 4 ---- 4 files changed, 8 insertions(+), 13 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 230a60324a19b..df80ba58748d8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -440,9 +440,6 @@ public boolean commit(String commitTime, ); } - // Clean temp files - table.cleanTemporaryDataFiles(jsc); - // add in extra metadata if (extraMetadata.isPresent()) { extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v)); @@ -698,9 +695,6 @@ private void rollback(List commits) { } }); - // clean data files in temporary folder - table.cleanTemporaryDataFiles(jsc); - try { if (commitTimeline.empty() && inflightTimeline.empty()) { // nothing to rollback diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 0677f4500a82c..082ececdb3048 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -61,7 +61,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "hoodie.copyonwrite.use.temp.folder"; private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "false"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; - private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = "5"; + private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; private HoodieWriteConfig(Properties props) { super(props); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 511c867fadcd2..4bcd1d292b210 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -569,6 +569,9 @@ public List rollback(JavaSparkContext jsc, List comm .withDeletedFileResults(results).build(); }).collect(); + // clean temporary data files + cleanTemporaryDataFiles(jsc); + // Remove the rolled back inflight commits commits.stream().map(s -> new HoodieInstant(true, actionType, s)) .forEach(activeTimeline::deleteInflight); @@ -609,11 +612,13 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) return new Tuple2<>(writeStat.getPath(), true); }).collect(); + // clean temporary data files + cleanTemporaryDataFiles(jsc); + return Optional.of(results.size()); } - @Override - public void cleanTemporaryDataFiles(JavaSparkContext jsc) { + private void cleanTemporaryDataFiles(JavaSparkContext jsc) { if (!config.shouldUseTempFolderForCopyOnWrite()) { return; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index c395450bb66b5..06eec8e458d2e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -281,8 +281,4 @@ public abstract List rollback(JavaSparkContext jsc, List finalizeWrite(JavaSparkContext jsc, List> writeStatuses); - /** - * Clean temporary data files after data files are finalized or commit is rolled back. - */ - public abstract void cleanTemporaryDataFiles(JavaSparkContext jsc); } From e8f947edd47255e21e17cb7904e99ba90f0cba6e Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 18 Jan 2018 11:29:10 -0800 Subject: [PATCH 013/374] Add finalizeWrite support for HoodieMergeHandle --- .../uber/hoodie/config/HoodieWriteConfig.java | 13 ++++++++++ .../com/uber/hoodie/io/HoodieMergeHandle.java | 26 ++++++++++++++----- .../hoodie/table/HoodieMergeOnReadTable.java | 5 ---- .../TestHoodieClientOnCopyOnWriteStorage.java | 1 + 4 files changed, 33 insertions(+), 12 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 082ececdb3048..4f19ee679844f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -60,6 +60,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "hoodie.copyonwrite.use.temp.folder"; private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "false"; + private static final String HOODIE_MERGEHANDLE_USE_TEMP_FOLDER = "hoodie.mergehandle.use.temp.folder"; + private static final String DEFAULT_HOODIE_MERGEHANDLE_USE_TEMP_FOLDER = "false"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; @@ -122,6 +124,10 @@ public boolean shouldUseTempFolderForCopyOnWrite() { return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER)); } + public boolean shouldUseTempFolderForMergeHandle() { + return Boolean.parseBoolean(props.getProperty(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER)); + } + public int getFinalizeWriteParallelism() { return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM)); } @@ -402,6 +408,11 @@ public Builder withUseTempFolderCopyOnWrite(boolean shouldUseTempFolderCopyOnWri return this; } + public Builder withUseTempFolderMergeHandle(boolean shouldUseTempFolderMergeHandle) { + props.setProperty(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER, String.valueOf(shouldUseTempFolderMergeHandle)); + return this; + } + public Builder withFinalizeWriteParallelism(int parallelism) { props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism)); return this; @@ -432,6 +443,8 @@ public HoodieWriteConfig build() { HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER), HOODIE_COPYONWRITE_USE_TEMP_FOLDER, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER); + setDefaultOnCondition(props, !props.containsKey(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER), + HOODIE_MERGEHANDLE_USE_TEMP_FOLDER, DEFAULT_HOODIE_MERGEHANDLE_USE_TEMP_FOLDER); setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index e6ce314682317..35706c630125a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -52,6 +52,7 @@ public class HoodieMergeHandle extends HoodieIOHa private TableFileSystemView.ReadOptimizedView fileSystemView; private Path newFilePath; private Path oldFilePath; + private Path tempPath = null; private long recordsWritten = 0; private long recordsDeleted = 0; private long updatedRecordsWritten = 0; @@ -100,6 +101,9 @@ private void init(String fileId, Iterator> newRecordsItr) { String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); newFilePath = new Path(config.getBasePath(), relativePath); + if (config.shouldUseTempFolderForCopyOnWrite() && config.shouldUseTempFolderForMergeHandle()) { + this.tempPath = makeTempPath(record.getPartitionPath(), TaskContext.getPartitionId(), fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); + } // handle cases of partial failures, for update task if (fs.exists(newFilePath)) { @@ -107,12 +111,12 @@ private void init(String fileId, Iterator> newRecordsItr) { } logger.info(String.format("Merging new data into oldPath %s, as newPath %s", - oldFilePath.toString(), newFilePath.toString())); + oldFilePath.toString(), getNewFilePath().toString())); // file name is same for all records, in this bunch writeStatus.setFileId(fileId); writeStatus.setPartitionPath(record.getPartitionPath()); writeStatus.getStat().setFileId(fileId); - writeStatus.getStat().setPath(relativePath); + writeStatus.getStat().setPaths(new Path(config.getBasePath()), newFilePath, tempPath); } keyToNewRecords.put(record.getRecordKey(), record); // update the new location of the record, so we know where to find it next @@ -120,7 +124,7 @@ private void init(String fileId, Iterator> newRecordsItr) { } // Create the writer for writing the new version file storageWriter = HoodieStorageWriterFactory - .getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema); + .getStorageWriter(commitTime, getNewFilePath(), hoodieTable, config, schema); } catch (Exception e) { logger.error("Error in update task at commit " + commitTime, e); @@ -186,18 +190,18 @@ public void write(GenericRecord oldRecord) { if (copyOldRecord) { // this should work as it is, since this is an existing record String errMsg = "Failed to merge old record into new file for key " + key + " from old file " - + getOldFilePath() + " to new file " + newFilePath; + + getOldFilePath() + " to new file " + getNewFilePath(); try { storageWriter.writeAvro(key, oldRecord); } catch (ClassCastException e) { logger.error( "Schema mismatch when rewriting old record " + oldRecord + " from file " - + getOldFilePath() + " to file " + newFilePath + " with schema " + schema + + getOldFilePath() + " to file " + getNewFilePath() + " with schema " + schema .toString(true)); throw new HoodieUpsertException(errMsg, e); } catch (IOException e) { logger.error("Failed to merge old record into new file for key " + key + " from old file " - + getOldFilePath() + " to new file " + newFilePath, e); + + getOldFilePath() + " to new file " + getNewFilePath(), e); throw new HoodieUpsertException(errMsg, e); } recordsWritten++; @@ -219,7 +223,7 @@ public void close() { storageWriter.close(); } - writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath)); + writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, getNewFilePath())); writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten); @@ -233,6 +237,14 @@ public Path getOldFilePath() { return oldFilePath; } + private Path getNewFilePath() { + // Use tempPath for storage writer if possible + if (this.tempPath != null) { + return this.tempPath; + } + return this.newFilePath; + } + public WriteStatus getWriteStatus() { return writeStatus; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 9cab45bc7b36c..1e507c6453429 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -255,9 +255,4 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) // do nothing for MOR tables return Optional.empty(); } - - @Override - public void cleanTemporaryDataFiles(JavaSparkContext jsc) { - // do nothing for MOR tables - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 286dcb7ba0f8d..a1822bdfca558 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -306,6 +306,7 @@ public void testUpserts() throws Exception { public void testUpsertsWithFinalizeWrite() throws Exception { HoodieWriteConfig cfg = getConfigBuilder() .withUseTempFolderCopyOnWrite(true) + .withUseTempFolderMergeHandle(true) .build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); From cdb5fc5ffab3171e79ddb7254af718368e23d1e1 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 18 Jan 2018 12:39:30 -0800 Subject: [PATCH 014/374] Rebases with latest upstream --- .../com/uber/hoodie/table/HoodieCopyOnWriteTable.java | 6 +++--- .../hoodie/TestHoodieClientOnCopyOnWriteStorage.java | 10 ++++------ 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 4bcd1d292b210..f13336f0e4a3c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -591,7 +591,7 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) .map(writeStatus -> { Tuple2 writeStatTuple2 = (Tuple2) writeStatus; HoodieWriteStat writeStat = writeStatTuple2._2(); - final FileSystem fs = FSUtils.getFs(); + final FileSystem fs = getMetaClient().getFs(); final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); if (writeStat.getTempPath() != null) { @@ -623,7 +623,7 @@ private void cleanTemporaryDataFiles(JavaSparkContext jsc) { return; } - final FileSystem fs = FSUtils.getFs(); + final FileSystem fs = getMetaClient().getFs(); final Path temporaryFolder = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); try { @@ -635,7 +635,7 @@ private void cleanTemporaryDataFiles(JavaSparkContext jsc) { List> results = jsc .parallelize(fileStatusesList, config.getFinalizeWriteParallelism()) .map(fileStatus -> { - FileSystem fs1 = FSUtils.getFs(); + FileSystem fs1 = getMetaClient().getFs(); boolean success = fs1.delete(fileStatus.getPath(), false); logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + success); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index a1822bdfca558..4c158e27d73a2 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -310,7 +310,6 @@ public void testUpsertsWithFinalizeWrite() throws Exception { .build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - FileSystem fs = FSUtils.getFs(); /** * Write 1 (only inserts) @@ -328,9 +327,8 @@ public void testUpsertsWithFinalizeWrite() throws Exception { assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) - .getCommitTimeline(); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); @@ -366,13 +364,13 @@ public void testUpsertsWithFinalizeWrite() throws Exception { assertNoWriteErrors(statuses); // verify there are now 2 commits - timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline(); + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); - metaClient = new HoodieTableMetaClient(fs, basePath); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, getConfig()); // Index should be able to locate all updates in correct locations. From 7a392a3cad6cbfbe49cc1e0da0e9c740e9fa1d1a Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Tue, 30 Jan 2018 11:18:00 -0800 Subject: [PATCH 015/374] Incorporating code review feedback for finalizeWrite for COW #4 --- .../uber/hoodie/config/HoodieWriteConfig.java | 41 +++-- .../uber/hoodie/io/HoodieCreateHandle.java | 10 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 21 ++- .../hoodie/table/HoodieCopyOnWriteTable.java | 10 ++ .../com/uber/hoodie/table/HoodieTable.java | 1 - .../TestHoodieClientOnCopyOnWriteStorage.java | 141 +++++++----------- 6 files changed, 98 insertions(+), 126 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 4f19ee679844f..b6d5f766b1639 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -58,10 +58,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class"; private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); - private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "hoodie.copyonwrite.use.temp.folder"; - private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER = "false"; - private static final String HOODIE_MERGEHANDLE_USE_TEMP_FOLDER = "hoodie.mergehandle.use.temp.folder"; - private static final String DEFAULT_HOODIE_MERGEHANDLE_USE_TEMP_FOLDER = "false"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "hoodie.copyonwrite.use.temp.folder.for.create"; + private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "false"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "hoodie.copyonwrite.use.temp.folder.for.merge"; + private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "false"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; @@ -120,12 +120,17 @@ public String getWriteStatusClassName() { return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP); } - public boolean shouldUseTempFolderForCopyOnWrite() { - return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER)); + public boolean shouldUseTempFolderForCopyOnWriteForCreate() { + return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE)); + } + + public boolean shouldUseTempFolderForCopyOnWriteForMerge() { + return Boolean.parseBoolean(props.getProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE)); } - public boolean shouldUseTempFolderForMergeHandle() { - return Boolean.parseBoolean(props.getProperty(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER)); + public boolean shouldUseTempFolderForCopyOnWrite() { + return shouldUseTempFolderForCopyOnWriteForCreate() || + shouldUseTempFolderForCopyOnWriteForMerge(); } public int getFinalizeWriteParallelism() { @@ -403,13 +408,17 @@ public Builder withWriteStatusClass(Class writeStatusClas return this; } - public Builder withUseTempFolderCopyOnWrite(boolean shouldUseTempFolderCopyOnWrite) { - props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER, String.valueOf(shouldUseTempFolderCopyOnWrite)); + public Builder withUseTempFolderCopyOnWriteForCreate( + boolean shouldUseTempFolderCopyOnWriteForCreate) { + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, String.valueOf + (shouldUseTempFolderCopyOnWriteForCreate)); return this; } - public Builder withUseTempFolderMergeHandle(boolean shouldUseTempFolderMergeHandle) { - props.setProperty(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER, String.valueOf(shouldUseTempFolderMergeHandle)); + public Builder withUseTempFolderCopyOnWriteForMerge( + boolean shouldUseTempFolderCopyOnWriteForMerge) { + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, String.valueOf + (shouldUseTempFolderCopyOnWriteForMerge)); return this; } @@ -441,10 +450,10 @@ public HoodieWriteConfig build() { HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING); setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); - setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER), - HOODIE_COPYONWRITE_USE_TEMP_FOLDER, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER); - setDefaultOnCondition(props, !props.containsKey(HOODIE_MERGEHANDLE_USE_TEMP_FOLDER), - HOODIE_MERGEHANDLE_USE_TEMP_FOLDER, DEFAULT_HOODIE_MERGEHANDLE_USE_TEMP_FOLDER); + setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE), + HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE); + setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE), + HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE); setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 8d933d40cdfd0..1fa52de256e49 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -58,8 +58,9 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, final int sparkPartitionId = TaskContext.getPartitionId(); this.path = makeNewPath(partitionPath, sparkPartitionId, status.getFileId()); - if (config.shouldUseTempFolderForCopyOnWrite()) { - this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(), TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); + if (config.shouldUseTempFolderForCopyOnWriteForCreate()) { + this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(), + TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); } try { @@ -144,9 +145,6 @@ public WriteStatus close() { private Path getStorageWriterPath() { // Use tempPath for storage writer if possible - if (this.tempPath != null) { - return this.tempPath; - } - return this.path; + return (this.tempPath == null) ? this.path : this.tempPath; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 35706c630125a..7500abf3aeeb9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -101,7 +101,7 @@ private void init(String fileId, Iterator> newRecordsItr) { String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); newFilePath = new Path(config.getBasePath(), relativePath); - if (config.shouldUseTempFolderForCopyOnWrite() && config.shouldUseTempFolderForMergeHandle()) { + if (config.shouldUseTempFolderForCopyOnWriteForMerge()) { this.tempPath = makeTempPath(record.getPartitionPath(), TaskContext.getPartitionId(), fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); } @@ -111,7 +111,7 @@ private void init(String fileId, Iterator> newRecordsItr) { } logger.info(String.format("Merging new data into oldPath %s, as newPath %s", - oldFilePath.toString(), getNewFilePath().toString())); + oldFilePath.toString(), getStorageWriterPath().toString())); // file name is same for all records, in this bunch writeStatus.setFileId(fileId); writeStatus.setPartitionPath(record.getPartitionPath()); @@ -124,7 +124,7 @@ private void init(String fileId, Iterator> newRecordsItr) { } // Create the writer for writing the new version file storageWriter = HoodieStorageWriterFactory - .getStorageWriter(commitTime, getNewFilePath(), hoodieTable, config, schema); + .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); } catch (Exception e) { logger.error("Error in update task at commit " + commitTime, e); @@ -190,18 +190,18 @@ public void write(GenericRecord oldRecord) { if (copyOldRecord) { // this should work as it is, since this is an existing record String errMsg = "Failed to merge old record into new file for key " + key + " from old file " - + getOldFilePath() + " to new file " + getNewFilePath(); + + getOldFilePath() + " to new file " + getStorageWriterPath(); try { storageWriter.writeAvro(key, oldRecord); } catch (ClassCastException e) { logger.error( "Schema mismatch when rewriting old record " + oldRecord + " from file " - + getOldFilePath() + " to file " + getNewFilePath() + " with schema " + schema + + getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema .toString(true)); throw new HoodieUpsertException(errMsg, e); } catch (IOException e) { logger.error("Failed to merge old record into new file for key " + key + " from old file " - + getOldFilePath() + " to new file " + getNewFilePath(), e); + + getOldFilePath() + " to new file " + getStorageWriterPath(), e); throw new HoodieUpsertException(errMsg, e); } recordsWritten++; @@ -223,7 +223,7 @@ public void close() { storageWriter.close(); } - writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, getNewFilePath())); + writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath())); writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten); @@ -237,12 +237,9 @@ public Path getOldFilePath() { return oldFilePath; } - private Path getNewFilePath() { + private Path getStorageWriterPath() { // Use tempPath for storage writer if possible - if (this.tempPath != null) { - return this.tempPath; - } - return this.newFilePath; + return (this.tempPath == null) ? this.newFilePath : this.tempPath; } public WriteStatus getWriteStatus() { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index f13336f0e4a3c..cb706a38975b2 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -579,6 +579,12 @@ public List rollback(JavaSparkContext jsc, List comm return stats; } + /** + * Finalize the written data files + * + * @param writeStatuses List of WriteStatus + * @return number of files finalized + */ @Override @SuppressWarnings("unchecked") public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { @@ -618,6 +624,10 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) return Optional.of(results.size()); } + /** + * Clean temporary data files that are produced from previous failed commit or retried spark + * stages. + */ private void cleanTemporaryDataFiles(JavaSparkContext jsc) { if (!config.shouldUseTempFolderForCopyOnWrite()) { return; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 06eec8e458d2e..c3b9d1f702298 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -280,5 +280,4 @@ public abstract List rollback(JavaSparkContext jsc, List finalizeWrite(JavaSparkContext jsc, List> writeStatuses); - } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 4c158e27d73a2..5caeec23b4347 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -48,6 +48,7 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; import java.io.File; import java.io.FileInputStream; @@ -55,6 +56,7 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -211,9 +213,12 @@ public void testAutoCommit() throws Exception { @Test public void testUpserts() throws Exception { - HoodieWriteConfig cfg = getConfig(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); + testUpsertsInternal(getConfig()); + } + + private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exception { + HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig); + HoodieIndex index = HoodieIndex.createIndex(hoodieWriteConfig, jsc); /** * Write 1 (only inserts) @@ -304,95 +309,11 @@ public void testUpserts() throws Exception { @Test public void testUpsertsWithFinalizeWrite() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder() - .withUseTempFolderCopyOnWrite(true) - .withUseTempFolderMergeHandle(true) + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder() + .withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(true) .build(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - - /** - * Write 1 (only inserts) - */ - String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - - List records = dataGen.generateInserts(newCommitTime, 200); - JavaRDD writeRecords = jsc.parallelize(records, 1); - - List statuses = client.upsert(writeRecords, newCommitTime).collect(); - assertNoWriteErrors(statuses); - - // check the partition metadata is written out - assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); - - // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, - timeline.lastInstant().get().getTimestamp()); - assertEquals("Must contain 200 records", - records.size(), - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); - // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) - .collect(); - checkTaggedRecords(taggedRecords, "001"); - - /** - * Write 2 (updates) - */ - newCommitTime = "004"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, 100); - LinkedHashMap recordsMap = new LinkedHashMap<>(); - for (HoodieRecord rec : records) { - if (!recordsMap.containsKey(rec.getKey())) { - recordsMap.put(rec.getKey(), rec); - } - } - List dedupedRecords = new ArrayList<>(recordsMap.values()); - - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - // verify there are now 2 commits - timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), - newCommitTime); - - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); - - // Index should be able to locate all updates in correct locations. - taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), table).collect(); - checkTaggedRecords(taggedRecords, "004"); - - // Check the entire dataset has 100 records still - String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; - for (int i = 0; i < fullPartitionPaths.length; i++) { - fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); - } - assertEquals("Must contain 200 records", - 200, - HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); - - // Check that the incremental consumption from time 000 - assertEquals("Incremental consumption from time 002, should give all records in commit 004", - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "002").count()); - assertEquals("Incremental consumption from time 001, should give all records in commit 004", - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); + testUpsertsInternal(hoodieWriteConfig); } @Test @@ -1575,6 +1496,32 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { .count() == 3); } + @Test + public void testCleanTemporaryDataFiles() throws IOException { + HoodieTestUtils.createCommitFiles(basePath, "000"); + List tempFiles = createTempFiles("000", 10); + assertEquals("Some temp files are created.",10, tempFiles.size()); + assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles()); + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withUseTempFolderCopyOnWriteForCreate(false) + .withUseTempFolderCopyOnWriteForMerge(false).build(); + HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true), + config); + table.rollback(jsc, Collections.emptyList()); + assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles()); + + config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(false).build(); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true), + config); + table.rollback(jsc, Collections.emptyList()); + assertEquals("All temp files are deleted.",0, getTotalTempFiles()); + } + public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); @@ -1642,6 +1589,18 @@ private List createFilesInPartition(String partitionPath, String commitT return files; } + private List createTempFiles(String commitTime, int numFiles) throws IOException { + List files = new ArrayList<>(); + for (int i = 0; i < numFiles; i++) { + files.add(HoodieTestUtils.createNewDataFile(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, commitTime)); + } + return files; + } + + private int getTotalTempFiles() throws IOException { + return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length; + } + @After public void clean() { if (basePath != null) { From 1fb940b6e3fd6e249d77c9b8ac52e1df409b0a76 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 15 Dec 2017 21:29:02 -0800 Subject: [PATCH 016/374] Adding global indexing to HbaseIndex implementation - Adding tests or HbaseIndex - Enabling global index functionality --- hoodie-client/pom.xml | 30 +- .../uber/hoodie/config/HoodieIndexConfig.java | 19 +- .../uber/hoodie/config/HoodieWriteConfig.java | 8 + .../uber/hoodie/index/hbase/HBaseIndex.java | 189 ++++++++--- .../com/uber/hoodie/index/TestHbaseIndex.java | 318 ++++++++++++++++++ 5 files changed, 508 insertions(+), 56 deletions(-) create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index a3b1ec640ef92..d082f2f60b172 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -191,6 +191,34 @@ test - + + + org.apache.hbase + hbase-client + 1.2.3 + + + + org.htrace + htrace-core + 3.0.4 + + + org.apache.hbase + hbase-testing-util + 1.2.3 + test + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java index a7a722de1bb5d..a84c57b11537c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java @@ -49,6 +49,9 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; + public final static String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; + public final static String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; + public final static String DEFAULT_HBASE_BATCH_SIZE = "100"; // ***** Bucketed Index Configs ***** public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets"; @@ -130,6 +133,16 @@ public Builder numBucketsPerPartition(int numBuckets) { return this; } + public Builder hbaseIndexGetBatchSize(int getBatchSize) { + props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize)); + return this; + } + + public Builder hbaseIndexPutBatchSize(int putBatchSize) { + props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize)); + return this; + } + public HoodieIndexConfig build() { HoodieIndexConfig config = new HoodieIndexConfig(props); setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), @@ -144,9 +157,13 @@ public HoodieIndexConfig build() { BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP), BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING); + setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP), + HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); + setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), + HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); // Throws IllegalArgumentException if the value set is not a known Hoodie Index Type HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP)); return config; } } -} +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index b6d5f766b1639..9bbc8fa705ff1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -237,6 +237,14 @@ public String getHbaseTableName() { return props.getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP); } + public int getHbaseIndexGetBatchSize() { + return Integer.valueOf(props.getProperty(HoodieIndexConfig.HBASE_GET_BATCH_SIZE_PROP)); + } + + public int getHbaseIndexPutBatchSize() { + return Integer.valueOf(props.getProperty(HoodieIndexConfig.HBASE_PUT_BATCH_SIZE_PROP)); + } + public int getBloomIndexParallelism() { return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP)); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java index 5d50ff646b749..e2542925c87fb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java @@ -18,6 +18,7 @@ package com.uber.hoodie.index.hbase; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieKey; @@ -32,10 +33,6 @@ import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -54,11 +51,16 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function2; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + /** * Hoodie Index implementation backed by HBase */ public class HBaseIndex extends HoodieIndex { - private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s"); private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts"); private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name"); @@ -70,22 +72,24 @@ public class HBaseIndex extends HoodieIndex { public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { super(config, jsc); - this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP); + this.tableName = config.getHbaseTableName(); + addShutDownHook(); } @Override public JavaPairRDD> fetchRecordLocation( JavaRDD hoodieKeys, HoodieTable table) { - throw new UnsupportedOperationException("HBase index does not implement check exist yet"); + //TODO : Change/Remove filterExists in HoodieReadClient() and revisit + throw new UnsupportedOperationException("HBase index does not implement check exist"); } private static Connection hbaseConnection = null; private Connection getHBaseConnection() { Configuration hbaseConfig = HBaseConfiguration.create(); - String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP); + String quorum = config.getHbaseZkQuorum(); hbaseConfig.set("hbase.zookeeper.quorum", quorum); - String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP); + String port = String.valueOf(config.getHbaseZkPort()); hbaseConfig.set("hbase.zookeeper.property.clientPort", port); try { return ConnectionFactory.createConnection(hbaseConfig); @@ -95,24 +99,53 @@ private Connection getHBaseConnection() { } } + /** + * Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is closed when + * JVM exits + */ + private void addShutDownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + try { + hbaseConnection.close(); + } catch(Exception e) { + // fail silently for any sort of exception + } + } + }); + } + + private Get generateStatement(String key) throws IOException { + return new Get(Bytes.toBytes(key)).setMaxVersions(1) + .addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN) + .addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN) + .addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN); + } + + private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) { + HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); + // Check if the last commit ts for this row is 1) present in the timeline or + // 2) is less than the first commit ts in the timeline + return !commitTimeline.empty() && (commitTimeline.containsInstant( + new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) || + HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), + commitTs, HoodieTimeline.GREATER)); + } + /** * Function that tags each HoodieRecord with an existing location, if known. */ - class LocationTagFunction - implements Function2>, Iterator>> { + private Function2>, Iterator>> + locationTagFunction(HoodieTable hoodieTable) { - private final HoodieTable hoodieTable; + return (Function2>, Iterator>>) + (partitionNum, hoodieRecordIterator) -> { - LocationTagFunction(HoodieTable hoodieTable) { - this.hoodieTable = hoodieTable; - } + Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize(); - @Override - public Iterator> call(Integer partitionNum, - Iterator> hoodieRecordIterator) { // Grab the global HBase connection synchronized (HBaseIndex.class) { - if (hbaseConnection == null) { + if (hbaseConnection == null || hbaseConnection.isClosed()) { hbaseConnection = getHBaseConnection(); } } @@ -120,31 +153,46 @@ public Iterator> call(Integer partitionNum, HTable hTable = null; try { hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName)); + List statements = new ArrayList<>(); + List currentBatchOfRecords = new LinkedList<>(); // Do the tagging. while (hoodieRecordIterator.hasNext()) { HoodieRecord rec = hoodieRecordIterator.next(); - // TODO(vc): This may need to be a multi get. - Result result = hTable.get( - new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1) - .addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN) - .addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN) - .addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - - // first, attempt to grab location from HBase - if (result.getRow() != null) { - String commitTs = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)); - String fileId = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)); + statements.add(generateStatement(rec.getRecordKey())); + currentBatchOfRecords.add(rec); + // iterator till we reach batch size + if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) { + // get results for batch from Hbase + Result[] results = hTable.get(statements); + // clear statements to be GC'd + statements.clear(); + for (Result result : results) { + // first, attempt to grab location from HBase + HoodieRecord currentRecord = currentBatchOfRecords.remove(0); + if (result.getRow() != null) { + String keyFromResult = Bytes.toString(result.getRow()); + String commitTs = + Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)); + String fileId = + Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)); + String partitionPath = + Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); - // if the last commit ts for this row is less than the system commit ts - if (!commitTimeline.empty() && commitTimeline.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) { - rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); + if (checkIfValidCommit(hoodieTable, commitTs)) { + currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), + partitionPath), currentRecord.getData()); + currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); + taggedRecords.add(currentRecord); + // the key from Result and the key being processed should be same + assert (currentRecord.getRecordKey().contentEquals(keyFromResult)); + } else { //if commit is invalid, treat this as a new taggedRecord + taggedRecords.add(currentRecord); + } + } else { + taggedRecords.add(currentRecord); + } } } - taggedRecords.add(rec); } } catch (IOException e) { throw new HoodieIndexException( @@ -160,25 +208,25 @@ public Iterator> call(Integer partitionNum, } return taggedRecords.iterator(); - } + }; } @Override public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) { - return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true); + HoodieTable hoodieTable) { + return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true); } - class UpdateLocationTask implements - Function2, Iterator> { + private Function2, Iterator> updateLocationFunction() { + + return (Function2, Iterator>) (partition, statusIterator) -> { - @Override - public Iterator call(Integer partition, Iterator statusIterator) { + Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); List writeStatusList = new ArrayList<>(); // Grab the global HBase connection synchronized (HBaseIndex.class) { - if (hbaseConnection == null) { + if (hbaseConnection == null || hbaseConnection.isClosed()) { hbaseConnection = getHBaseConnection(); } } @@ -194,6 +242,10 @@ public Iterator call(Integer partition, Iterator statu if (!writeStatus.isErrored(rec.getKey())) { java.util.Optional loc = rec.getNewLocation(); if (loc.isPresent()) { + if (rec.getCurrentLocation() != null) { + // This is an update, no need to update index + continue; + } Put put = new Put(Bytes.toBytes(rec.getRecordKey())); put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN, Bytes.toBytes(loc.get().getCommitTime())); @@ -208,10 +260,13 @@ public Iterator call(Integer partition, Iterator statu deletes.add(delete); } } + if (puts.size() + deletes.size() < multiPutBatchSize) { + continue; + } + doPutsAndDeletes(hTable, puts, deletes); } - hTable.put(puts); - hTable.delete(deletes); - hTable.flushCommits(); + //process remaining puts and deletes, if any + doPutsAndDeletes(hTable, puts, deletes); } catch (Exception e) { Exception we = new Exception("Error updating index for " + writeStatus, e); logger.error(we); @@ -232,24 +287,43 @@ public Iterator call(Integer partition, Iterator statu } } return writeStatusList.iterator(); + }; + } + + /** + * Helper method to facilitate performing puts and deletes in Hbase + * @param hTable + * @param puts + * @param deletes + * @throws IOException + */ + private void doPutsAndDeletes(HTable hTable, List puts, List deletes) throws IOException { + if(puts.size() > 0) { + hTable.put(puts); + } + if(deletes.size() > 0) { + hTable.delete(deletes); } + hTable.flushCommits(); + puts.clear(); + deletes.clear(); } @Override public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) { - return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true); + HoodieTable hoodieTable) { + return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); } @Override public boolean rollbackCommit(String commitTime) { - // Can't really rollback here. HBase only can let you go from recordKey to fileID, - // not the other way around + // Rollback in HbaseIndex is managed via method {@link #checkIfValidCommit()} return true; } /** * Only looks up by recordKey + * @return */ @Override public boolean isGlobal() { @@ -258,6 +332,7 @@ public boolean isGlobal() { /** * Mapping is available in HBase already. + * @return */ @Override public boolean canIndexLogFiles() { @@ -266,9 +341,15 @@ public boolean canIndexLogFiles() { /** * Index needs to be explicitly updated after storage write. + * @return */ @Override public boolean isImplicitWithStorage() { return false; } -} + + @VisibleForTesting + public void setHbaseConnection(Connection hbaseConnection) { + HBaseIndex.hbaseConnection = hbaseConnection; + } +} \ No newline at end of file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java new file mode 100644 index 0000000000000..990e62d611c44 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -0,0 +1,318 @@ +/* + * 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.index; + +import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.table.HoodieTableConfig; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieStorageConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.hbase.HBaseIndex; +import com.uber.hoodie.table.HoodieTable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runners.MethodSorters; +import org.mockito.Mockito; +import scala.Tuple2; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.times; + +/** + * Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown + * across tests, (see one problem here : https://issues.apache.org/jira/browse/HBASE-15835). + * Hence, the need to use MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter + * the order of tests running carefully. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class TestHbaseIndex { + + private static JavaSparkContext jsc = null; + private String basePath = null; + private transient FileSystem fs; + private static HBaseTestingUtility utility; + private static Configuration hbaseConfig; + private static String tableName = "test_table"; + private HoodieTableMetaClient metaClient; + + @AfterClass + public static void clean() throws Exception { + if (jsc != null) { + jsc.stop(); + } + if (utility != null) { + utility.shutdownMiniCluster(); + } + } + + @After + public void clear() throws Exception { + if (basePath != null) { + new File(basePath).delete(); + } + } + + @Before + public void before() throws Exception { + // Create a temp folder as the base path + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + basePath = folder.getRoot().getAbsolutePath(); + // Initialize table + metaClient = HoodieTableMetaClient + .initTableType(utility.getTestFileSystem(), basePath, HoodieTableType.COPY_ON_WRITE, + tableName, HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); + } + + public TestHbaseIndex() throws Exception { + } + + @BeforeClass + public static void init() throws Exception { + + // Initialize HbaseMiniCluster + utility = new HBaseTestingUtility(); + utility.startMiniCluster(); + hbaseConfig = utility.getConnection().getConfiguration(); + utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s")); + // Initialize a local spark env + SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]"); + jsc = new JavaSparkContext(sparkConf); + jsc.hadoopConfiguration().addResource(utility.getConfiguration()); + } + + @Test + public void testSimpleTagLocationAndUpdate() throws Exception { + + String newCommitTime = "001"; + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + List records = dataGen.generateInserts(newCommitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + // Load to memory + HoodieWriteConfig config = getConfig(); + HBaseIndex index = new HBaseIndex(config, jsc); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + writeClient.startCommit(); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + + // Test tagLocation without any entries in index + JavaRDD javaRDD = index.tagLocation(writeRecords, hoodieTable); + assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); + + // Insert 200 records + JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); + assertNoWriteErrors(writeStatues.collect()); + + // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed commit + javaRDD = index.tagLocation(writeRecords, hoodieTable); + assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); + + // Now commit this & update location of records inserted and validate no errors + writeClient.commit(newCommitTime, writeStatues); + + // Now tagLocation for these records, hbaseIndex should tag them correctly + javaRDD = index.tagLocation(writeRecords, hoodieTable); + assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); + assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); + assertTrue(javaRDD.filter(record -> (record.getCurrentLocation() != null + && record.getCurrentLocation().getCommitTime().equals(newCommitTime))).distinct().count() == 200); + + } + + @Test + public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + // Load to memory + HoodieWriteConfig config = getConfig(); + HBaseIndex index = new HBaseIndex(config, jsc); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + + String newCommitTime = writeClient.startCommit(); + List records = dataGen.generateInserts(newCommitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + + // Insert 200 records + JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); + assertNoWriteErrors(writeStatues.collect()); + + // commit this upsert + writeClient.commit(newCommitTime, writeStatues); + + // Now tagLocation for these records, hbaseIndex should tag them + JavaRDD javaRDD = index.tagLocation(writeRecords, hoodieTable); + assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); + + // check tagged records are tagged with correct fileIds + List fileIds = writeStatues.map(status -> status.getFileId()).collect(); + assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0); + List taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect(); + + // both lists should match + assertTrue(taggedFileIds.containsAll(fileIds) && fileIds.containsAll(taggedFileIds)); + // Rollback the last commit + writeClient.rollback(newCommitTime); + + // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled back commit + javaRDD = index.tagLocation(writeRecords, hoodieTable); + assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); + assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); + } + + @Test + public void testTotalGetsBatching() throws Exception { + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + HoodieWriteConfig config = getConfig(); + HBaseIndex index = new HBaseIndex(config, jsc); + + // Mock hbaseConnection and related entities + Connection hbaseConnection = Mockito.mock(Connection.class); + HTable table = Mockito.mock(HTable.class); + Mockito.when(hbaseConnection.getTable(TableName.valueOf(tableName))).thenReturn(table); + Mockito.when(table.get((List) anyObject())).thenReturn(new Result[0]); + + // only for test, set the hbaseConnection to mocked object + index.setHbaseConnection(hbaseConnection); + + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + + // start a commit and generate test data + String newCommitTime = writeClient.startCommit(); + List records = dataGen.generateInserts(newCommitTime, 250); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + + // Insert 250 records + JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); + assertNoWriteErrors(writeStatues.collect()); + + // Now tagLocation for these records, hbaseIndex should tag them + index.tagLocation(writeRecords, hoodieTable); + + // 3 batches should be executed given batchSize = 100 and parallelism = 1 + Mockito.verify(table, times(3)).get((List) anyObject()); + + } + + @Test + public void testTotalPutsBatching() throws Exception { + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + HoodieWriteConfig config = getConfig(); + HBaseIndex index = new HBaseIndex(config, jsc); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + + // start a commit and generate test data + String newCommitTime = writeClient.startCommit(); + List records = dataGen.generateInserts(newCommitTime, 250); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + + // Insert 200 records + JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); + + // commit this upsert + writeClient.commit(newCommitTime, writeStatues); + + // Mock hbaseConnection and related entities + Connection hbaseConnection = Mockito.mock(Connection.class); + HTable table = Mockito.mock(HTable.class); + Mockito.when(hbaseConnection.getTable(TableName.valueOf(tableName))).thenReturn(table); + Mockito.when(table.get((List) anyObject())).thenReturn(new Result[0]); + + // only for test, set the hbaseConnection to mocked object + index.setHbaseConnection(hbaseConnection); + + // Get all the files generated + int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count(); + + index.updateLocation(writeStatues, hoodieTable); + // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated, + // so each fileId ideally gets updates + Mockito.verify(table, atMost(numberOfDataFileIds)).put((List) anyObject()); + } + + private void assertNoWriteErrors(List statuses) { + // Verify there are no errors + for (WriteStatus status : statuses) { + assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors()); + } + } + + private HoodieWriteConfig getConfig() { + return getConfigBuilder().build(); + } + + private HoodieWriteConfig.Builder getConfigBuilder() { + return HoodieWriteConfig.newBuilder().withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(1, 1) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) + .withInlineCompaction(false).build()) + .withAutoCommit(false) + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .forTable("test-trip-table").withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE) + .hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort"))) + .hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName) + .hbaseIndexGetBatchSize(100).hbaseIndexPutBatchSize(100).build()); + } +} \ No newline at end of file From bc531b4678616c339a1154d75c17054ea0423fca Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Thu, 14 Dec 2017 21:34:54 -0800 Subject: [PATCH 017/374] Fixing Rollback for compaction/commit operation, added check for null commit - Fallback to old way of rollback by listing all partitions - Added null check to ensure only partitions which are to be rolledback are considered - Added location (committime) to workload stat - Added checks in CompactedScanner to guard against task retries - Introduce new logic for rollback (bounded by instant_time and target_instant time) - Reversed logfiles order --- .../com/uber/hoodie/HoodieWriteClient.java | 3 +- .../uber/hoodie/io/HoodieAppendHandle.java | 2 +- .../compact/HoodieRealtimeTableCompactor.java | 6 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 3 +- .../hoodie/table/HoodieMergeOnReadTable.java | 186 +++++++++--------- .../com/uber/hoodie/table/WorkloadStat.java | 8 +- .../hoodie/table/TestMergeOnReadTable.java | 3 +- .../log/HoodieCompactedLogRecordScanner.java | 122 ++++++++---- .../common/table/log/HoodieLogFormatTest.java | 175 ++++++++++++++-- .../realtime/HoodieRealtimeInputFormat.java | 2 +- 10 files changed, 348 insertions(+), 162 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index df80ba58748d8..e960e1be545b7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -318,7 +318,8 @@ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> { HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(entry.getKey()); - writeStat.setNumUpdateWrites(entry.getValue()); + writeStat.setPrevCommit(entry.getValue().getKey()); + writeStat.setNumUpdateWrites(entry.getValue().getValue()); metadata.addWriteStat(path.toString(), writeStat); }); }); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index fb107444d4543..9752dc4444e6b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -187,7 +187,7 @@ public void doAppend() { } } catch (Exception e) { throw new HoodieAppendException( - "Failed while appeding records to " + currentLogFile.getPath(), e); + "Failed while appending records to " + currentLogFile.getPath(), e); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 501e452c41bb8..13899246951d1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -23,6 +23,7 @@ import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -86,7 +87,8 @@ public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf .flatMap((FlatMapFunction) partitionPath -> fileSystemView .getLatestFileSlices(partitionPath) .map(s -> new CompactionOperation(s.getDataFile().get(), - partitionPath, s.getLogFiles().collect(Collectors.toList()), config)) + partitionPath, s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed()) + .collect(Collectors.toList()), config)) .filter(c -> !c.getDeltaFilePaths().isEmpty()) .collect(toList()).iterator()).collect(); log.info("Total of " + operations.size() + " compactions are retrieved"); @@ -144,7 +146,7 @@ private List executeCompaction(HoodieTable hoodieTable, HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); String maxInstantTime = metaClient.getActiveTimeline() .getTimelineOfActions( - Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, + Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) .filterCompletedInstants().lastInstant().get().getTimestamp(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index cb706a38975b2..019153293e284 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -54,6 +54,7 @@ import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -199,7 +200,7 @@ class UpsertPartitioner extends Partitioner { private void assignUpdates(WorkloadProfile profile) { // each update location gets a partition WorkloadStat gStat = profile.getGlobalStat(); - for (Map.Entry updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) { + for (Map.Entry> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) { addUpdateBucket(updateLocEntry.getKey()); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 1e507c6453429..e97108086be6a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -32,11 +32,20 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + import java.io.IOException; import java.io.UncheckedIOException; import java.util.Arrays; @@ -47,38 +56,36 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; /** * Implementation of a more real-time read-optimized Hoodie Table where - * + *

* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) * Merge with the smallest existing file, to expand it - * + *

+ *

* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the * log file into the base file. - * + *

+ *

* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an * attempted commit action + *

*/ public class HoodieMergeOnReadTable extends HoodieCopyOnWriteTable { private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class); - public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { + public HoodieMergeOnReadTable(HoodieWriteConfig config, + HoodieTableMetaClient metaClient) { super(config, metaClient); } @Override public Iterator> handleUpdate(String commitTime, String fileId, - Iterator> recordItr) throws IOException { + Iterator> recordItr) throws IOException { logger.info("Merging updates for commit " + commitTime + " for file " + fileId); HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr); @@ -126,7 +133,8 @@ public List rollback(JavaSparkContext jsc, List comm } Map commitsAndCompactions = this.getActiveTimeline() - .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION)) + .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, + HoodieActiveTimeline.DELTA_COMMIT_ACTION)) .getInstants() .filter(i -> commits.contains(i.getTimestamp())) .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); @@ -139,41 +147,28 @@ public List rollback(JavaSparkContext jsc, List comm Long startTime = System.currentTimeMillis(); - List allRollbackStats = commits.stream().map(commit -> { - HoodieInstant instant = commitsAndCompactions.get(commit); - List stats = null; - switch (instant.getAction()) { - case HoodieTimeline.COMMIT_ACTION: - try { - logger.info("Starting to rollback Commit/Compaction " + instant); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(this.getCommitsTimeline().getInstantDetails( - new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); - - stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream() - .collect(Collectors.toList())) - .map((Function) partitionPath -> { - Map results = super - .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); - return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) + List allRollbackStats = jsc.parallelize + (FSUtils.getAllPartitionPaths(this.metaClient.getFs(), + this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) + .map((Function>) partitionPath -> { + return commits.stream().map(commit -> { + HoodieInstant instant = commitsAndCompactions.get(commit); + HoodieRollbackStat hoodieRollbackStats = null; + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + try { + Map results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit)); + hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) .withDeletedFileResults(results).build(); - }).collect(); - logger.info("Finished rollback of Commit/Compaction " + instant); - break; - } catch (IOException io) { - throw new UncheckedIOException("Failed to rollback for commit " + commit, io); - } - case HoodieTimeline.DELTA_COMMIT_ACTION: - try { - logger.info("Starting to rollback delta commit " + instant); - - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(this.getCommitsTimeline().getInstantDetails( - new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); + break; + } catch (IOException io) { + throw new UncheckedIOException("Failed to rollback for commit " + commit, io); + } + case HoodieTimeline.DELTA_COMMIT_ACTION: + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); - stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream() - .collect(Collectors.toList())) - .map((Function) partitionPath -> { // read commit file and (either append delete blocks or delete file) Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); @@ -183,69 +178,64 @@ public List rollback(JavaSparkContext jsc, List comm .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); // append rollback blocks for updates - commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() - .filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT) - .forEach(wStat -> { - HoodieLogFormat.Writer writer = null; - try { - writer = HoodieLogFormat.newWriterBuilder() - .onParentPath( - new Path(this.getMetaClient().getBasePath(), partitionPath)) - .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) - .withFs(getMetaClient().getFs()) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - Long numRollbackBlocks = 0L; - // generate metadata - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit); - // if update belongs to an existing log file - writer.appendBlock(new HoodieCommandBlock( - HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, - metadata)); - numRollbackBlocks++; - if (wStat.getNumDeletes() > 0) { - writer.appendBlock(new HoodieCommandBlock( + if(commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { + commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() + .filter(wStat -> { + return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT + && wStat.getPrevCommit() != null; + }) + .forEach(wStat -> { + HoodieLogFormat.Writer writer = null; + try { + writer = HoodieLogFormat.newWriterBuilder() + .onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath)) + .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) + .withFs(this.metaClient.getFs()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + Long numRollbackBlocks = 0L; + // generate metadata + Map metadata = Maps.newHashMap(); + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, + metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit); + // if update belongs to an existing log file + writer = writer.appendBlock(new HoodieCommandBlock( HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata)); numRollbackBlocks++; + filesToNumBlocksRollback + .put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()), + numRollbackBlocks); + } catch (IOException | InterruptedException io) { + throw new HoodieRollbackException( + "Failed to rollback for commit " + commit, io); + } finally { + try { + writer.close(); + } catch (IOException io) { + throw new UncheckedIOException(io); + } } - filesToNumBlocksRollback - .put(getMetaClient().getFs() - .getFileStatus(writer.getLogFile().getPath()), - numRollbackBlocks); - } catch (IOException | InterruptedException io) { - throw new HoodieRollbackException( - "Failed to rollback for commit " + commit, io); - } finally { - try { - writer.close(); - } catch (IOException io) { - throw new UncheckedIOException(io); - } - } - }); - return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) - .withDeletedFileResults(filesToDeletedStatus) - .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); - }).collect(); - logger.info("Fnished rollback of delta commit " + instant); - break; - } catch (IOException io) { - throw new UncheckedIOException("Failed to rollback for commit " + commit, io); - } - } - return stats; - }).flatMap(x -> x.stream()).collect(Collectors.toList()); + }); + hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) + .withDeletedFileResults(filesToDeletedStatus) + .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); + } + break; + } catch (IOException io) { + throw new UncheckedIOException("Failed to rollback for commit " + commit, io); + } + } + return hoodieRollbackStats; + }).collect(Collectors.toList()); + }).flatMap(x -> x.iterator()).filter(x -> x != null).collect(); commitsAndCompactions.entrySet().stream() .map(entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp())) .forEach(this.getActiveTimeline()::deleteInflight); - logger - .debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); + logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); return allRollbackStats; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java index 10bf6735a2b14..33764fa59de02 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java @@ -17,6 +17,8 @@ package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecordLocation; +import org.apache.commons.lang3.tuple.Pair; + import java.io.Serializable; import java.util.HashMap; @@ -29,7 +31,7 @@ public class WorkloadStat implements Serializable { private long numUpdates = 0L; - private HashMap updateLocationToCount; + private HashMap> updateLocationToCount; public WorkloadStat() { updateLocationToCount = new HashMap<>(); @@ -40,7 +42,7 @@ long addInserts(long numInserts) { } long addUpdates(HoodieRecordLocation location, long numUpdates) { - updateLocationToCount.put(location.getFileId(), numUpdates); + updateLocationToCount.put(location.getFileId(), Pair.of(location.getCommitTime(), numUpdates)); return this.numUpdates += numUpdates; } @@ -52,7 +54,7 @@ public long getNumInserts() { return numInserts; } - public HashMap getUpdateLocationToCount() { + public HashMap> getUpdateLocationToCount() { return updateLocationToCount; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index a9ed76379f0fe..5f0d51d2f3ee3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -463,7 +463,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { newCommitTime = "002"; client.startCommitWithTime(newCommitTime); - records = dataGen.generateUpdates(newCommitTime, 200); + records = dataGen.generateUpdates(newCommitTime, records); statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors @@ -556,6 +556,7 @@ private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { return HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withAutoCommit(autoCommit) + .withAssumeDatePartitioning(true) .withCompactionConfig( HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) .withInlineCompaction(false).build()) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 140d5d4fa9098..905da5c11dc93 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -16,9 +16,6 @@ package com.uber.hoodie.common.table.log; -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME; - import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; @@ -32,6 +29,14 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.exception.HoodieIOException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; @@ -43,13 +48,9 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; + +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME; /** * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of @@ -76,8 +77,8 @@ public class HoodieCompactedLogRecordScanner implements private HoodieTableMetaClient hoodieTableMetaClient; // Merge strategy to use when combining records from log private String payloadClassFQN; - // Store only the last log blocks (needed to implement rollback) - Deque lastBlocks = new ArrayDeque<>(); + // Store the last instant log blocks (needed to implement rollback) + Deque currentInstantLogBlocks = new ArrayDeque<>(); public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime) { @@ -100,8 +101,8 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List Running + // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1 + // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2) + // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2) + // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same. + // Say, commit C1 eventually failed and a rollback is triggered. + // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file, + // but in reality we need to rollback (B1 & B2) + // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2 + if(isNewInstantBlock(r)) { + // If this is a avro data block, then merge the last block records into the main result + merge(records, currentInstantLogBlocks); + } + // store the current block + currentInstantLogBlocks.push(r); break; case DELETE_BLOCK: log.info("Reading a delete block from file " + logFile.getPath()); - String lastBlockInstantTime = lastBlocks.peek().getLogMetadata().get(INSTANT_TIME); - if (!lastBlockInstantTime.equals(blockInstantTime)) { + if (isNewInstantBlock(r)) { // Block with the keys listed as to be deleted, data and delete blocks written in different batches // so it is safe to merge // This is a delete block, so lets merge any records from previous data block - merge(records, lastBlocks); + merge(records, currentInstantLogBlocks); } // store deletes so can be rolled back - lastBlocks.push(r); + currentInstantLogBlocks.push(r); break; case COMMAND_BLOCK: log.info("Reading a command block from file " + logFile.getPath()); @@ -137,28 +149,46 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List 0 && currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK + && !logBlock.getLogMetadata().get(INSTANT_TIME) + .contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME)); + } + /** * Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge * with the application specific payload if the same key was found before Sufficient to just merge @@ -218,15 +259,18 @@ private Map> loadRecordsFrom private void merge(Map> records, Deque lastBlocks) { while (!lastBlocks.isEmpty()) { - HoodieLogBlock lastBlock = lastBlocks.pop(); + // poll the element at the bottom of the stack since that's the order it was inserted + HoodieLogBlock lastBlock = lastBlocks.pollLast(); switch (lastBlock.getBlockType()) { case AVRO_DATA_BLOCK: merge(records, loadRecordsFromBlock((HoodieAvroDataBlock) lastBlock)); break; case DELETE_BLOCK: + // TODO : If delete is the only block written and/or records are present in parquet file Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove); break; case CORRUPT_BLOCK: + log.warn("Found a corrupt block which was not rolled back"); break; } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 00fd9ca5e4eac..a8be14ee4fb5f 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -520,23 +520,25 @@ public void testAvroLogRecordReaderWithRollbackTombstone() .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema, metadata); writer = writer.appendBlock(dataBlock); // Write 2 + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); writer = writer.appendBlock(dataBlock); // Rollback the last write + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); HoodieCommandBlock commandBlock = new HoodieCommandBlock( HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); writer = writer.appendBlock(commandBlock); // Write 3 + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords3 = records3.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) @@ -552,8 +554,8 @@ public void testAvroLogRecordReaderWithRollbackTombstone() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "100"); - assertEquals("We only read 200 records, but only 200 of them are valid", 200, + schema, "102"); + assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -583,12 +585,13 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema, metadata); writer = writer.appendBlock(dataBlock); writer.close(); + // Write 2 + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); @@ -605,6 +608,8 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() outputStream.close(); // Rollback the last write + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102"); + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); HoodieCommandBlock commandBlock = new HoodieCommandBlock( HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) @@ -613,6 +618,7 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() writer = writer.appendBlock(commandBlock); // Write 3 + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords3 = records3.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) @@ -629,7 +635,7 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "100"); + schema, "103"); assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -660,12 +666,12 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() .collect(Collectors.toList()); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema, metadata); writer = writer.appendBlock(dataBlock); // Write 2 + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords2 = records2.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) @@ -682,6 +688,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102"); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata); writer = writer.appendBlock(deleteBlock); @@ -693,7 +700,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "100"); + schema, "102"); assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); @@ -706,12 +713,14 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() readKeys); // Rollback the last block + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103"); + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "102"); HoodieCommandBlock commandBlock = new HoodieCommandBlock( HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); writer = writer.appendBlock(commandBlock); readKeys.clear(); - scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "100"); + scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101"); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); @@ -756,7 +765,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks() metadata); writer = writer.appendBlock(deleteBlock); - // Attemp 1 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write + // Attempt 1 : Write rollback block for a failed write HoodieCommandBlock commandBlock = new HoodieCommandBlock( HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); try { @@ -766,8 +775,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks() } catch (Exception e) { // it's okay } - // Attempt 2 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write - writer = writer.appendBlock(commandBlock); + // Attempt 2 : Write another rollback blocks for a failed write writer = writer.appendBlock(commandBlock); List allLogFiles = FSUtils @@ -778,12 +786,13 @@ public void testAvroLogRecordReaderWithFailedRollbacks() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "100"); - assertEquals("We would read 100 records", 100, + // all data must be rolled back before merge + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); - final List readKeys = new ArrayList<>(100); + final List readKeys = new ArrayList<>(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals("Stream collect should return all 150 records", 100, readKeys.size()); + assertEquals("Stream collect should return all 0 records", 0, readKeys.size()); } @Test @@ -850,12 +859,12 @@ public void testAvroLogRecordReaderWithInvalidRollback() List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema, metadata); writer = writer.appendBlock(dataBlock); // Write invalid rollback for a failed write (possible for in-flight commits) + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); HoodieCommandBlock commandBlock = new HoodieCommandBlock( HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); writer = writer.appendBlock(commandBlock); @@ -873,4 +882,140 @@ public void testAvroLogRecordReaderWithInvalidRollback() scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 100, readKeys.size()); } + + @Test + public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() + throws IOException, URISyntaxException, InterruptedException { + + // Write a 3 Data blocs with same InstantTime (written in same batch) + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + // Set a small threshold so that every block is a new version + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + + // Write 1 + List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + Map metadata = Maps.newHashMap(); + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, + schema, metadata); + writer = writer.appendBlock(dataBlock); + writer = writer.appendBlock(dataBlock); + writer = writer.appendBlock(dataBlock); + + List originalKeys = copyOfRecords1.stream() + .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) + .collect( + Collectors.toList()); + + // Delete 50 keys + List deletedKeys = originalKeys.subList(0, 50); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), + metadata); + writer = writer.appendBlock(deleteBlock); + + // Write 1 rollback block for a failed write + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); + HoodieCommandBlock commandBlock = new HoodieCommandBlock( + HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + writer = writer.appendBlock(commandBlock); + + List allLogFiles = FSUtils + .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") + .map(s -> s.getPath().toString()) + .collect(Collectors.toList()); + + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, + allLogFiles, schema, "101"); + assertEquals("We would read 0 records", 0, + scanner.getTotalLogRecords()); + } + + @Test + public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() + throws IOException, URISyntaxException, InterruptedException { + + // Write a 3 Data blocs with same InstantTime (written in same batch) + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + // Set a small threshold so that every block is a new version + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + + // Write 1 + List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + Map metadata = Maps.newHashMap(); + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); + metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, + schema, metadata); + writer = writer.appendBlock(dataBlock); + writer = writer.appendBlock(dataBlock); + writer = writer.appendBlock(dataBlock); + + writer.close(); + // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + // Write out a length that does not confirm with the content + outputStream.writeInt(100); + outputStream.flush(); + outputStream.close(); + + // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + // Write out a length that does not confirm with the content + outputStream.writeInt(100); + outputStream.flush(); + outputStream.close(); + + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + + writer = writer.appendBlock(dataBlock); + writer.close(); + + // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + // Write out a length that does not confirm with the content + outputStream.writeInt(100); + outputStream.flush(); + outputStream.close(); + + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + // Write 1 rollback block for a failed write + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); + HoodieCommandBlock commandBlock = new HoodieCommandBlock( + HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + writer = writer.appendBlock(commandBlock); + + List allLogFiles = FSUtils + .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") + .map(s -> s.getPath().toString()) + .collect(Collectors.toList()); + + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, + allLogFiles, schema, "101"); + assertEquals("We would read 0 records", 0, + scanner.getTotalLogRecords()); + } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index 6add30565753f..939f582976ec3 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -124,7 +124,7 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { // Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table String maxCommitTime = metaClient.getActiveTimeline() .getTimelineOfActions( - Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, + Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) .filterCompletedInstants().lastInstant().get().getTimestamp(); rtSplits.add( From f6b48d51c37fcab43bce0050df5ae1dd384a5995 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 10 Jan 2018 21:10:22 -0800 Subject: [PATCH 018/374] Small File Size correction handling for MOR table type --- .../hoodie/table/HoodieCopyOnWriteTable.java | 11 +- .../hoodie/table/HoodieMergeOnReadTable.java | 91 ++++++++++++-- .../hoodie/table/TestMergeOnReadTable.java | 119 +++++++++++++++--- 3 files changed, 192 insertions(+), 29 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 019153293e284..9ced403bd06df 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -182,6 +182,11 @@ class UpsertPartitioner extends Partitioner { */ private HashMap bucketInfoMap; + /** + * List of all small files to be corrected + */ + List smallFiles = new ArrayList(); + UpsertPartitioner(WorkloadProfile profile) { updateLocationToBucket = new HashMap<>(); partitionPathToInsertBuckets = new HashMap<>(); @@ -296,7 +301,9 @@ private void assignInserts(WorkloadProfile profile) { /** * Returns a list of small files in the given partition path */ - private List getSmallFiles(String partitionPath) { + protected List getSmallFiles(String partitionPath) { + + // smallFiles only for partitionPath List smallFileLocations = new ArrayList<>(); HoodieTimeline commitTimeline = getCompletedCommitTimeline(); @@ -315,6 +322,8 @@ private List getSmallFiles(String partitionPath) { FSUtils.getFileId(filename)); sf.sizeBytes = file.getFileSize(); smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); } } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index e97108086be6a..88d7b8d8a8a97 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -20,9 +20,11 @@ import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieRollbackStat; +import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -36,18 +38,20 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieRollbackException; +import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -57,7 +61,6 @@ import java.util.Optional; import java.util.stream.Collectors; - /** * Implementation of a more real-time read-optimized Hoodie Table where *

@@ -78,21 +81,89 @@ public class HoodieMergeOnReadTable extends private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class); - public HoodieMergeOnReadTable(HoodieWriteConfig config, - HoodieTableMetaClient metaClient) { + // UpsertPartitioner for MergeOnRead table type + private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner; + + public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { super(config, metaClient); } + /** + * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet + * files to larger ones without the need for an index in the logFile. + */ + class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner { + + MergeOnReadUpsertPartitioner(WorkloadProfile profile) { + super(profile); + } + + @Override + protected List getSmallFiles(String partitionPath) { + + // smallFiles only for partitionPath + List smallFileLocations = new ArrayList<>(); + + // Init here since this class (and member variables) might not have been initialized + HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + + if (!commitTimeline.empty()) { + HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); + // find smallest file in partition and append to it + Optional smallFileSlice = getRTFileSystemView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()) + .filter(fileSlice -> fileSlice.getLogFiles().count() < 1 && + fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit()) + .sorted((FileSlice left, FileSlice right) -> + left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() ? -1 : 1) + .findFirst(); + + if(smallFileSlice.isPresent()) { + String filename = smallFileSlice.get().getDataFile().get().getFileName(); + SmallFile sf = new SmallFile(); + sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), + FSUtils.getFileId(filename)); + sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize(); + smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); + } + } + + return smallFileLocations; + } + + public List getSmallFileIds() { + return (List) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId()) + .collect(Collectors.toList()); + } + } + + @Override + public Partitioner getUpsertPartitioner(WorkloadProfile profile) { + if (profile == null) { + throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); + } + mergeOnReadUpsertPartitioner = new MergeOnReadUpsertPartitioner(profile); + return mergeOnReadUpsertPartitioner; + } + @Override public Iterator> handleUpdate(String commitTime, String fileId, Iterator> recordItr) throws IOException { logger.info("Merging updates for commit " + commitTime + " for file " + fileId); - HoodieAppendHandle appendHandle = - new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr); - appendHandle.doAppend(); - appendHandle.close(); - return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())) - .iterator(); + + if(mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { + logger.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId); + return super.handleUpdate(commitTime, fileId, recordItr); + } else { + HoodieAppendHandle appendHandle = + new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr); + appendHandle.doAppend(); + appendHandle.close(); + return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())) + .iterator(); + } } @Override diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 5f0d51d2f3ee3..38ee2101a0abc 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -19,11 +19,6 @@ package com.uber.hoodie.table; -import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieClientTestUtils; @@ -49,14 +44,6 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.compact.HoodieCompactor; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -73,6 +60,20 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + public class TestMergeOnReadTable { private transient JavaSparkContext jsc = null; @@ -257,7 +258,7 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { } @Test - public void testSimpleInsertAndDelete() throws Exception { + public void testSimpleInsertUpdateAndDelete() throws Exception { HoodieWriteConfig cfg = getConfig(true); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); @@ -301,12 +302,12 @@ public void testSimpleInsertAndDelete() throws Exception { dataFilesToRead.findAny().isPresent()); /** - * Write 2 (only inserts, written to .log file) + * Write 2 (only updates, written to .log file) */ newCommitTime = "002"; client.startCommitWithTime(newCommitTime); - records = dataGen.generateInserts(newCommitTime, 20); + records = dataGen.generateUpdates(newCommitTime, records); writeRecords = jsc.parallelize(records, 1); statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); @@ -548,6 +549,88 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { }).findAny().isPresent()); } + @Test + public void testUpsertPartitioner() throws Exception { + HoodieWriteConfig cfg = getConfig(true); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + + /** + * Write 1 (only inserts, written as parquet file) + */ + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + List records = dataGen.generateInserts(newCommitTime, 20); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + List statuses = client.upsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + + Optional deltaCommit = + metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + assertTrue(deltaCommit.isPresent()); + assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); + + Optional commit = + metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + assertFalse(commit.isPresent()); + + FileStatus[] allFiles = HoodieTestUtils + .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, + hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles); + Stream dataFilesToRead = roView.getLatestDataFiles(); + Map parquetFileIdToSize = dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), + allFiles); + dataFilesToRead = roView.getLatestDataFiles(); + assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead.findAny().isPresent()); + + /** + * Write 2 (only updates + inserts, written to .log file + correction of existing parquet file size) + */ + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + + List newRecords = dataGen.generateUpdates(newCommitTime, records); + newRecords.addAll(dataGen.generateInserts(newCommitTime, 20)); + + statuses = client.upsert(jsc.parallelize(newRecords), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); + assertTrue(deltaCommit.isPresent()); + assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp()); + + commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + assertFalse(commit.isPresent()); + + allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getActiveTimeline().reload() + .getCommitsTimeline().filterCompletedInstants(), allFiles); + dataFilesToRead = roView.getLatestDataFiles(); + Map parquetFileIdToNewSize = dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + + assertTrue(parquetFileIdToNewSize.entrySet().stream() + .filter(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue()).count() > 0); + + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) + .collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils + .getRecordsUsingInputFormat(dataFiles, basePath); + //Wrote 20 records in 2 batches + assertEquals("Must contain 40 records", 40, recordsRead.size()); + } + + private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit).build(); } @@ -558,9 +641,9 @@ private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { .withAutoCommit(autoCommit) .withAssumeDatePartitioning(true) .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) .withInlineCompaction(false).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) .forTable("test-trip-table").withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } From f368d4648d7ceac35f0465cfd325024ddde1ae76 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 24 Jan 2018 13:34:14 -0800 Subject: [PATCH 019/374] refactor classes to accept Map passed by RealtimeCompactor to avoid multiple map creations in HoodieMergeHandle --- .../com/uber/hoodie/io/HoodieMergeHandle.java | 130 ++++++++++-------- .../compact/HoodieRealtimeTableCompactor.java | 2 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 20 ++- .../log/HoodieCompactedLogRecordScanner.java | 14 +- 4 files changed, 103 insertions(+), 63 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 7500abf3aeeb9..a974e8b38573c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -33,7 +33,9 @@ import java.io.IOException; import java.util.HashMap; import java.util.Iterator; +import java.util.Map; import java.util.Optional; + import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -47,7 +49,7 @@ public class HoodieMergeHandle extends HoodieIOHa private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class); private WriteStatus writeStatus; - private HashMap> keyToNewRecords; + private Map> keyToNewRecords; private HoodieStorageWriter storageWriter; private TableFileSystemView.ReadOptimizedView fileSystemView; private Path newFilePath; @@ -64,77 +66,93 @@ public HoodieMergeHandle(HoodieWriteConfig config, String fileId) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); - init(fileId, recordItr); + init(fileId, init(fileId, recordItr)); + } + + public HoodieMergeHandle(HoodieWriteConfig config, + String commitTime, + HoodieTable hoodieTable, + Map> keyToNewRecords, + String fileId) { + super(config, commitTime, hoodieTable); + this.fileSystemView = hoodieTable.getROFileSystemView(); + this.keyToNewRecords = keyToNewRecords; + init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath()); } /** - * Load the new incoming records in a map, and extract the old file path. + * Extract old file path, initialize StorageWriter and WriteStatus + * @param fileId + * @param partitionPath */ - private void init(String fileId, Iterator> newRecordsItr) { + private void init(String fileId, String partitionPath) { WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieWriteStat()); this.writeStatus = writeStatus; - this.keyToNewRecords = new HashMap<>(); - try { - // Load the new records in a map - while (newRecordsItr.hasNext()) { - HoodieRecord record = newRecordsItr.next(); - // If the first record, we need to extract some info out - if (oldFilePath == null) { - String latestValidFilePath = fileSystemView - .getLatestDataFiles(record.getPartitionPath()) - .filter(dataFile -> dataFile.getFileId().equals(fileId)) - .findFirst() - .get().getFileName(); - writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); - - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, - commitTime, - new Path(config.getBasePath()), - new Path(config.getBasePath(), record.getPartitionPath())); - partitionMetadata.trySave(TaskContext.getPartitionId()); - - oldFilePath = new Path( - config.getBasePath() + "/" + record.getPartitionPath() + "/" - + latestValidFilePath); - String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils - .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); - newFilePath = new Path(config.getBasePath(), relativePath); - if (config.shouldUseTempFolderForCopyOnWriteForMerge()) { - this.tempPath = makeTempPath(record.getPartitionPath(), TaskContext.getPartitionId(), fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); - } - - // handle cases of partial failures, for update task - if (fs.exists(newFilePath)) { - fs.delete(newFilePath, false); - } - - logger.info(String.format("Merging new data into oldPath %s, as newPath %s", - oldFilePath.toString(), getStorageWriterPath().toString())); - // file name is same for all records, in this bunch - writeStatus.setFileId(fileId); - writeStatus.setPartitionPath(record.getPartitionPath()); - writeStatus.getStat().setFileId(fileId); - writeStatus.getStat().setPaths(new Path(config.getBasePath()), newFilePath, tempPath); - } - keyToNewRecords.put(record.getRecordKey(), record); - // update the new location of the record, so we know where to find it next - record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); + String latestValidFilePath = fileSystemView + .getLatestDataFiles(partitionPath) + .filter(dataFile -> dataFile.getFileId().equals(fileId)) + .findFirst() + .get().getFileName(); + writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); + + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, + commitTime, + new Path(config.getBasePath()), + new Path(config.getBasePath(), partitionPath)); + partitionMetadata.trySave(TaskContext.getPartitionId()); + + oldFilePath = new Path( + config.getBasePath() + "/" + partitionPath + "/" + + latestValidFilePath); + String relativePath = new Path(partitionPath + "/" + FSUtils + .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); + newFilePath = new Path(config.getBasePath(), relativePath); + + // handle cases of partial failures, for update task + if (fs.exists(newFilePath)) { + fs.delete(newFilePath, false); } + + logger.info(String.format("Merging new data into oldPath %s, as newPath %s", + oldFilePath.toString(), newFilePath.toString())); + // file name is same for all records, in this bunch + writeStatus.setFileId(fileId); + writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setFileId(fileId); + writeStatus.getStat().setPath(relativePath); // Create the writer for writing the new version file storageWriter = HoodieStorageWriterFactory - .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); - - } catch (Exception e) { - logger.error("Error in update task at commit " + commitTime, e); - writeStatus.setGlobalError(e); + .getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema); + } catch (IOException io) { + logger.error("Error in update task at commit " + commitTime, io); + writeStatus.setGlobalError(io); throw new HoodieUpsertException( "Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " - + commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e); + + commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io); } } + /** + * Load the new incoming records in a map and return partitionPath + * @param fileId + * @param newRecordsItr + * @return + */ + private String init(String fileId, Iterator> newRecordsItr) { + // Load the new records in a map + this.keyToNewRecords = new HashMap<>(); + String partitionPath = null; + while (newRecordsItr.hasNext()) { + HoodieRecord record = newRecordsItr.next(); + partitionPath = record.getPartitionPath(); + keyToNewRecords.put(record.getRecordKey(), record); + // update the new location of the record, so we know where to find it next + record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); + } + return partitionPath; + } private boolean writeUpdateRecord(HoodieRecord hoodieRecord, Optional indexedRecord) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 13899246951d1..6e89c38cc4a8e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -160,7 +160,7 @@ private List executeCompaction(HoodieTable hoodieTable, // Compacting is very similar to applying updates to existing file HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); Iterator> result = table - .handleUpdate(commitTime, operation.getFileId(), scanner.iterator()); + .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false) .flatMap(Collection::stream) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 9ced403bd06df..5b4f176b6b483 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -416,10 +416,23 @@ public boolean isWorkloadProfileNeeded() { public Iterator> handleUpdate(String commitTime, String fileLoc, - Iterator> recordItr) + Iterator> recordItr) throws IOException { // these are updates HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr); + return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + } + + public Iterator> handleUpdate(String commitTime, String fileLoc, + Map> keyToNewRecords) + throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords); + return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + } + + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc) + throws IOException { if (upsertHandle.getOldFilePath() == null) { throw new HoodieUpsertException("Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc); @@ -459,6 +472,11 @@ protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); } + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, + Map> keyToNewRecords) { + return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc); + } + public Iterator> handleInsert(String commitTime, Iterator> recordItr) throws Exception { return new LazyInsertIterable<>(recordItr, config, commitTime, this); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 905da5c11dc93..f01e15f8e233c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -43,6 +43,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Deque; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -62,8 +63,8 @@ public class HoodieCompactedLogRecordScanner implements private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); - // Final list of compacted/merged records to iterate - private final Collection> logRecords; + // Final map of compacted/merged records + private final Map> records; // Reader schema for the records private final Schema readerSchema; // Total log files read - for metrics @@ -89,7 +90,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List> records = Maps.newHashMap(); + this.records = Maps.newHashMap(); // iterate over the paths Iterator logFilePathsItr = logFilePaths.iterator(); while (logFilePathsItr.hasNext()) { @@ -202,7 +203,6 @@ else if(!targetInstantForCommandBlock merge(records, currentInstantLogBlocks); } } - this.logRecords = Collections.unmodifiableCollection(records.values()); this.totalRecordsToUpdate = records.size(); } @@ -297,7 +297,7 @@ private void merge(Map> reco @Override public Iterator> iterator() { - return logRecords.iterator(); + return records.values().iterator(); } public long getTotalLogFiles() { @@ -308,6 +308,10 @@ public long getTotalLogRecords() { return totalLogRecords.get(); } + public Map> getRecords() { + return records; + } + public long getTotalRecordsToUpdate() { return totalRecordsToUpdate; } From 7036bd16a4b367737fbdf1ce6b0fac8c7f84142c Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Tue, 30 Jan 2018 17:13:54 -0800 Subject: [PATCH 020/374] Use FastDateFormat for thread safety Use FastDateFormat for thread safety, this is to fix an exception when a job is used to ingest multiple tables. An example exception: ``` Caused by: java.lang.NumberFormatException: multiple points at sun.misc.FloatingDecimal.readJavaFormatString(FloatingDecimal.java:1890) at sun.misc.FloatingDecimal.parseDouble(FloatingDecimal.java:110) at java.lang.Double.parseDouble(Double.java:538) at java.text.DigitList.getDouble(DigitList.java:169) at java.text.DecimalFormat.parse(DecimalFormat.java:2056) at java.text.SimpleDateFormat.subParse(SimpleDateFormat.java:1867) at java.text.SimpleDateFormat.parse(SimpleDateFormat.java:1514) at java.text.DateFormat.parse(DateFormat.java:364) at com.uber.hoodie.HoodieWriteClient.commit(HoodieWriteClient.java:442) ``` --- .../hoodie/common/table/timeline/HoodieActiveTimeline.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index 4c9f5c5232fc9..8c34717b23b0f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -24,7 +24,6 @@ import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; -import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Comparator; import java.util.Date; @@ -35,6 +34,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.time.FastDateFormat; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -51,7 +51,7 @@ */ public class HoodieActiveTimeline extends HoodieDefaultTimeline { - public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); + public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat.getInstance("yyyyMMddHHmmss"); private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); private HoodieTableMetaClient metaClient; From 710754752f0ddad74042ae28d3f19b8ba0e1ed56 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Tue, 13 Feb 2018 17:40:09 -0800 Subject: [PATCH 021/374] Fix formatting in HoodieWriteClient --- .../com/uber/hoodie/HoodieWriteClient.java | 1646 ++++++++--------- 1 file changed, 819 insertions(+), 827 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index e960e1be545b7..100dcfaa030d6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -57,7 +57,6 @@ import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.text.ParseException; -import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashMap; @@ -65,9 +64,7 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileStatus; 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.Partitioner; @@ -88,831 +85,826 @@ */ public class HoodieWriteClient implements Serializable { - private static Logger logger = LogManager.getLogger(HoodieWriteClient.class); - private transient final FileSystem fs; - private transient final JavaSparkContext jsc; - private final HoodieWriteConfig config; - private transient final HoodieMetrics metrics; - private transient final HoodieIndex index; - private transient Timer.Context writeContext = null; - - /** - * @param jsc - * @param clientConfig - * @throws Exception - */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) - throws Exception { - this(jsc, clientConfig, false); - } - - /** - * @param jsc - * @param clientConfig - * @param rollbackInFlight - */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, - boolean rollbackInFlight) { - this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration()); - this.jsc = jsc; - this.config = clientConfig; - this.index = HoodieIndex.createIndex(config, jsc); - this.metrics = new HoodieMetrics(config, config.getTableName()); - - if (rollbackInFlight) { - rollbackInflightCommits(); - } - } - - /** - * Filter out HoodieRecords that already exists in the output folder. This is useful in - * deduplication. - * - * @param hoodieRecords Input RDD of Hoodie records. - * @return A subset of hoodieRecords RDD, with existing records filtered out. - */ - public JavaRDD> filterExists(JavaRDD> hoodieRecords) { - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); - return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); - } - - /** - * Upserts a bunch of new records into the Hoodie table, at the supplied commitTime - */ - public JavaRDD upsert(JavaRDD> records, final String commitTime) { - writeContext = metrics.getCommitCtx(); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - try { - // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeUpsert(), records, - config.getUpsertShuffleParallelism()); - - // perform index loop up to get existing location of records - JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, table); - return upsertRecordsInternal(taggedRecords, commitTime, table, true); - } catch (Throwable e) { - if (e instanceof HoodieUpsertException) { - throw (HoodieUpsertException) e; - } - throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e); - } - } - - /** - * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal - * writes. - * - * This implementation skips the index check and is able to leverage benefits such as small file - * handling/blocking alignment, as with upsert(), by profiling the workload - * - * @param records HoodieRecords to insert - * @param commitTime Commit Time handle - * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts - */ - public JavaRDD insert(JavaRDD> records, final String commitTime) { - writeContext = metrics.getCommitCtx(); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - try { - // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeInsert(), records, - config.getInsertShuffleParallelism()); - - return upsertRecordsInternal(dedupedRecords, commitTime, table, false); - } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } - throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e); - } - } - - /** - * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk - * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to - * Hoodie). - * - * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and - * attempts to control the numbers of files with less memory compared to the {@link - * HoodieWriteClient#insert(JavaRDD, String)} - * - * @param records HoodieRecords to insert - * @param commitTime Commit Time handle - * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts - */ - public JavaRDD bulkInsert(JavaRDD> records, - final String commitTime) { - return bulkInsert(records, commitTime, Option.empty()); - } - - /** - * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk - * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to - * Hoodie). - * - * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and - * attempts to control the numbers of files with less memory compared to the {@link - * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own - * partitioner. If specified then it will be used for repartitioning records. See {@link - * UserDefinedBulkInsertPartitioner}. - * - * @param records HoodieRecords to insert - * @param commitTime Commit Time handle - * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. - * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts - */ - public JavaRDD bulkInsert(JavaRDD> records, - final String commitTime, - Option bulkInsertPartitioner) { - writeContext = metrics.getCommitCtx(); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - try { - // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeInsert(), records, - config.getInsertShuffleParallelism()); - - final JavaRDD> repartitionedRecords; - if (bulkInsertPartitioner.isDefined()) { - repartitionedRecords = - bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, - config.getBulkInsertShuffleParallelism()); - } else { - // Now, sort the records and line them up nicely for loading. - repartitionedRecords = dedupedRecords - .sortBy(record -> { - // Let's use "partitionPath + key" as the sort key. Spark, will ensure - // the records split evenly across RDD partitions, such that small partitions fit - // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions - return String - .format("%s+%s", record.getPartitionPath(), record.getRecordKey()); - }, true, config.getBulkInsertShuffleParallelism()); - } - JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), - true) - .flatMap(writeStatuses -> writeStatuses.iterator()); - - return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); - } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } - throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, - e); - } - } - - private void commitOnAutoCommit(String commitTime, JavaRDD resultRDD) { - if (config.shouldAutoCommit()) { - logger.info("Auto commit enabled: Committing " + commitTime); - boolean commitResult = commit(commitTime, resultRDD); - if (!commitResult) { - throw new HoodieCommitException("Failed to commit " + commitTime); - } - } else { - logger.info("Auto commit disabled for " + commitTime); - } - } - - private JavaRDD> combineOnCondition(boolean condition, - JavaRDD> records, - int parallelism) { - if (condition) { - return deduplicateRecords(records, parallelism); - } - return records; - } - - /** - * Save the workload profile in an intermediate file (here re-using commit files) This is useful - * when performing rollback for MOR datasets. Only updates are recorded in the workload profile - * metadata since updates to log blocks are unknown across batches Inserts (which are new parquet - * files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata - * file instead of using HoodieCommitMetadata - */ - private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, - HoodieTable table, String commitTime) throws HoodieCommitException { - try { - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - profile.getPartitionPaths().stream().forEach(path -> { - WorkloadStat partitionStat = profile.getWorkloadStat(path.toString()); - partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> { - HoodieWriteStat writeStat = new HoodieWriteStat(); - writeStat.setFileId(entry.getKey()); - writeStat.setPrevCommit(entry.getValue().getKey()); - writeStat.setNumUpdateWrites(entry.getValue().getValue()); - metadata.addWriteStat(path.toString(), writeStat); - }); - }); - - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - Optional instant = activeTimeline.filterInflights().lastInstant(); - activeTimeline.saveToInflight(instant.get(), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - } catch (IOException io) { - throw new HoodieCommitException( - "Failed to commit " + commitTime + " unable to save inflight metadata ", io); - } - } - - private JavaRDD upsertRecordsInternal(JavaRDD> preppedRecords, - String commitTime, - HoodieTable hoodieTable, - final boolean isUpsert) { - - // Cache the tagged records, so we don't end up computing both - preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER()); - - WorkloadProfile profile = null; - if (hoodieTable.isWorkloadProfileNeeded()) { - profile = new WorkloadProfile(preppedRecords); - logger.info("Workload profile :" + profile); - saveWorkloadProfileMetadataToInflight(profile, hoodieTable, commitTime); - } - - // partition using the insert partitioner - final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile); - JavaRDD> partitionedRecords = partition(preppedRecords, partitioner); - JavaRDD writeStatusRDD = partitionedRecords - .mapPartitionsWithIndex((partition, recordItr) -> { - if (isUpsert) { - return hoodieTable - .handleUpsertPartition(commitTime, partition, recordItr, partitioner); - } else { - return hoodieTable - .handleInsertPartition(commitTime, partition, recordItr, partitioner); - } - }, true) - .flatMap(writeStatuses -> writeStatuses.iterator()); - - return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime); - } - - private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, - WorkloadProfile profile) { - if (isUpsert) { - return table.getUpsertPartitioner(profile); - } else { - return table.getInsertPartitioner(profile); - } - } - - private JavaRDD updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, - HoodieTable table, String commitTime) { - // Update the index back - JavaRDD statuses = index.updateLocation(writeStatusRDD, table); - // Trigger the insert and collect statuses - statuses = statuses.persist(config.getWriteStatusStorageLevel()); - commitOnAutoCommit(commitTime, statuses); - return statuses; - } - - private JavaRDD> partition(JavaRDD> dedupedRecords, - Partitioner partitioner) { - return dedupedRecords - .mapToPair(record -> - new Tuple2<>( - new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), - record)) - .partitionBy(partitioner) - .map(tuple -> tuple._2()); - } - - /** - * Commit changes performed at the given commitTime marker - */ - public boolean commit(String commitTime, JavaRDD writeStatuses) { - return commit(commitTime, writeStatuses, Optional.empty()); - } - - /** - * Commit changes performed at the given commitTime marker - */ - public boolean commit(String commitTime, - JavaRDD writeStatuses, - Optional> extraMetadata) { - - logger.info("Commiting " + commitTime); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - - List> stats = writeStatuses - .mapToPair((PairFunction) writeStatus -> - new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) - .collect(); - - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - for (Tuple2 stat : stats) { - metadata.addWriteStat(stat._1(), stat._2()); - } - - // Finalize write - final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); - final Optional result = table.finalizeWrite(jsc, stats); - if (finalizeCtx != null && result.isPresent()) { - Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); - durationInMs.ifPresent(duration -> { - logger.info("Finalize write elapsed time (milliseconds): " + duration); - metrics.updateFinalizeWriteMetrics(duration, result.get()); - } - ); - } - - // add in extra metadata - if (extraMetadata.isPresent()) { - extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v)); - } - - try { - String actionType = table.getCommitActionType(); - activeTimeline.saveAsComplete( - new HoodieInstant(true, actionType, commitTime), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - // Save was a success - // Do a inline compaction if enabled - if (config.isInlineCompaction()) { - metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); - forceCompact(); - } else { - metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); - } - - // We cannot have unbounded commit files. Archive commits if we have to archive - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); - archiveLog.archiveIfRequired(); - if (config.isAutoClean()) { - // Call clean to cleanup if there is anything to cleanup after the commit, - logger.info("Auto cleaning is enabled. Running cleaner now"); - clean(commitTime); - } else { - logger.info("Auto cleaning is not enabled. Not running cleaner now"); - } - if (writeContext != null) { - long durationInMs = metrics.getDurationInMs(writeContext.stop()); - metrics.updateCommitMetrics( - HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs, - metadata); - writeContext = null; - } - logger.info("Committed " + commitTime); - } catch (IOException e) { - throw new HoodieCommitException( - "Failed to commit " + config.getBasePath() + " at time " + commitTime, e); - } catch (ParseException e) { - throw new HoodieCommitException( - "Commit time is not of valid format.Failed to commit " + config.getBasePath() - + " at time " + commitTime, e); - } - return true; - } - - /** - * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will - * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be - * rolledback or archived. - * - * This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be - * manually created and deleted. - * - * Savepoint should be on a commit that could not have been cleaned. - * - * @param user - User creating the savepoint - * @param comment - Comment for the savepoint - * @return true if the savepoint was created successfully - */ - public boolean savepoint(String user, String comment) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - if (table.getCompletedCommitTimeline().empty()) { - throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); - } - - String latestCommit = table.getCompletedCommitTimeline().lastInstant().get().getTimestamp(); - logger.info("Savepointing latest commit " + latestCommit); - return savepoint(latestCommit, user, comment); - } - - /** - * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will - * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be - * rolledback or archived. - * - * This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be - * manually created and deleted. - * - * Savepoint should be on a commit that could not have been cleaned. - * - * @param commitTime - commit that should be savepointed - * @param user - User creating the savepoint - * @param comment - Comment for the savepoint - * @return true if the savepoint was created successfully - */ - public boolean savepoint(String commitTime, String user, String comment) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); - - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); - if (!table.getCompletedCommitTimeline().containsInstant(commitInstant)) { - throw new HoodieSavepointException( - "Could not savepoint non-existing commit " + commitInstant); - } - - try { - // Check the last commit that was not cleaned and check if savepoint time is > that commit - String lastCommitRetained; - if (cleanInstant.isPresent()) { - HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata( - table.getActiveTimeline().getInstantDetails(cleanInstant.get()).get()); - lastCommitRetained = cleanMetadata.getEarliestCommitToRetain(); - } else { - lastCommitRetained = - table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); - } - - // Cannot allow savepoint time on a commit that could have been cleaned - Preconditions.checkArgument(HoodieTimeline - .compareTimestamps(commitTime, lastCommitRetained, HoodieTimeline.GREATER_OR_EQUAL), - "Could not savepoint commit " + commitTime + " as this is beyond the lookup window " - + lastCommitRetained); - - Map> latestFilesMap = jsc.parallelize( - FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning())) - .mapToPair((PairFunction>) partitionPath -> { - // Scan all partitions files with this commit time - logger.info("Collecting latest files in partition path " + partitionPath); - TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); - List latestFiles = - view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) - .map(HoodieDataFile::getFileName).collect(Collectors.toList()); - return new Tuple2<>(partitionPath, latestFiles); - }).collectAsMap(); - - HoodieSavepointMetadata metadata = - AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap); - // Nothing to save in the savepoint - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime), - AvroUtils.serializeSavepointMetadata(metadata)); - logger.info("Savepoint " + commitTime + " created"); - return true; - } catch (IOException e) { - throw new HoodieSavepointException("Failed to savepoint " + commitTime, e); - } - } - - /** - * Delete a savepoint that was created. Once the savepoint is deleted, the commit can be - * rolledback and cleaner may clean up data files. - * - * @param savepointTime - delete the savepoint - * @return true if the savepoint was deleted successfully - */ - public void deleteSavepoint(String savepointTime) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - - HoodieInstant savePoint = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); - boolean isSavepointPresent = - table.getCompletedSavepointTimeline().containsInstant(savePoint); - if (!isSavepointPresent) { - logger.warn("No savepoint present " + savepointTime); - return; - } - - activeTimeline.revertToInflight(savePoint); - activeTimeline.deleteInflight( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); - logger.info("Savepoint " + savepointTime + " deleted"); - } - - /** - * Rollback the state to the savepoint. WARNING: This rollsback recent commits and deleted data - * files. Queries accessing the files will mostly fail. This should be done during a downtime. - * - * @param savepointTime - savepoint time to rollback to - * @return true if the savepoint was rollecback to successfully - */ - public boolean rollbackToSavepoint(String savepointTime) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline commitTimeline = table.getCommitsTimeline(); - - HoodieInstant savePoint = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); - boolean isSavepointPresent = - table.getCompletedSavepointTimeline().containsInstant(savePoint); - if (!isSavepointPresent) { - throw new HoodieRollbackException("No savepoint for commitTime " + savepointTime); - } - - List commitsToRollback = - commitTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - logger.info("Rolling back commits " + commitsToRollback); - - rollback(commitsToRollback); - - // Make sure the rollback was successful - Optional lastInstant = - activeTimeline.reload().getCommitsTimeline().filterCompletedInstants() - .lastInstant(); - Preconditions.checkArgument(lastInstant.isPresent()); - Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), - savepointTime + "is not the last commit after rolling back " + commitsToRollback - + ", last commit was " + lastInstant.get().getTimestamp()); - return true; - } - - /** - * Rollback the (inflight/committed) record changes with the given commit time. Three steps: (1) - * Atomically unpublish this commit (2) clean indexing data, (3) clean new generated parquet - * files. (4) Finally delete .commit or .inflight file, - */ - public boolean rollback(final String commitTime) throws HoodieRollbackException { - rollback(Lists.newArrayList(commitTime)); - return true; - } - - private void rollback(List commits) { - if (commits.isEmpty()) { - logger.info("List of commits to rollback is empty"); - return; - } - - final Timer.Context context = metrics.getRollbackCtx(); - String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); - - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); - HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); - - // Check if any of the commits is a savepoint - do not allow rollback on those commits - List savepoints = - table.getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); - commits.forEach(s -> { - if (savepoints.contains(s)) { - throw new HoodieRollbackException( - "Could not rollback a savepointed commit. Delete savepoint first before rolling back" - + s); - } + private static Logger logger = LogManager.getLogger(HoodieWriteClient.class); + private transient final FileSystem fs; + private transient final JavaSparkContext jsc; + private final HoodieWriteConfig config; + private transient final HoodieMetrics metrics; + private transient final HoodieIndex index; + private transient Timer.Context writeContext = null; + + /** + * @param jsc + * @param clientConfig + * @throws Exception + */ + public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) + throws Exception { + this(jsc, clientConfig, false); + } + + /** + * @param jsc + * @param clientConfig + * @param rollbackInFlight + */ + public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, + boolean rollbackInFlight) { + this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration()); + this.jsc = jsc; + this.config = clientConfig; + this.index = HoodieIndex.createIndex(config, jsc); + this.metrics = new HoodieMetrics(config, config.getTableName()); + + if (rollbackInFlight) { + rollbackInflightCommits(); + } + } + + /** + * Filter out HoodieRecords that already exists in the output folder. This is useful in + * deduplication. + * + * @param hoodieRecords Input RDD of Hoodie records. + * @return A subset of hoodieRecords RDD, with existing records filtered out. + */ + public JavaRDD> filterExists(JavaRDD> hoodieRecords) { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); + return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); + } + + /** + * Upserts a bunch of new records into the Hoodie table, at the supplied commitTime + */ + public JavaRDD upsert(JavaRDD> records, final String commitTime) { + writeContext = metrics.getCommitCtx(); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + try { + // De-dupe/merge if needed + JavaRDD> dedupedRecords = + combineOnCondition(config.shouldCombineBeforeUpsert(), records, + config.getUpsertShuffleParallelism()); + + // perform index loop up to get existing location of records + JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, table); + return upsertRecordsInternal(taggedRecords, commitTime, table, true); + } catch (Throwable e) { + if (e instanceof HoodieUpsertException) { + throw (HoodieUpsertException) e; + } + throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e); + } + } + + /** + * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal + * writes. + * + * This implementation skips the index check and is able to leverage benefits such as small file + * handling/blocking alignment, as with upsert(), by profiling the workload + * + * @param records HoodieRecords to insert + * @param commitTime Commit Time handle + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public JavaRDD insert(JavaRDD> records, final String commitTime) { + writeContext = metrics.getCommitCtx(); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + try { + // De-dupe/merge if needed + JavaRDD> dedupedRecords = + combineOnCondition(config.shouldCombineBeforeInsert(), records, + config.getInsertShuffleParallelism()); + + return upsertRecordsInternal(dedupedRecords, commitTime, table, false); + } catch (Throwable e) { + if (e instanceof HoodieInsertException) { + throw e; + } + throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e); + } + } + + /** + * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk + * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to + * Hoodie). + * + * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and + * attempts to control the numbers of files with less memory compared to the {@link + * HoodieWriteClient#insert(JavaRDD, String)} + * + * @param records HoodieRecords to insert + * @param commitTime Commit Time handle + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public JavaRDD bulkInsert(JavaRDD> records, + final String commitTime) { + return bulkInsert(records, commitTime, Option.empty()); + } + + /** + * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk + * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to + * Hoodie). + * + * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and + * attempts to control the numbers of files with less memory compared to the {@link + * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own + * partitioner. If specified then it will be used for repartitioning records. See {@link + * UserDefinedBulkInsertPartitioner}. + * + * @param records HoodieRecords to insert + * @param commitTime Commit Time handle + * @param bulkInsertPartitioner If specified then it will be used to partition input records + * before they are inserted into hoodie. + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public JavaRDD bulkInsert(JavaRDD> records, + final String commitTime, + Option bulkInsertPartitioner) { + writeContext = metrics.getCommitCtx(); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + try { + // De-dupe/merge if needed + JavaRDD> dedupedRecords = + combineOnCondition(config.shouldCombineBeforeInsert(), records, + config.getInsertShuffleParallelism()); + + final JavaRDD> repartitionedRecords; + if (bulkInsertPartitioner.isDefined()) { + repartitionedRecords = + bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, + config.getBulkInsertShuffleParallelism()); + } else { + // Now, sort the records and line them up nicely for loading. + repartitionedRecords = dedupedRecords + .sortBy(record -> { + // Let's use "partitionPath + key" as the sort key. Spark, will ensure + // the records split evenly across RDD partitions, such that small partitions fit + // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions + return String + .format("%s+%s", record.getPartitionPath(), record.getRecordKey()); + }, true, config.getBulkInsertShuffleParallelism()); + } + JavaRDD writeStatusRDD = repartitionedRecords + .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), + true) + .flatMap(writeStatuses -> writeStatuses.iterator()); + + return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); + } catch (Throwable e) { + if (e instanceof HoodieInsertException) { + throw e; + } + throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, + e); + } + } + + private void commitOnAutoCommit(String commitTime, JavaRDD resultRDD) { + if (config.shouldAutoCommit()) { + logger.info("Auto commit enabled: Committing " + commitTime); + boolean commitResult = commit(commitTime, resultRDD); + if (!commitResult) { + throw new HoodieCommitException("Failed to commit " + commitTime); + } + } else { + logger.info("Auto commit disabled for " + commitTime); + } + } + + private JavaRDD> combineOnCondition(boolean condition, + JavaRDD> records, + int parallelism) { + if (condition) { + return deduplicateRecords(records, parallelism); + } + return records; + } + + /** + * Save the workload profile in an intermediate file (here re-using commit files) This is useful + * when performing rollback for MOR datasets. Only updates are recorded in the workload profile + * metadata since updates to log blocks are unknown across batches Inserts (which are new parquet + * files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata + * file instead of using HoodieCommitMetadata + */ + private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, + HoodieTable table, String commitTime) throws HoodieCommitException { + try { + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + profile.getPartitionPaths().stream().forEach(path -> { + WorkloadStat partitionStat = profile.getWorkloadStat(path.toString()); + partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(entry.getKey()); + writeStat.setPrevCommit(entry.getValue().getKey()); + writeStat.setNumUpdateWrites(entry.getValue().getValue()); + metadata.addWriteStat(path.toString(), writeStat); }); - - try { - if (commitTimeline.empty() && inflightTimeline.empty()) { - // nothing to rollback - logger.info("No commits to rollback " + commits); - } - - // Make sure only the last n commits are being rolled back - // If there is a commit in-between or after that is not rolled back, then abort - String lastCommit = commits.get(commits.size() - 1); - if (!commitTimeline.empty() && !commitTimeline - .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { - throw new HoodieRollbackException("Found commits after time :" + lastCommit + - ", please rollback greater commits first"); - } - - List inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); - if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) { - throw new HoodieRollbackException( - "Found in-flight commits after time :" + lastCommit + - ", please rollback greater commits first"); - } - - List stats = table.rollback(jsc, commits); - - // cleanup index entries - commits.stream().forEach(s -> { - if (!index.rollbackCommit(s)) { - throw new HoodieRollbackException( - "Rollback index changes failed, for time :" + s); - } - }); - logger.info("Index rolled back for commits " + commits); - - Optional durationInMs = Optional.empty(); - if (context != null) { - durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); - Long numFilesDeleted = stats.stream() - .mapToLong(stat -> stat.getSuccessDeleteFiles().size()) - .sum(); - metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); - } - HoodieRollbackMetadata rollbackMetadata = - AvroUtils.convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime), - AvroUtils.serializeRollbackMetadata(rollbackMetadata)); - logger.info("Commits " + commits + " rollback is complete"); - - if (!table.getActiveTimeline().getCleanerTimeline().empty()) { - logger.info("Cleaning up older rollback meta files"); - // Cleanup of older cleaner meta files - // TODO - make the commit archival generic and archive rollback metadata - FSUtils.deleteOlderRollbackMetaFiles(fs, table.getMetaClient().getMetaPath(), - table.getActiveTimeline().getRollbackTimeline().getInstants()); - } - } catch (IOException e) { - throw new HoodieRollbackException("Failed to rollback " + - config.getBasePath() + " commits " + commits, e); - } - } - - /** - * Releases any resources used by the client. - */ - public void close() { - // UNDER CONSTRUCTION - } - - /** - * Clean up any stale/old files/data lying around (either on file storage or index storage) based - * on the configurations and CleaningPolicy used. (typically files that no longer can be used by a - * running query can be cleaned) - */ - public void clean() throws HoodieIOException { - String startCleanTime = HoodieActiveTimeline.createNewCommitTime(); - clean(startCleanTime); - } - - /** - * Clean up any stale/old files/data lying around (either on file storage or index storage) based - * on the configurations and CleaningPolicy used. (typically files that no longer can be used by a - * running query can be cleaned) - */ - private void clean(String startCleanTime) throws HoodieIOException { - try { - logger.info("Cleaner started"); - final Timer.Context context = metrics.getCleanCtx(); - - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), - true), config); - - List cleanStats = table.clean(jsc); - if (cleanStats.isEmpty()) { - return; - } - - // Emit metrics (duration, numFilesDeleted) if needed - Optional durationInMs = Optional.empty(); - if (context != null) { - durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); - logger.info("cleanerElaspsedTime (Minutes): " + durationInMs.get() / (1000 * 60)); - } - - // Create the metadata and save it - HoodieCleanMetadata metadata = - AvroUtils.convertCleanMetadata(startCleanTime, durationInMs, cleanStats); - logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"); - metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), - metadata.getTotalFilesDeleted()); - - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime), - AvroUtils.serializeCleanMetadata(metadata)); - logger.info("Marked clean started on " + startCleanTime + " as complete"); - - if (!table.getActiveTimeline().getCleanerTimeline().empty()) { - // Cleanup of older cleaner meta files - // TODO - make the commit archival generic and archive clean metadata - FSUtils.deleteOlderCleanMetaFiles(fs, table.getMetaClient().getMetaPath(), - table.getActiveTimeline().getCleanerTimeline().getInstants()); - } - } catch (IOException e) { - throw new HoodieIOException("Failed to clean up after commit", e); - } - } - - /** - * Provides a new commit time for a write operation (insert/update) - */ - public String startCommit() { - String commitTime = HoodieActiveTimeline.createNewCommitTime(); - startCommitWithTime(commitTime); - return commitTime; - } - - public void startCommitWithTime(String commitTime) { - logger.info("Generate a new commit time " + commitTime); - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = table.getCommitActionType(); - activeTimeline.createInflight( - new HoodieInstant(true, commitActionType, commitTime)); - } - - /** - * Performs a compaction operation on a dataset. - * WARNING: Compaction operation cannot be executed asynchronously. Please always use this serially - * before or after an insert/upsert action. - * @param compactionCommitTime - * @throws IOException - */ - private void compact(String compactionCommitTime) throws IOException { - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - Optional compactionMetadata = table.compact(jsc, compactionCommitTime); - if (compactionMetadata.isPresent()) { - logger.info("Compacted successfully on commit " + compactionCommitTime); - } else { - logger.info("Compaction did not run for commit " + compactionCommitTime); - } - } - - /** - * Performs a compaction operation on a dataset. - * WARNING: Compaction operation cannot be executed asynchronously. Please always use this serially - * before or after an insert/upsert action. - * @throws IOException - */ - public String forceCompact() throws IOException { - String compactionCommitTime = HoodieActiveTimeline.createNewCommitTime(); - compact(compactionCommitTime); - return compactionCommitTime; - } - - public static SparkConf registerClasses(SparkConf conf) { - conf.registerKryoClasses( - new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); - return conf; - } - - /** - * Deduplicate Hoodie records, using the given deduplication funciton. - */ - private JavaRDD> deduplicateRecords(JavaRDD> records, - int parallelism) { - return records - .mapToPair(record -> new Tuple2<>(record.getKey(), record)) - .reduceByKey((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect everything - // so pick it from one of the records. - return new HoodieRecord(rec1.getKey(), reducedData); - }, parallelism) - .map(recordTuple -> recordTuple._2()); - } - - /** - * Cleanup all inflight commits - */ - private void rollbackInflightCommits() { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); - List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) + }); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + Optional instant = activeTimeline.filterInflights().lastInstant(); + activeTimeline.saveToInflight(instant.get(), + Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException io) { + throw new HoodieCommitException( + "Failed to commit " + commitTime + " unable to save inflight metadata ", io); + } + } + + private JavaRDD upsertRecordsInternal(JavaRDD> preppedRecords, + String commitTime, + HoodieTable hoodieTable, + final boolean isUpsert) { + + // Cache the tagged records, so we don't end up computing both + preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER()); + + WorkloadProfile profile = null; + if (hoodieTable.isWorkloadProfileNeeded()) { + profile = new WorkloadProfile(preppedRecords); + logger.info("Workload profile :" + profile); + saveWorkloadProfileMetadataToInflight(profile, hoodieTable, commitTime); + } + + // partition using the insert partitioner + final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile); + JavaRDD> partitionedRecords = partition(preppedRecords, partitioner); + JavaRDD writeStatusRDD = partitionedRecords + .mapPartitionsWithIndex((partition, recordItr) -> { + if (isUpsert) { + return hoodieTable + .handleUpsertPartition(commitTime, partition, recordItr, partitioner); + } else { + return hoodieTable + .handleInsertPartition(commitTime, partition, recordItr, partitioner); + } + }, true) + .flatMap(writeStatuses -> writeStatuses.iterator()); + + return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime); + } + + private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, + WorkloadProfile profile) { + if (isUpsert) { + return table.getUpsertPartitioner(profile); + } else { + return table.getInsertPartitioner(profile); + } + } + + private JavaRDD updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, + HoodieTable table, String commitTime) { + // Update the index back + JavaRDD statuses = index.updateLocation(writeStatusRDD, table); + // Trigger the insert and collect statuses + statuses = statuses.persist(config.getWriteStatusStorageLevel()); + commitOnAutoCommit(commitTime, statuses); + return statuses; + } + + private JavaRDD> partition(JavaRDD> dedupedRecords, + Partitioner partitioner) { + return dedupedRecords + .mapToPair(record -> + new Tuple2<>( + new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), + record)) + .partitionBy(partitioner) + .map(tuple -> tuple._2()); + } + + /** + * Commit changes performed at the given commitTime marker + */ + public boolean commit(String commitTime, JavaRDD writeStatuses) { + return commit(commitTime, writeStatuses, Optional.empty()); + } + + /** + * Commit changes performed at the given commitTime marker + */ + public boolean commit(String commitTime, + JavaRDD writeStatuses, + Optional> extraMetadata) { + + logger.info("Commiting " + commitTime); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + + List> stats = writeStatuses + .mapToPair((PairFunction) writeStatus -> + new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) + .collect(); + + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + for (Tuple2 stat : stats) { + metadata.addWriteStat(stat._1(), stat._2()); + } + + // Finalize write + final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); + final Optional result = table.finalizeWrite(jsc, stats); + if (finalizeCtx != null && result.isPresent()) { + Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); + durationInMs.ifPresent(duration -> { + logger.info("Finalize write elapsed time (milliseconds): " + duration); + metrics.updateFinalizeWriteMetrics(duration, result.get()); + } + ); + } + + // add in extra metadata + if (extraMetadata.isPresent()) { + extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v)); + } + + try { + String actionType = table.getCommitActionType(); + activeTimeline.saveAsComplete( + new HoodieInstant(true, actionType, commitTime), + Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + // Save was a success + // Do a inline compaction if enabled + if (config.isInlineCompaction()) { + metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); + forceCompact(); + } else { + metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); + } + + // We cannot have unbounded commit files. Archive commits if we have to archive + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); + archiveLog.archiveIfRequired(); + if (config.isAutoClean()) { + // Call clean to cleanup if there is anything to cleanup after the commit, + logger.info("Auto cleaning is enabled. Running cleaner now"); + clean(commitTime); + } else { + logger.info("Auto cleaning is not enabled. Not running cleaner now"); + } + if (writeContext != null) { + long durationInMs = metrics.getDurationInMs(writeContext.stop()); + metrics.updateCommitMetrics( + HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs, + metadata); + writeContext = null; + } + logger.info("Committed " + commitTime); + } catch (IOException e) { + throw new HoodieCommitException( + "Failed to commit " + config.getBasePath() + " at time " + commitTime, e); + } catch (ParseException e) { + throw new HoodieCommitException( + "Commit time is not of valid format.Failed to commit " + config.getBasePath() + + " at time " + commitTime, e); + } + return true; + } + + /** + * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will + * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be + * rolledback or archived. + * + * This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be + * manually created and deleted. + * + * Savepoint should be on a commit that could not have been cleaned. + * + * @param user - User creating the savepoint + * @param comment - Comment for the savepoint + * @return true if the savepoint was created successfully + */ + public boolean savepoint(String user, String comment) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + if (table.getCompletedCommitTimeline().empty()) { + throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); + } + + String latestCommit = table.getCompletedCommitTimeline().lastInstant().get().getTimestamp(); + logger.info("Savepointing latest commit " + latestCommit); + return savepoint(latestCommit, user, comment); + } + + /** + * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will + * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be + * rolledback or archived. + * + * This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be + * manually created and deleted. + * + * Savepoint should be on a commit that could not have been cleaned. + * + * @param commitTime - commit that should be savepointed + * @param user - User creating the savepoint + * @param comment - Comment for the savepoint + * @return true if the savepoint was created successfully + */ + public boolean savepoint(String commitTime, String user, String comment) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); + + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, + commitTime); + if (!table.getCompletedCommitTimeline().containsInstant(commitInstant)) { + throw new HoodieSavepointException( + "Could not savepoint non-existing commit " + commitInstant); + } + + try { + // Check the last commit that was not cleaned and check if savepoint time is > that commit + String lastCommitRetained; + if (cleanInstant.isPresent()) { + HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata( + table.getActiveTimeline().getInstantDetails(cleanInstant.get()).get()); + lastCommitRetained = cleanMetadata.getEarliestCommitToRetain(); + } else { + lastCommitRetained = + table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); + } + + // Cannot allow savepoint time on a commit that could have been cleaned + Preconditions.checkArgument(HoodieTimeline + .compareTimestamps(commitTime, lastCommitRetained, HoodieTimeline.GREATER_OR_EQUAL), + "Could not savepoint commit " + commitTime + " as this is beyond the lookup window " + + lastCommitRetained); + + Map> latestFilesMap = jsc.parallelize( + FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning())) + .mapToPair((PairFunction>) partitionPath -> { + // Scan all partitions files with this commit time + logger.info("Collecting latest files in partition path " + partitionPath); + TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); + List latestFiles = + view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) + .map(HoodieDataFile::getFileName).collect(Collectors.toList()); + return new Tuple2<>(partitionPath, latestFiles); + }).collectAsMap(); + + HoodieSavepointMetadata metadata = + AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap); + // Nothing to save in the savepoint + table.getActiveTimeline().saveAsComplete( + new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime), + AvroUtils.serializeSavepointMetadata(metadata)); + logger.info("Savepoint " + commitTime + " created"); + return true; + } catch (IOException e) { + throw new HoodieSavepointException("Failed to savepoint " + commitTime, e); + } + } + + /** + * Delete a savepoint that was created. Once the savepoint is deleted, the commit can be + * rolledback and cleaner may clean up data files. + * + * @param savepointTime - delete the savepoint + * @return true if the savepoint was deleted successfully + */ + public void deleteSavepoint(String savepointTime) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + + HoodieInstant savePoint = + new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); + boolean isSavepointPresent = + table.getCompletedSavepointTimeline().containsInstant(savePoint); + if (!isSavepointPresent) { + logger.warn("No savepoint present " + savepointTime); + return; + } + + activeTimeline.revertToInflight(savePoint); + activeTimeline.deleteInflight( + new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); + logger.info("Savepoint " + savepointTime + " deleted"); + } + + /** + * Rollback the state to the savepoint. WARNING: This rollsback recent commits and deleted data + * files. Queries accessing the files will mostly fail. This should be done during a downtime. + * + * @param savepointTime - savepoint time to rollback to + * @return true if the savepoint was rollecback to successfully + */ + public boolean rollbackToSavepoint(String savepointTime) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieTimeline commitTimeline = table.getCommitsTimeline(); + + HoodieInstant savePoint = + new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); + boolean isSavepointPresent = + table.getCompletedSavepointTimeline().containsInstant(savePoint); + if (!isSavepointPresent) { + throw new HoodieRollbackException("No savepoint for commitTime " + savepointTime); + } + + List commitsToRollback = + commitTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + logger.info("Rolling back commits " + commitsToRollback); + + rollback(commitsToRollback); + + // Make sure the rollback was successful + Optional lastInstant = + activeTimeline.reload().getCommitsTimeline().filterCompletedInstants() + .lastInstant(); + Preconditions.checkArgument(lastInstant.isPresent()); + Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), + savepointTime + "is not the last commit after rolling back " + commitsToRollback + + ", last commit was " + lastInstant.get().getTimestamp()); + return true; + } + + /** + * Rollback the (inflight/committed) record changes with the given commit time. Three steps: (1) + * Atomically unpublish this commit (2) clean indexing data, (3) clean new generated parquet + * files. (4) Finally delete .commit or .inflight file, + */ + public boolean rollback(final String commitTime) throws HoodieRollbackException { + rollback(Lists.newArrayList(commitTime)); + return true; + } + + private void rollback(List commits) { + if (commits.isEmpty()) { + logger.info("List of commits to rollback is empty"); + return; + } + + final Timer.Context context = metrics.getRollbackCtx(); + String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); + + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); + HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); + + // Check if any of the commits is a savepoint - do not allow rollback on those commits + List savepoints = + table.getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); - Collections.reverse(commits); - for (String commit : commits) { - rollback(commit); + commits.forEach(s -> { + if (savepoints.contains(s)) { + throw new HoodieRollbackException( + "Could not rollback a savepointed commit. Delete savepoint first before rolling back" + + s); + } + }); + + try { + if (commitTimeline.empty() && inflightTimeline.empty()) { + // nothing to rollback + logger.info("No commits to rollback " + commits); + } + + // Make sure only the last n commits are being rolled back + // If there is a commit in-between or after that is not rolled back, then abort + String lastCommit = commits.get(commits.size() - 1); + if (!commitTimeline.empty() && !commitTimeline + .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { + throw new HoodieRollbackException("Found commits after time :" + lastCommit + + ", please rollback greater commits first"); + } + + List inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) { + throw new HoodieRollbackException( + "Found in-flight commits after time :" + lastCommit + + ", please rollback greater commits first"); + } + + List stats = table.rollback(jsc, commits); + + // cleanup index entries + commits.stream().forEach(s -> { + if (!index.rollbackCommit(s)) { + throw new HoodieRollbackException( + "Rollback index changes failed, for time :" + s); } - } + }); + logger.info("Index rolled back for commits " + commits); + + Optional durationInMs = Optional.empty(); + if (context != null) { + durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); + Long numFilesDeleted = stats.stream() + .mapToLong(stat -> stat.getSuccessDeleteFiles().size()) + .sum(); + metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); + } + HoodieRollbackMetadata rollbackMetadata = + AvroUtils.convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); + table.getActiveTimeline().saveAsComplete( + new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime), + AvroUtils.serializeRollbackMetadata(rollbackMetadata)); + logger.info("Commits " + commits + " rollback is complete"); + + if (!table.getActiveTimeline().getCleanerTimeline().empty()) { + logger.info("Cleaning up older rollback meta files"); + // Cleanup of older cleaner meta files + // TODO - make the commit archival generic and archive rollback metadata + FSUtils.deleteOlderRollbackMetaFiles(fs, table.getMetaClient().getMetaPath(), + table.getActiveTimeline().getRollbackTimeline().getInstants()); + } + } catch (IOException e) { + throw new HoodieRollbackException("Failed to rollback " + + config.getBasePath() + " commits " + commits, e); + } + } + + /** + * Releases any resources used by the client. + */ + public void close() { + // UNDER CONSTRUCTION + } + + /** + * Clean up any stale/old files/data lying around (either on file storage or index storage) based + * on the configurations and CleaningPolicy used. (typically files that no longer can be used by a + * running query can be cleaned) + */ + public void clean() throws HoodieIOException { + String startCleanTime = HoodieActiveTimeline.createNewCommitTime(); + clean(startCleanTime); + } + + /** + * Clean up any stale/old files/data lying around (either on file storage or index storage) based + * on the configurations and CleaningPolicy used. (typically files that no longer can be used by a + * running query can be cleaned) + */ + private void clean(String startCleanTime) throws HoodieIOException { + try { + logger.info("Cleaner started"); + final Timer.Context context = metrics.getCleanCtx(); + + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), + true), config); + + List cleanStats = table.clean(jsc); + if (cleanStats.isEmpty()) { + return; + } + + // Emit metrics (duration, numFilesDeleted) if needed + Optional durationInMs = Optional.empty(); + if (context != null) { + durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); + logger.info("cleanerElaspsedTime (Minutes): " + durationInMs.get() / (1000 * 60)); + } + + // Create the metadata and save it + HoodieCleanMetadata metadata = + AvroUtils.convertCleanMetadata(startCleanTime, durationInMs, cleanStats); + logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"); + metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), + metadata.getTotalFilesDeleted()); + + table.getActiveTimeline().saveAsComplete( + new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime), + AvroUtils.serializeCleanMetadata(metadata)); + logger.info("Marked clean started on " + startCleanTime + " as complete"); + + if (!table.getActiveTimeline().getCleanerTimeline().empty()) { + // Cleanup of older cleaner meta files + // TODO - make the commit archival generic and archive clean metadata + FSUtils.deleteOlderCleanMetaFiles(fs, table.getMetaClient().getMetaPath(), + table.getActiveTimeline().getCleanerTimeline().getInstants()); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to clean up after commit", e); + } + } + + /** + * Provides a new commit time for a write operation (insert/update) + */ + public String startCommit() { + String commitTime = HoodieActiveTimeline.createNewCommitTime(); + startCommitWithTime(commitTime); + return commitTime; + } + + public void startCommitWithTime(String commitTime) { + logger.info("Generate a new commit time " + commitTime); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + String commitActionType = table.getCommitActionType(); + activeTimeline.createInflight( + new HoodieInstant(true, commitActionType, commitTime)); + } + + /** + * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed + * asynchronously. Please always use this serially before or after an insert/upsert action. + */ + private void compact(String compactionCommitTime) throws IOException { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + Optional compactionMetadata = table.compact(jsc, compactionCommitTime); + if (compactionMetadata.isPresent()) { + logger.info("Compacted successfully on commit " + compactionCommitTime); + } else { + logger.info("Compaction did not run for commit " + compactionCommitTime); + } + } + + /** + * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed + * asynchronously. Please always use this serially before or after an insert/upsert action. + */ + public String forceCompact() throws IOException { + String compactionCommitTime = HoodieActiveTimeline.createNewCommitTime(); + compact(compactionCommitTime); + return compactionCommitTime; + } + + public static SparkConf registerClasses(SparkConf conf) { + conf.registerKryoClasses( + new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); + return conf; + } + + /** + * Deduplicate Hoodie records, using the given deduplication funciton. + */ + private JavaRDD> deduplicateRecords(JavaRDD> records, + int parallelism) { + return records + .mapToPair(record -> new Tuple2<>(record.getKey(), record)) + .reduceByKey((rec1, rec2) -> { + @SuppressWarnings("unchecked") + T reducedData = (T) rec1.getData().preCombine(rec2.getData()); + // we cannot allow the user to change the key or partitionPath, since that will affect everything + // so pick it from one of the records. + return new HoodieRecord(rec1.getKey(), reducedData); + }, parallelism) + .map(recordTuple -> recordTuple._2()); + } + + /** + * Cleanup all inflight commits + */ + private void rollbackInflightCommits() { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); + List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + Collections.reverse(commits); + for (String commit : commits) { + rollback(commit); + } + } } From 269c443e2e7a3d2583a4f19d415e937d4c8de537 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Thu, 1 Feb 2018 12:36:12 -0800 Subject: [PATCH 022/374] Write smaller sized multiple blocks to log file instead of a large one - Use SizeEstimator to size number of records to write - Configurable block size - Configurable log file size --- docs/configurations.md | 4 + .../hoodie/config/HoodieStorageConfig.java | 20 +++ .../uber/hoodie/config/HoodieWriteConfig.java | 9 ++ .../uber/hoodie/io/HoodieAppendHandle.java | 140 ++++++++++-------- 4 files changed, 111 insertions(+), 62 deletions(-) diff --git a/docs/configurations.md b/docs/configurations.md index 51f2d315b3b94..328c5a6e72191 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -53,6 +53,10 @@ summary: "Here we list all possible configurations and what they mean" Parquet RowGroup size. Its better than this is aligned with the file size, so that a single column within a file is stored continuously on disk - [parquetPageSize](#parquetPageSize) (pagesize = 1MB)
Parquet page size. Page is the unit of read within a parquet file. Within a block, pages are compressed seperately. + - [logFileMaxSize](#logFileMaxSize) (logFileSize = 1GB)
+ LogFile max size. This is the maximum size allowed for a log file before it is rolled over to the next version. + - [logFileDataBlockMaxSize](#logFileDataBlockMaxSize) (dataBlockSize = 256MB)
+ LogFile Data block max size. This is the maximum size allowed for a single data block to be appended to a log file. This helps to make sure the data appended to the log file is broken up into sizable blocks to prevent from OOM errors. This size should be greater than the JVM memory. - [withCompactionConfig](#withCompactionConfig) (HoodieCompactionConfig)
Cleaning and configurations related to compaction techniques diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java index cc4e6d4652e7d..c9ca6aa4b8d31 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java @@ -34,6 +34,12 @@ public class HoodieStorageConfig extends DefaultHoodieConfig { public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES; public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size"; public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024); + // used to size log files + public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size"; + public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024*1024*1024); // 1 GB + // used to size data blocks in log file + public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size"; + public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256*1024*1024); // 256 MB private HoodieStorageConfig(Properties props) { super(props); @@ -77,6 +83,16 @@ public Builder parquetPageSize(int pageSize) { return this; } + public Builder logFileDataBlockMaxSize(int dataBlockSize) { + props.setProperty(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize)); + return this; + } + + public Builder logFileMaxSize(int logFileSize) { + props.setProperty(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize)); + return this; + } + public HoodieStorageConfig build() { HoodieStorageConfig config = new HoodieStorageConfig(props); setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), @@ -85,6 +101,10 @@ public HoodieStorageConfig build() { PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES); setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES), PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES); + setDefaultOnCondition(props, !props.containsKey(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES), + LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES); + setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES), + LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 9bbc8fa705ff1..7b167ebbd98c9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -277,6 +277,15 @@ public int getParquetPageSize() { return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES)); } + public int getLogFileDataBlockMaxSize() { + return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES)); + } + + public int getLogFileMaxSize() { + return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES)); + } + + /** * metrics properties **/ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 9752dc4444e6b..35f7518994a2a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -17,7 +17,6 @@ package com.uber.hoodie.io; import com.beust.jcommander.internal.Maps; -import com.clearspring.analytics.util.Lists; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDeltaWriteStat; @@ -38,6 +37,14 @@ import com.uber.hoodie.exception.HoodieAppendException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.table.HoodieTable; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.util.SizeEstimator; + import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -45,30 +52,28 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; /** * IO Operation to append data onto an existing file. */ public class HoodieAppendHandle extends HoodieIOHandle { - private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class); + private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); private static AtomicLong recordIndex = new AtomicLong(1); private TableFileSystemView.RealtimeView fileSystemView; private final WriteStatus writeStatus; private final String fileId; private String partitionPath; - private List> records; + private Iterator> recordItr; + List recordList = new ArrayList<>(); + List keysToDelete = new ArrayList<>(); private long recordsWritten = 0; private long recordsDeleted = 0; + private long averageRecordSize = 0; private HoodieLogFile currentLogFile; private Writer writer; + private boolean doInit = true; public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, @@ -81,55 +86,48 @@ public HoodieAppendHandle(HoodieWriteConfig config, this.writeStatus = writeStatus; this.fileId = fileId; this.fileSystemView = hoodieTable.getRTFileSystemView(); - init(recordItr); + this.recordItr = recordItr; } - private void init(Iterator> recordItr) { - List> records = Lists.newArrayList(); - recordItr.forEachRemaining(record -> { - records.add(record); - // extract some information from the first record - if (partitionPath == null) { - partitionPath = record.getPartitionPath(); - FileSlice fileSlice = fileSystemView.getLatestFileSlices(record.getPartitionPath()) - .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)) - .findFirst().get(); - // HACK(vc) This also assumes a base file. It will break, if appending without one. - String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); - String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); - writeStatus.getStat().setPrevCommit(baseCommitTime); - writeStatus.setFileId(fileId); - writeStatus.setPartitionPath(record.getPartitionPath()); - writeStatus.getStat().setFileId(fileId); - - try { - this.writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) - .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles() - .max(HoodieLogFile.getLogVersionComparator()::compare) - .map(logFile -> logFile.getLogVersion()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) - .withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - this.currentLogFile = writer.getLogFile(); - ((HoodieDeltaWriteStat) writeStatus.getStat()) - .setLogVersion(currentLogFile.getLogVersion()); - ((HoodieDeltaWriteStat) writeStatus.getStat()) - .setLogOffset(writer.getCurrentSize()); - } catch (Exception e) { - logger.error("Error in update task at commit " + commitTime, e); - writeStatus.setGlobalError(e); - throw new HoodieUpsertException( - "Failed to initialize HoodieUpdateHandle for FileId: " + fileId - + " on commit " + commitTime + " on HDFS path " + hoodieTable - .getMetaClient().getBasePath() + partitionPath, e); - } - Path path = new Path(record.getPartitionPath(), - FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); - writeStatus.getStat().setPath(path.toString()); + private void init(String partitionPath) { + + // extract some information from the first record + FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath) + .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)) + .findFirst().get(); + // HACK(vc) This also assumes a base file. It will break, if appending without one. + String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); + String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); + writeStatus.getStat().setPrevCommit(baseCommitTime); + writeStatus.setFileId(fileId); + writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setFileId(fileId); + this.partitionPath = partitionPath; + + try { + this.writer = HoodieLogFormat.newWriterBuilder() + .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) + .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles() + .max(HoodieLogFile.getLogVersionComparator().reversed()::compare) + .map(logFile -> logFile.getLogVersion()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) + .withSizeThreshold(config.getLogFileMaxSize()) + .withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + this.currentLogFile = writer.getLogFile(); + ((HoodieDeltaWriteStat) writeStatus.getStat()) + .setLogVersion(currentLogFile.getLogVersion()); + ((HoodieDeltaWriteStat) writeStatus.getStat()) + .setLogOffset(writer.getCurrentSize()); + } catch (Exception e) { + logger.error("Error in update task at commit " + commitTime, e); + writeStatus.setGlobalError(e); + throw new HoodieUpsertException( + "Failed to initialize HoodieUpdateHandle for FileId: " + fileId + + " on commit " + commitTime + " on HDFS path " + hoodieTable + .getMetaClient().getBasePath() + partitionPath, e); } - // update the new location of the record, so we know where to find it next - record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); - }); - this.records = records; + Path path = new Path(partitionPath, + FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); + writeStatus.getStat().setPath(path.toString()); } private Optional getIndexedRecord(HoodieRecord hoodieRecord) { @@ -162,28 +160,47 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { public void doAppend() { - List recordList = new ArrayList<>(); - List keysToDelete = new ArrayList<>(); + int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0; Map metadata = Maps.newHashMap(); metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime); - Iterator> recordsItr = records.iterator(); - while (recordsItr.hasNext()) { - HoodieRecord record = recordsItr.next(); + while (recordItr.hasNext()) { + HoodieRecord record = recordItr.next(); + // update the new location of the record, so we know where to find it next + record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); + if(doInit) { + init(record.getPartitionPath()); + averageRecordSize = SizeEstimator.estimate(record); + doInit = false; + } + // Append if max number of records reached to achieve block size + if(numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { + // Recompute averageRecordSize before writing a new block and update existing value with avg of new and old + logger.info("AvgRecordSize => " + averageRecordSize); + averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2; + doAppend(metadata); + numberOfRecords = 0; + } Optional indexedRecord = getIndexedRecord(record); if (indexedRecord.isPresent()) { recordList.add(indexedRecord.get()); } else { keysToDelete.add(record.getRecordKey()); } - recordsItr.remove(); //remove entries when IndexedRecord added to new list + numberOfRecords++; } + doAppend(metadata); + } + + private void doAppend(Map metadata) { try { if (recordList.size() > 0) { writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata)); + recordList.clear(); } if (keysToDelete.size() > 0) { writer = writer.appendBlock( new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata)); + keysToDelete.clear(); } } catch (Exception e) { throw new HoodieAppendException( @@ -208,5 +225,4 @@ public WriteStatus getWriteStatus() { return writeStatus; } - } From 56a19626611b83ab859d11346507730b5d7be42f Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 6 Dec 2017 13:11:27 -0800 Subject: [PATCH 023/374] Added support for Disk Spillable Compaction to prevent OOM issues --- .../hoodie/config/HoodieCompactionConfig.java | 19 +- .../uber/hoodie/config/HoodieWriteConfig.java | 5 + .../uber/hoodie/io/HoodieAppendHandle.java | 5 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 1 + .../compact/HoodieRealtimeTableCompactor.java | 20 +- .../log/HoodieCompactedLogRecordScanner.java | 70 ++-- .../hoodie/common/util/SpillableMapUtils.java | 155 +++++++++ .../common/util/collection/DiskBasedMap.java | 326 ++++++++++++++++++ .../util/collection/ExternalSpillableMap.java | 256 ++++++++++++++ .../util/collection/LazyFileIterable.java | 116 +++++++ .../io/storage/SizeAwareDataOutputStream.java | 69 ++++ .../HoodieCorruptedDataException.java | 32 ++ .../common/table/log/HoodieLogFormatTest.java | 69 ++-- .../hoodie/common/util/SchemaTestUtil.java | 23 +- .../common/util/SpillableMapTestUtils.java | 47 +++ .../util/collection/TestDiskBasedMap.java | 107 ++++++ .../collection/TestExternalSpillableMap.java | 245 +++++++++++++ .../realtime/HoodieRealtimeRecordReader.java | 8 +- 18 files changed, 1487 insertions(+), 86 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index 39f076e5a2fd2..75dd49d6cfadd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -64,7 +64,6 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { // Turned off by default public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0); - /** * Configs related to specific table types **/ @@ -102,6 +101,10 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName(); public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class"; + public static final String MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP = "hoodie.compaction.spill.threshold"; + // Default memory size per compaction, excess spills to disk + public static final String DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES = String.valueOf(1024*1024*1024L); //1GB + private HoodieCompactionConfig(Properties props) { super(props); } @@ -210,6 +213,18 @@ public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) { return this; } + public Builder withMaxMemorySizePerCompactionInBytes(long maxMemorySizePerCompactionInBytes) { + props.setProperty(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, + String.valueOf(maxMemorySizePerCompactionInBytes)); + return this; + } + + public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) { + props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, + String.valueOf(maxNumDeltaCommitsBeforeCompaction)); + return this; + } + public HoodieCompactionConfig build() { HoodieCompactionConfig config = new HoodieCompactionConfig(props); setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), @@ -245,6 +260,8 @@ public HoodieCompactionConfig build() { PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS); setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP), TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB); + setDefaultOnCondition(props, !props.containsKey(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP), + MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES); HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP)); Preconditions.checkArgument( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 7b167ebbd98c9..9933f3c9c2333 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -210,6 +210,11 @@ public Long getTargetIOPerCompactionInMB() { .parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP)); } + public Long getMaxMemorySizePerCompactionInBytes() { + return Long + .parseLong(props.getProperty(HoodieCompactionConfig.MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP)); + } + /** * index properties **/ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 35f7518994a2a..0720b133f403d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -47,6 +47,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -108,8 +109,8 @@ private void init(String partitionPath) { this.writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles() - .max(HoodieLogFile.getLogVersionComparator().reversed()::compare) - .map(logFile -> logFile.getLogVersion()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) + .map(logFile -> logFile.getLogVersion()) + .max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) .withSizeThreshold(config.getLogFileMaxSize()) .withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); this.currentLogFile = writer.getLogFile(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index a974e8b38573c..5dbe39d1f4ce6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -142,6 +142,7 @@ private void init(String fileId, String partitionPath) { */ private String init(String fileId, Iterator> newRecordsItr) { // Load the new records in a map + // TODO (NA) instantiate a ExternalSpillableMap this.keyToNewRecords = new HashMap<>(); String partitionPath = null; while (newRecordsItr.hasNext()) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 6e89c38cc4a8e..c72406d126297 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -16,8 +16,6 @@ package com.uber.hoodie.io.compact; -import static java.util.stream.Collectors.toList; - import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -38,6 +36,13 @@ import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Collection; @@ -46,12 +51,8 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FlatMapFunction; + +import static java.util.stream.Collectors.toList; /** * HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all @@ -73,6 +74,7 @@ public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() .getTableType().name()); + //TODO : check if maxMemory is not greater than JVM or spark.executor memory // TODO - rollback any compactions in flight HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); @@ -152,7 +154,7 @@ private List executeCompaction(HoodieTable hoodieTable, HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(), - operation.getDeltaFilePaths(), readerSchema, maxInstantTime); + operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes()); if (!scanner.iterator().hasNext()) { return Lists.newArrayList(); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index f01e15f8e233c..465b53867821d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -27,7 +27,8 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.util.ReflectionUtils; +import com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.common.util.collection.ExternalSpillableMap; import com.uber.hoodie.exception.HoodieIOException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -40,10 +41,7 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.Deque; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -64,7 +62,7 @@ public class HoodieCompactedLogRecordScanner implements private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); // Final map of compacted/merged records - private final Map> records; + private final ExternalSpillableMap> records; // Reader schema for the records private final Schema readerSchema; // Total log files read - for metrics @@ -82,22 +80,24 @@ public class HoodieCompactedLogRecordScanner implements Deque currentInstantLogBlocks = new ArrayDeque<>(); public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime) { + Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); // load class from the payload fully qualified class name this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); - // Store merged records for all versions for this log file - this.records = Maps.newHashMap(); - // iterate over the paths - Iterator logFilePathsItr = logFilePaths.iterator(); - while (logFilePathsItr.hasNext()) { - HoodieLogFile logFile = new HoodieLogFile(new Path(logFilePathsItr.next())); - log.info("Scanning log file " + logFile.getPath()); - totalLogFiles.incrementAndGet(); - try { + try { + // Store merged records for all versions for this log file, set the maxInMemoryMapSize to half, + // assign other half to the temporary map needed to read next block + records = new ExternalSpillableMap<>(maxMemorySizeInBytes, readerSchema, + payloadClassFQN, Optional.empty()); + // iterate over the paths + Iterator logFilePathsItr = logFilePaths.iterator(); + while (logFilePathsItr.hasNext()) { + HoodieLogFile logFile = new HoodieLogFile(new Path(logFilePathsItr.next())); + log.info("Scanning log file " + logFile.getPath()); + totalLogFiles.incrementAndGet(); // Use the HoodieLogFormatReader to iterate through the blocks in the log file HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, logFile, readerSchema, true); while (reader.hasNext()) { @@ -193,17 +193,21 @@ else if(!targetInstantForCommandBlock break; } } - - } catch (IOException e) { - throw new HoodieIOException("IOException when reading log file " + logFile); - } - // merge the last read block when all the blocks are done reading - if (!currentInstantLogBlocks.isEmpty()) { - log.info("Merging the final data blocks in " + logFile.getPath()); - merge(records, currentInstantLogBlocks); + // merge the last read block when all the blocks are done reading + if (!currentInstantLogBlocks.isEmpty()) { + log.info("Merging the final blocks in " + logFile.getPath()); + merge(records, currentInstantLogBlocks); + } } + } catch (IOException e) { + throw new HoodieIOException("IOException when reading compacting log files"); } this.totalRecordsToUpdate = records.size(); + log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); + log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); + log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); + log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); + log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } /** @@ -223,21 +227,15 @@ private boolean isNewInstantBlock(HoodieLogBlock logBlock) { * the log records since the base data is merged on previous compaction */ private Map> loadRecordsFromBlock( - HoodieAvroDataBlock dataBlock) { - Map> recordsFromLastBlock = Maps - .newHashMap(); + HoodieAvroDataBlock dataBlock) throws IOException { + Map> recordsFromLastBlock = Maps.newHashMap(); List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); recs.forEach(rec -> { String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD) .toString(); - String partitionPath = - ((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD) - .toString(); - HoodieRecord hoodieRecord = new HoodieRecord<>( - new HoodieKey(key, partitionPath), - ReflectionUtils - .loadPayload(this.payloadClassFQN, new Object[]{Optional.of(rec)}, Optional.class)); + HoodieRecord hoodieRecord = + SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); if (recordsFromLastBlock.containsKey(key)) { // Merge and store the merged record HoodieRecordPayload combinedValue = recordsFromLastBlock.get(key).getData() @@ -257,7 +255,7 @@ private Map> loadRecordsFrom * Merge the last seen log blocks with the accumulated records */ private void merge(Map> records, - Deque lastBlocks) { + Deque lastBlocks) throws IOException { while (!lastBlocks.isEmpty()) { // poll the element at the bottom of the stack since that's the order it was inserted HoodieLogBlock lastBlock = lastBlocks.pollLast(); @@ -280,7 +278,7 @@ private void merge(Map> reco * Merge the records read from a single data block with the accumulated records */ private void merge(Map> records, - Map> recordsFromLastBlock) { + Map> recordsFromLastBlock) { recordsFromLastBlock.forEach((key, hoodieRecord) -> { if (records.containsKey(key)) { // Merge and store the merged record @@ -297,7 +295,7 @@ private void merge(Map> reco @Override public Iterator> iterator() { - return records.values().iterator(); + return records.iterator(); } public long getTotalLogFiles() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java new file mode 100644 index 0000000000000..4f915897e872f --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -0,0 +1,155 @@ +/* + * 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; + +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.util.collection.DiskBasedMap; +import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; +import com.uber.hoodie.exception.HoodieCorruptedDataException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Optional; +import java.util.zip.CRC32; + +public class SpillableMapUtils { + + public static ObjectMapper objectMapper = new ObjectMapper(); + /** + * Using the schema and payload class, read and convert the bytes on disk to a HoodieRecord + * @param file + * @param schema + * @param payloadClazz + * @param valuePosition + * @param valueLength + * @param + * @return + * @throws IOException + */ + public static R readFromDisk(RandomAccessFile file, Schema schema, String payloadClazz, + long valuePosition, int valueLength) throws IOException { + + DiskBasedMap.FileEntry fileEntry = readInternal(file, valuePosition, valueLength); + return (R) convertToHoodieRecordPayload(HoodieAvroUtils.bytesToAvro(fileEntry.getValue(), schema), + payloadClazz); + } + + /** + * |crc|timestamp|sizeOfKey|SizeOfValue|key|value| + * @param file + * @param valuePosition + * @param valueLength + * @return + * @throws IOException + */ + private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition, int valueLength) throws IOException { + file.seek(valuePosition); + long crc = file.readLong(); + long timestamp = file.readLong(); + int keySize = file.readInt(); + int valueSize = file.readInt(); + byte [] key = new byte[keySize]; + file.read(key, 0, keySize); + byte [] value = new byte[valueSize]; + if(!(valueSize == valueLength)) { + throw new HoodieCorruptedDataException("unequal size of payload written to external file, data may be corrupted"); + } + file.read(value, 0, valueSize); + long crcOfReadValue = generateChecksum(value); + if(!(crc == crcOfReadValue)) { + throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " + + "data may be corrupted"); + } + return new DiskBasedMap.FileEntry(crc, keySize, valueSize, key, value, timestamp); + } + + /** + * Write Value and other metadata necessary to disk. Each entry has the following sequence of data + * + * |crc|timestamp|sizeOfKey|SizeOfValue|key|value| + * + * @param outputStream + * @param fileEntry + * @return + * @throws IOException + */ + public static long spillToDisk(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) throws IOException { + return spill(outputStream, fileEntry); + } + + private static long spill(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) + throws IOException { + outputStream.writeLong(fileEntry.getCrc()); + outputStream.writeLong(fileEntry.getTimestamp()); + outputStream.writeInt(fileEntry.getSizeOfKey()); + outputStream.writeInt(fileEntry.getSizeOfValue()); + outputStream.write(fileEntry.getKey()); + outputStream.write(fileEntry.getValue()); + return outputStream.getSize(); + } + + /** + * Generate a checksum for a given set of bytes + * @param data + * @return + */ + public static long generateChecksum(byte [] data) { + CRC32 crc = new CRC32(); + crc.update(data); + return crc.getValue(); + } + + /** + * Compute a bytes representation of the payload by serializing the contents + * This is used to estimate the size of the payload (either in memory or when written to disk) + * @param + * @param value + * @param schema + * @return + * @throws IOException + */ + public static int computePayloadSize(R value, Schema schema) throws IOException { + HoodieRecord payload = (HoodieRecord) value; + byte [] val = HoodieAvroUtils.avroToBytes((GenericRecord) payload.getData().getInsertValue(schema).get()); + return val.length; + } + + /** + * Utility method to convert bytes to HoodieRecord using schema and payload class + * @param rec + * @param payloadClazz + * @param + * @return + * @throws IOException + */ + public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz) { + String recKey = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD) + .toString(); + String partitionPath = + rec.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + .toString(); + HoodieRecord hoodieRecord = new HoodieRecord<>( + new HoodieKey(recKey, partitionPath), + ReflectionUtils + .loadPayload(payloadClazz, new Object[]{Optional.of(rec)}, Optional.class)); + return (R) hoodieRecord; + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java new file mode 100644 index 0000000000000..2503c409fb554 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -0,0 +1,326 @@ +/* + * 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 com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieNotSupportedException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; + +/** + * This class provides a disk spillable only map implementation. All of the data is + * currenly written to one file, without any rollover support. It uses the following : + * 1) An in-memory map that tracks the key-> latest ValueMetadata. + * 2) Current position in the file + * NOTE : Only String.class type supported for Key + * @param + * @param + */ +final public class DiskBasedMap implements Map { + + // Stores the key and corresponding value's latest metadata spilled to disk + final private Map inMemoryMetadataOfSpilledData; + // Read only file access to be able to seek to random positions to readFromDisk values + private RandomAccessFile readOnlyFileHandle; + // Write only OutputStream to be able to ONLY append to the file + private SizeAwareDataOutputStream writeOnlyFileHandle; + // FileOutputStream for the file handle to be able to force fsync + // since FileOutputStream's flush() does not force flush to disk + private FileOutputStream fileOutputStream; + // Current position in the file + private AtomicLong filePosition; + // Schema used to de-serialize payload written to disk + private Schema schema; + // Class used to de-serialize/realize payload written to disk + private String payloadClazz; + // FilePath to store the spilled data + private String filePath; + // Default file path prefix to put the spillable file + private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; + + public final class ValueMetadata { + // FilePath to store the spilled data + private String filePath; + // Size (numberOfBytes) of the value written to disk + private Integer sizeOfValue; + // FilePosition of the value written to disk + private Long offsetOfValue; + // Current timestamp when the value was written to disk + private Long timestamp; + + protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) { + this.filePath = filePath; + this.sizeOfValue = sizeOfValue; + this.offsetOfValue = offsetOfValue; + this.timestamp = timestamp; + } + + public String getFilePath() { + return filePath; + } + + public int getSizeOfValue() { + return sizeOfValue; + } + + public Long getOffsetOfValue() { + return offsetOfValue; + } + + public long getTimestamp() { + return timestamp; + } + } + + public static final class FileEntry { + // Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption + private Long crc; + // Size (numberOfBytes) of the key written to disk + private Integer sizeOfKey; + // Size (numberOfBytes) of the value written to disk + private Integer sizeOfValue; + // Actual key + private byte [] key; + // Actual value + private byte [] value; + // Current timestamp when the value was written to disk + private Long timestamp; + + public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte [] key, byte [] value, long timestamp) { + this.crc = crc; + this.sizeOfKey = sizeOfKey; + this.sizeOfValue = sizeOfValue; + this.key = key; + this.value = value; + this.timestamp = timestamp; + } + + public long getCrc() { + return crc; + } + + public int getSizeOfKey() { + return sizeOfKey; + } + + public int getSizeOfValue() { + return sizeOfValue; + } + + public byte[] getKey() { + return key; + } + + public byte[] getValue() { + return value; + } + + public long getTimestamp() { + return timestamp; + } + } + + protected DiskBasedMap(Schema schema, String payloadClazz, Optional baseFilePath) throws IOException { + this.inMemoryMetadataOfSpilledData = new HashMap<>(); + + if(!baseFilePath.isPresent()) { + baseFilePath = Optional.of(DEFAULT_BASE_FILE_PATH); + } + this.filePath = baseFilePath.get() + UUID.randomUUID().toString(); + File writeOnlyFileHandle = new File(filePath); + initFile(writeOnlyFileHandle); + + this.fileOutputStream = new FileOutputStream(writeOnlyFileHandle, true); + this.writeOnlyFileHandle = new SizeAwareDataOutputStream(fileOutputStream); + this.filePosition = new AtomicLong(0L); + this.schema = schema; + this.payloadClazz = payloadClazz; + } + + private void initFile(File writeOnlyFileHandle) throws IOException { + // delete the file if it exists + if(writeOnlyFileHandle.exists()) { + writeOnlyFileHandle.delete(); + } + writeOnlyFileHandle.createNewFile(); + // Open file in readFromDisk-only mode + readOnlyFileHandle = new RandomAccessFile(filePath, "r"); + readOnlyFileHandle.seek(0); + // Make sure file is deleted when JVM exits + writeOnlyFileHandle.deleteOnExit(); + addShutDownHook(); + } + + /** + * Register shutdown hook to force flush contents of the data written to FileOutputStream + * from OS page cache (typically 4 KB) to disk + */ + private void addShutDownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + try { + if(writeOnlyFileHandle != null) { + writeOnlyFileHandle.flush(); + fileOutputStream.getChannel().force(false); + writeOnlyFileHandle.close(); + } + } catch(Exception e) { + // fail silently for any sort of exception + } + } + }); + } + + /** + * Custom iterator to iterate over values written to disk + * @return + */ + public Iterator iterator() { + return new LazyFileIterable(readOnlyFileHandle, + inMemoryMetadataOfSpilledData, schema, payloadClazz).iterator(); + } + + /** + * Number of bytes spilled to disk + * @return + */ + public long sizeOfFileOnDiskInBytes() { + return filePosition.get(); + } + + @Override + public int size() { + return inMemoryMetadataOfSpilledData.size(); + } + + @Override + public boolean isEmpty() { + return inMemoryMetadataOfSpilledData.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return inMemoryMetadataOfSpilledData.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + throw new HoodieNotSupportedException("unable to compare values in map"); + } + + @Override + public R get(Object key) { + ValueMetadata entry = inMemoryMetadataOfSpilledData.get(key); + if(entry == null) { + return null; + } + try { + return SpillableMapUtils.readFromDisk(readOnlyFileHandle, schema, + payloadClazz, entry.getOffsetOfValue(), entry.getSizeOfValue()); + } catch(IOException e) { + throw new HoodieIOException("Unable to readFromDisk Hoodie Record from disk", e); + } + } + + @Override + public R put(T key, R value) { + //TODO (na) : check value instanceof HoodieRecordPayload, now assume every payload is HoodieRecord + HoodieRecord payload = (HoodieRecord) value; + try { + byte [] val = HoodieAvroUtils.avroToBytes((GenericRecord) payload.getData().getInsertValue(this.schema).get()); + Integer valueSize = val.length; + Long timestamp = new Date().getTime(); + this.inMemoryMetadataOfSpilledData.put(key, new DiskBasedMap.ValueMetadata(this.filePath, valueSize, + filePosition.get(), timestamp)); + // TODO(na) : Test serializer performance for generic types + String serializedKey = SpillableMapUtils.objectMapper.writeValueAsString(key); + filePosition.set(SpillableMapUtils.spillToDisk(writeOnlyFileHandle, + new FileEntry(SpillableMapUtils.generateChecksum(val), + serializedKey.getBytes().length, valueSize, serializedKey.getBytes(), val, timestamp))); + } catch(IOException io) { + throw new HoodieIOException("Unable to store data in Disk Based map", io); + } + return value; + } + + @Override + public R remove(Object key) { + R value = get(key); + inMemoryMetadataOfSpilledData.remove(key); + return value; + } + + @Override + public void putAll(Map m) { + for(Map.Entry entry: m.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + @Override + public void clear() { + inMemoryMetadataOfSpilledData.clear(); + // close input/output streams + try { + writeOnlyFileHandle.flush(); + writeOnlyFileHandle.close(); + new File(filePath).delete(); + } catch(IOException e) { + throw new HoodieIOException("unable to clear map or delete file on disk", e); + } + } + + @Override + public Set keySet() { + return inMemoryMetadataOfSpilledData.keySet(); + } + + @Override + public Collection values() { + throw new HoodieException("Unsupported Operation Exception"); + } + + @Override + public Set> entrySet() { + Set> entrySet = new HashSet<>(); + for(T key: inMemoryMetadataOfSpilledData.keySet()) { + entrySet.add(new AbstractMap.SimpleEntry<>(key, get(key))); + } + return entrySet; + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java new file mode 100644 index 0000000000000..261ca2c31f602 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -0,0 +1,256 @@ +/* + * 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 com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieNotSupportedException; +import org.apache.avro.Schema; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * An external map that spills content to disk when there is insufficient space for it + * to grow. + * + * This map holds 2 types of data structures : + * + * (1) Key-Value pairs in a in-memory map + * (2) Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk + * + * NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed from the in-memory + * key-valueMetadata map but it's values will be lying around in the temp file on disk until the file is cleaned. + * + * The setting of the spill threshold faces the following trade-off: If the spill threshold is + * too high, the in-memory map may occupy more memory than is available, resulting in OOM. + * However, if the spill threshold is too low, we spill frequently and incur unnecessary disk + * writes. + * @param + * @param + */ +public class ExternalSpillableMap implements Map { + + // maximum space allowed in-memory for this map + final private long maxInMemorySizeInBytes; + // current space occupied by this map in-memory + private Long currentInMemoryMapSize; + // Map to store key-values in memory until it hits maxInMemorySizeInBytes + final private Map inMemoryMap; + // Map to store key-valuemetadata important to find the values spilled to disk + final private DiskBasedMap diskBasedMap; + // Schema used to de-serialize and readFromDisk the records written to disk + final private Schema schema; + // An estimate of the size of each payload written to this map + private volatile long estimatedPayloadSize = 0; + // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and incorrect payload estimation + final private Double sizingFactorForInMemoryMap = 0.8; + + private static Logger log = LogManager.getLogger(ExternalSpillableMap.class); + + + public ExternalSpillableMap(Long maxInMemorySizeInBytes, Schema schema, + String payloadClazz, Optional baseFilePath) throws IOException { + this.inMemoryMap = new HashMap<>(); + this.diskBasedMap = new DiskBasedMap<>(schema, payloadClazz, baseFilePath); + this.maxInMemorySizeInBytes = (long) Math.floor(maxInMemorySizeInBytes*sizingFactorForInMemoryMap); + this.schema = schema; + this.currentInMemoryMapSize = 0L; + } + + /** + * A custom iterator to wrap over iterating in-memory + disk spilled data + * @return + */ + public Iterator iterator() { + return new IteratorWrapper<>(inMemoryMap.values().iterator(), diskBasedMap.iterator()); + } + + /** + * Number of entries in DiskBasedMap + * @return + */ + public int getDiskBasedMapNumEntries() { + return diskBasedMap.size(); + } + + /** + * Number of bytes spilled to disk + * @return + */ + public long getSizeOfFileOnDiskInBytes() { + return diskBasedMap.sizeOfFileOnDiskInBytes(); + } + + /** + * Number of entries in InMemoryMap + * @return + */ + public int getInMemoryMapNumEntries() { + return inMemoryMap.size(); + } + + /** + * Approximate memory footprint of the in-memory map + * @return + */ + public long getCurrentInMemoryMapSize() { + return currentInMemoryMapSize; + } + + @Override + public int size() { + return inMemoryMap.size() + diskBasedMap.size(); + } + + @Override + public boolean isEmpty() { + return inMemoryMap.isEmpty() && diskBasedMap.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return inMemoryMap.containsKey(key) || diskBasedMap.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return inMemoryMap.containsValue(value) || diskBasedMap.containsValue(value); + } + + @Override + public R get(Object key) { + if(inMemoryMap.containsKey(key)) { + return inMemoryMap.get(key); + } else if(diskBasedMap.containsKey(key)) { + return diskBasedMap.get(key); + } + return null; + } + + @Override + public R put(T key, R value) { + try { + if (this.currentInMemoryMapSize < maxInMemorySizeInBytes || inMemoryMap.containsKey(key)) { + // Naive approach for now + if (estimatedPayloadSize == 0) { + this.estimatedPayloadSize = SpillableMapUtils.computePayloadSize(value, schema); + log.info("Estimated Payload size => " + estimatedPayloadSize); + } + if(!inMemoryMap.containsKey(key)) { + currentInMemoryMapSize += this.estimatedPayloadSize; + } + inMemoryMap.put(key, value); + } else { + diskBasedMap.put(key, value); + } + return value; + } catch(IOException io) { + throw new HoodieIOException("Unable to estimate size of payload", io); + } + } + + @Override + public R remove(Object key) { + // NOTE : diskBasedMap.remove does not delete the data from disk + if(inMemoryMap.containsKey(key)) { + currentInMemoryMapSize -= estimatedPayloadSize; + return inMemoryMap.remove(key); + } else if(diskBasedMap.containsKey(key)) { + return diskBasedMap.remove(key); + } + return null; + } + + @Override + public void putAll(Map m) { + for(Map.Entry entry: m.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + @Override + public void clear() { + inMemoryMap.clear(); + diskBasedMap.clear(); + currentInMemoryMapSize = 0L; + } + + @Override + public Set keySet() { + Set keySet = new HashSet(); + keySet.addAll(inMemoryMap.keySet()); + keySet.addAll(diskBasedMap.keySet()); + return keySet; + } + + @Override + public Collection values() { + if(diskBasedMap.isEmpty()) { + return inMemoryMap.values(); + } + throw new HoodieNotSupportedException("Cannot return all values in memory"); + } + + @Override + public Set> entrySet() { + Set> entrySet = new HashSet<>(); + entrySet.addAll(inMemoryMap.entrySet()); + entrySet.addAll(diskBasedMap.entrySet()); + return entrySet; + } + + /** + * Iterator that wraps iterating over all the values for this map + * 1) inMemoryIterator - Iterates over all the data in-memory map + * 2) diskLazyFileIterator - Iterates over all the data spilled to disk + * @param + */ + private class IteratorWrapper implements Iterator { + + private Iterator inMemoryIterator; + private Iterator diskLazyFileIterator; + + public IteratorWrapper(Iterator inMemoryIterator, Iterator diskLazyFileIterator) { + this.inMemoryIterator = inMemoryIterator; + this.diskLazyFileIterator = diskLazyFileIterator; + } + @Override + public boolean hasNext() { + if(inMemoryIterator.hasNext()) { + return true; + } + return diskLazyFileIterator.hasNext(); + } + + @Override + public R next() { + if(inMemoryIterator.hasNext()) { + return inMemoryIterator.next(); + } + return diskLazyFileIterator.next(); + } + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java new file mode 100644 index 0000000000000..f91894ecdc193 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java @@ -0,0 +1,116 @@ +/* + * 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 com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import org.apache.avro.Schema; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * Iterable to lazily fetch values spilled to disk. + * This class uses RandomAccessFile to randomly access the position of + * the latest value for a key spilled to disk and returns the result. + * @param + */ +public class LazyFileIterable implements Iterable { + + // Used to access the value written at a specific position in the file + private RandomAccessFile readOnlyFileHandle; + // Stores the key and corresponding value's latest metadata spilled to disk + private Map inMemoryMetadataOfSpilledData; + // Schema used to de-serialize payload written to disk + private Schema schema; + // Class used to de-serialize/realize payload written to disk + private String payloadClazz; + + public LazyFileIterable(RandomAccessFile file, Map map, + Schema schema, String payloadClazz) { + this.readOnlyFileHandle = file; + this.inMemoryMetadataOfSpilledData = map; + this.schema = schema; + this.payloadClazz = payloadClazz; + } + @Override + public Iterator iterator() { + try { + return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData, schema, payloadClazz); + } catch(IOException io) { + throw new HoodieException("Unable to initialize iterator for file on disk", io); + } + } + + /** + * Iterator implementation for the iterable defined above. + * @param + */ + public class LazyFileIterator implements Iterator { + + private RandomAccessFile readOnlyFileHandle; + private Schema schema; + private String payloadClazz; + private Iterator> metadataIterator; + + public LazyFileIterator(RandomAccessFile file, Map map, + Schema schema, String payloadClazz) throws IOException { + this.readOnlyFileHandle = file; + this.schema = schema; + this.payloadClazz = payloadClazz; + // sort the map in increasing order of offset of value so disk seek is only in one(forward) direction + this.metadataIterator = map + .entrySet() + .stream() + .sorted((Map.Entry o1, Map.Entry o2) -> + o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue())) + .collect(Collectors.toList()).iterator(); + } + + @Override + public boolean hasNext() { + return this.metadataIterator.hasNext(); + } + + @Override + public T next() { + Map.Entry entry = this.metadataIterator.next(); + try { + return SpillableMapUtils.readFromDisk(readOnlyFileHandle, schema, + payloadClazz, entry.getValue().getOffsetOfValue(), entry.getValue().getSizeOfValue()); + } catch(IOException e) { + throw new HoodieIOException("Unable to read hoodie record from value spilled to disk", e); + } + } + + @Override + public void remove() { + this.metadataIterator.remove(); + } + + @Override + public void forEachRemaining(Consumer action) { + action.accept(next()); + } + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java new file mode 100644 index 0000000000000..8280dda3f594a --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java @@ -0,0 +1,69 @@ +/* + * 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.io.storage; + +import java.io.DataOutputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Wrapper for DataOutpuStream to keep track of number of bytes written + */ +public class SizeAwareDataOutputStream { + // Actual outpuStream + private DataOutputStream outputStream; + // Counter to keep track of number of bytes written + private AtomicLong size; + + public SizeAwareDataOutputStream(FileOutputStream fileOutputStream) { + this.outputStream = new DataOutputStream(fileOutputStream); + this.size = new AtomicLong(0L); + } + + public void writeLong(long v) throws IOException { + size.addAndGet(Long.BYTES); + outputStream.writeLong(v); + } + + public void writeInt(int v) throws IOException { + size.addAndGet(Integer.BYTES); + outputStream.writeInt(v); + } + + public void write(byte [] v) throws IOException { + size.addAndGet(v.length); + outputStream.write(v); + } + + public void write(byte [] v, int offset, int len) throws IOException { + size.addAndGet(len + offset); + outputStream.write(v, offset, len); + } + + public void flush() throws IOException { + outputStream.flush(); + } + + public void close() throws IOException { + outputStream.close(); + } + + public long getSize() { + return size.get(); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java new file mode 100644 index 0000000000000..3c10a15377d7d --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java @@ -0,0 +1,32 @@ +/* + * 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.exception; + +/** + *

Exception thrown when any data corruption happens when reading/writing from temporary disk

+ */ +public class HoodieCorruptedDataException extends HoodieException { + + public HoodieCorruptedDataException(String msg) { + super(msg); + } + + public HoodieCorruptedDataException(String msg, Throwable e) { + super(msg, e); + } + +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index a8be14ee4fb5f..e57c7ebe7335c 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -16,12 +16,6 @@ package com.uber.hoodie.common.table.log; -import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import com.google.common.collect.Maps; import com.uber.hoodie.common.minicluster.MiniClusterUtil; import com.uber.hoodie.common.model.HoodieArchivedLogFile; @@ -41,15 +35,7 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; -import java.io.IOException; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; +import com.uber.hoodie.common.util.collection.DiskBasedMap; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -64,6 +50,23 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + @SuppressWarnings("Duplicates") public class HoodieLogFormatTest { @@ -490,7 +493,7 @@ public void testAvroLogRecordReaderBasic() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "100"); + schema, "100", 10240L); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -554,9 +557,8 @@ public void testAvroLogRecordReaderWithRollbackTombstone() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "102"); - assertEquals("We read 200 records from 2 write batches", 200, - scanner.getTotalLogRecords()); + schema, "102", 10240L); + assertEquals("We only read 200 records, but only 200 of them are valid", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); @@ -635,9 +637,8 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "103"); - assertEquals("We would read 200 records", 200, - scanner.getTotalLogRecords()); + schema, "103", 10240L); + assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); @@ -700,9 +701,8 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "102"); - assertEquals("We still would read 200 records", 200, - scanner.getTotalLogRecords()); + schema, "102", 10240L); + assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 150, readKeys.size()); @@ -720,7 +720,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() writer = writer.appendBlock(commandBlock); readKeys.clear(); - scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101"); + scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); @@ -783,12 +783,10 @@ public void testAvroLogRecordReaderWithFailedRollbacks() .map(s -> s.getPath().toString()) .collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "100"); // all data must be rolled back before merge - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, + allLogFiles, schema, "100", 10240L); + assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -841,7 +839,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100"); + allLogFiles, schema, "100", 10240L); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -875,7 +873,7 @@ public void testAvroLogRecordReaderWithInvalidRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100"); + allLogFiles, schema, "100", 10240L); assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); @@ -931,7 +929,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101"); + allLogFiles, schema, "101", 10240L); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -1014,8 +1012,9 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101"); + allLogFiles, schema, "101", 10240L); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } + } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java index 9af16929495c8..1de52e8ed593b 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java @@ -18,6 +18,7 @@ import com.uber.hoodie.avro.MercifulJsonConverter; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.net.URI; @@ -35,6 +36,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -53,7 +55,7 @@ public static List generateTestRecords(int from, int limit) } private static List toRecords(Schema writerSchema, Schema readerSchema, int from, - int limit) throws IOException, URISyntaxException { + int limit) throws IOException, URISyntaxException { GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema); // Required to register the necessary JAR:// file system @@ -93,12 +95,29 @@ static Path uriToPath(URI uri) throws IOException { public static List generateHoodieTestRecords(int from, int limit) throws IOException, URISyntaxException { List records = generateTestRecords(from, limit); + String commitTime = HoodieActiveTimeline.createNewCommitTime(); Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); return records.stream() .map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)) .map(p -> { p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString()); p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); + p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); + return p; + }).collect( + Collectors.toList()); + + } + + public static List updateHoodieTestRecords(List oldRecordKeys, List newRecords, + String commitTime) + throws IOException, URISyntaxException { + + return newRecords.stream() + .map(p -> { + ((GenericRecord)p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0)); + ((GenericRecord)p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); + ((GenericRecord)p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); return p; }).collect( Collectors.toList()); @@ -121,7 +140,7 @@ public static Schema getComplexEvolvedSchema() throws IOException { } public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, - String commitTime, String fileId) throws IOException { + String commitTime, String fileId) throws IOException { TestRecord record = new TestRecord(commitTime, recordNumber, fileId); MercifulJsonConverter converter = new MercifulJsonConverter(schema); return converter.convert(record.toJsonString()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java new file mode 100644 index 0000000000000..05be501448d8d --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java @@ -0,0 +1,47 @@ +/* + * 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; + +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 org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class SpillableMapTestUtils { + + public static List upsertRecords(List iRecords, + Map> records) { + List recordKeys = new ArrayList<>(); + iRecords + .stream() + .forEach(r -> { + String key = ((GenericRecord)r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String partitionPath = ((GenericRecord)r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + recordKeys.add(key); + records.put(key, new HoodieRecord<>(new HoodieKey(key, partitionPath), + new HoodieAvroPayload(Optional.of((GenericRecord) r)))); + }); + return recordKeys; + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java new file mode 100644 index 0000000000000..eb570a6558ced --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -0,0 +1,107 @@ +/* + * 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 com.uber.hoodie.common.model.HoodieAvroPayload; +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.SchemaTestUtil; +import com.uber.hoodie.common.util.SpillableMapTestUtils; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.junit.Test; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; + +public class TestDiskBasedMap { + + @Test + public void testSimpleInsert() throws IOException, URISyntaxException { + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty()); + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + + // make sure records have spilled to disk + assertTrue(records.sizeOfFileOnDiskInBytes() > 0); + Iterator> itr = records.iterator(); + List oRecords = new ArrayList<>(); + while(itr.hasNext()) { + HoodieRecord rec = itr.next(); + oRecords.add(rec); + assert recordKeys.contains(rec.getRecordKey()); + } + } + + @Test + public void testSimpleUpsert() throws IOException, URISyntaxException { + + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty()); + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + // perform some inserts + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + + long fileSize = records.sizeOfFileOnDiskInBytes(); + // make sure records have spilled to disk + assertTrue(fileSize > 0); + + // generate updates from inserts + List updatedRecords = + SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100), + HoodieActiveTimeline.createNewCommitTime()); + String newCommitTime = ((GenericRecord)updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + + // new commit time should be different + assertEquals(commitTime, newCommitTime); + + // 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> itr = records.iterator(); + while(itr.hasNext()) { + HoodieRecord 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); + } + } + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java new file mode 100644 index 0000000000000..d15caf6457f01 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -0,0 +1,245 @@ +/* + * 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 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.SchemaTestUtil; +import com.uber.hoodie.common.util.SpillableMapTestUtils; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class TestExternalSpillableMap { + + private static final String FAILURE_OUTPUT_PATH = "/tmp/test_fail"; + + @Test + public void simpleInsertTest() throws IOException, URISyntaxException { + ExternalSpillableMap> records = + new ExternalSpillableMap<> + (16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()), + HoodieAvroPayload.class.getName(), Optional.empty()); //16B + + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + assert (recordKeys.size() == 100); + Iterator> itr = records.iterator(); + List oRecords = new ArrayList<>(); + while(itr.hasNext()) { + HoodieRecord rec = itr.next(); + oRecords.add(rec); + assert recordKeys.contains(rec.getRecordKey()); + } + } + + @Test + public void testSimpleUpsert() throws IOException, URISyntaxException { + + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + ExternalSpillableMap> records = + new ExternalSpillableMap<> + (16L, schema, + HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B + + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + assert (recordKeys.size() == 100); + Iterator> itr = records.iterator(); + while(itr.hasNext()) { + HoodieRecord rec = itr.next(); + assert recordKeys.contains(rec.getRecordKey()); + } + List updatedRecords = + SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100), + HoodieActiveTimeline.createNewCommitTime()); + + // update records already inserted + SpillableMapTestUtils.upsertRecords(updatedRecords, records); + + // make sure we have records spilled to disk + assertTrue(records.getDiskBasedMapNumEntries() > 0); + + // iterate over the updated records and compare the value from Map + updatedRecords.stream().forEach(record -> { + HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); + try { + assertEquals(rec.getData().getInsertValue(schema).get(),record); + } catch(IOException io) { + throw new UncheckedIOException(io); + } + }); + } + + @Test + public void testAllMapOperations() throws IOException, URISyntaxException { + + ExternalSpillableMap> records = + new ExternalSpillableMap<> + (16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()), + HoodieAvroPayload.class.getName(), Optional.empty()); //16B + + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + // insert a bunch of records so that values spill to disk too + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + IndexedRecord inMemoryRecord = iRecords.get(0); + String ikey = ((GenericRecord)inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String iPartitionPath = ((GenericRecord)inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath), + new HoodieAvroPayload(Optional.of((GenericRecord)inMemoryRecord))); + + IndexedRecord onDiskRecord = iRecords.get(99); + String dkey = ((GenericRecord)onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String dPartitionPath = ((GenericRecord)onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath), + new HoodieAvroPayload(Optional.of((GenericRecord)onDiskRecord))); + // assert size + assert records.size() == 100; + // get should return the same HoodieKey and same value + assert inMemoryHoodieRecord.getKey().equals(records.get(ikey).getKey()); + assert onDiskHoodieRecord.getKey().equals(records.get(dkey).getKey()); + //assert inMemoryHoodieRecord.equals(records.get(ikey)); + //assert onDiskHoodieRecord.equals(records.get(dkey)); + + // test contains + assertTrue(records.containsKey(ikey)); + assertTrue(records.containsKey(dkey)); + + // test isEmpty + assertFalse(records.isEmpty()); + + // test containsAll + assertTrue(records.keySet().containsAll(recordKeys)); + + // remove (from inMemory and onDisk) + HoodieRecord removedRecord = records.remove(ikey); + assertTrue(removedRecord != null); + assertFalse(records.containsKey(ikey)); + + removedRecord = records.remove(dkey); + assertTrue(removedRecord != null); + assertFalse(records.containsKey(dkey)); + + // test clear + records.clear(); + assertTrue(records.size() == 0); + } + + @Test(expected = IOException.class) + public void simpleTestWithException() throws IOException, URISyntaxException { + ExternalSpillableMap> records = + new ExternalSpillableMap<> + (16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()), + HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B + + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + assert (recordKeys.size() == 100); + Iterator> itr = records.iterator(); + while(itr.hasNext()) { + throw new IOException("Testing failures..."); + } + } + + @Test + public void simpleTestWithExceptionValidateFileIsRemoved() throws Exception { + File file = new File(FAILURE_OUTPUT_PATH); + assertFalse(file.exists()); + } + + @Test + public void testDataCorrectnessInMapAndDisk() throws IOException, URISyntaxException { + + Schema schema = SchemaTestUtil.getSimpleSchema(); + ExternalSpillableMap> records = + new ExternalSpillableMap<> + (16L, HoodieAvroUtils.addMetadataFields(schema), + HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B + + List recordKeys = new ArrayList<>(); + // Ensure we spill to disk + while(records.getDiskBasedMapNumEntries() < 1) { + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + recordKeys.addAll(SpillableMapTestUtils.upsertRecords(iRecords, records)); + } + + // Get a record from the in-Memory map + String key = recordKeys.get(0); + HoodieRecord record = records.get(key); + List recordsToUpdate = new ArrayList<>(); + schema = HoodieAvroUtils.addMetadataFields(schema); + recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get()); + + String newCommitTime = HoodieActiveTimeline.createNewCommitTime(); + List keysToBeUpdated = new ArrayList<>(); + keysToBeUpdated.add(key); + // Update the commitTime for this record + List updatedRecords = + SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); + // Upsert this updated record + SpillableMapTestUtils.upsertRecords(updatedRecords, records); + GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); + // The record returned for this key should have the updated commitTime + assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + + + // Get a record from the disk based map + key = recordKeys.get(recordKeys.size() - 1); + record = records.get(key); + recordsToUpdate = new ArrayList<>(); + recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get()); + + newCommitTime = HoodieActiveTimeline.createNewCommitTime(); + keysToBeUpdated = new ArrayList<>(); + keysToBeUpdated.add(key); + // Update the commitTime for this record + updatedRecords = + SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); + // Upsert this updated record + SpillableMapTestUtils.upsertRecords(updatedRecords, records); + gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); + // The record returned for this key should have the updated commitTime + assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + + } + + // TODO : come up with a performance eval test for spillableMap + @Test + public void testLargeInsertUpsert() { + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 35ef39eda8ca4..9f231084a53ef 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -68,6 +68,10 @@ public class HoodieRealtimeRecordReader implements RecordReader deltaRecordMap; @@ -126,7 +130,9 @@ private void readAndCompactLog(JobConf jobConf) throws IOException { new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), split.getDeltaFilePaths(), - readerSchema, split.getMaxCommitTime()); + readerSchema, split.getMaxCommitTime(), + (long) Math.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) + *jobConf.getMemoryForMapTask())); // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit // but can return records for completed commits > the commit we are trying to read (if using readCommit() API) for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { From ebcfece41af16ba37edf0363712b2d860dcaf046 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 22 Feb 2018 11:20:54 -0800 Subject: [PATCH 024/374] Add new APIs in HoodieReadClient and HoodieWriteClient --- .../com/uber/hoodie/HoodieReadClient.java | 56 ++++-- .../com/uber/hoodie/HoodieWriteClient.java | 161 +++++++++++++----- .../TestHoodieClientOnCopyOnWriteStorage.java | 21 +-- 3 files changed, 173 insertions(+), 65 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index eb9ad62e470ec..82bb9218f89b3 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -19,11 +19,14 @@ import com.google.common.base.Optional; 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.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.index.bloom.HoodieBloomIndex; +import com.uber.hoodie.exception.HoodieIndexException; +import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; import java.io.Serializable; import java.util.HashSet; @@ -45,7 +48,7 @@ /** * Provides an RDD based API for accessing/filtering Hoodie tables, based on keys. */ -public class HoodieReadClient implements Serializable { +public class HoodieReadClient implements Serializable { private static Logger logger = LogManager.getLogger(HoodieReadClient.class); @@ -57,7 +60,7 @@ public class HoodieReadClient implements Serializable { * just with a simple basepath pointing to the dataset. Until, then just always assume a * BloomIndex */ - private transient final HoodieBloomIndex index; + private transient final HoodieIndex index; private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; private transient Optional sqlContextOpt; @@ -66,15 +69,12 @@ public class HoodieReadClient implements Serializable { * @param basePath path to Hoodie dataset */ public HoodieReadClient(JavaSparkContext jsc, String basePath) { - this.jsc = jsc; - this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); - // Create a Hoodie table which encapsulated the commits and files visible - this.hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), null); - this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); - this.index = - new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc); - this.sqlContextOpt = Optional.absent(); + this(jsc, HoodieWriteConfig.newBuilder() + .withPath(basePath) + // by default we use HoodieBloomIndex + .withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .build()); } /** @@ -88,6 +88,22 @@ public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlCon this.sqlContextOpt = Optional.of(sqlContext); } + /** + * @param clientConfig instance of HoodieWriteConfig + */ + public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { + final String basePath = clientConfig.getBasePath(); + this.jsc = jsc; + this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + // Create a Hoodie table which encapsulated the commits and files visible + this.hoodieTable = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), + clientConfig); + this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); + this.index = HoodieIndex.createIndex(clientConfig, jsc); + this.sqlContextOpt = Optional.absent(); + } + /** * Adds support for accessing Hoodie built tables from SparkSQL, as you normally would. * @@ -158,8 +174,20 @@ public JavaPairRDD> checkExists(JavaRDD h * @param hoodieRecords Input RDD of Hoodie records. * @return A subset of hoodieRecords RDD, with existing records filtered out. */ - public JavaRDD filterExists(JavaRDD hoodieRecords) { - JavaRDD recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable); + public JavaRDD> filterExists(JavaRDD> hoodieRecords) { + JavaRDD> recordsWithLocation = tagLocation(hoodieRecords); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); } + + /** + * Looks up the index and tags each incoming record with a location of a file that contains the + * row (if it is actually present). Input RDD should contain no duplicates if needed. + * + * @param hoodieRecords Input RDD of Hoodie records + * @return Tagged RDD of Hoodie records + */ + public JavaRDD> tagLocation(JavaRDD> hoodieRecords) + throws HoodieIndexException { + return index.tagLocation(hoodieRecords, hoodieTable); + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 100dcfaa030d6..81224fab767f5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -142,12 +142,7 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco * Upserts a bunch of new records into the Hoodie table, at the supplied commitTime */ public JavaRDD upsert(JavaRDD> records, final String commitTime) { - writeContext = metrics.getCommitCtx(); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - + HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed JavaRDD> dedupedRecords = @@ -165,6 +160,30 @@ public JavaRDD upsert(JavaRDD> records, final Strin } } + /** + * Upserts the given prepared records into the Hoodie table, at the supplied commitTime. + * + * This implementation requires that the input records are already tagged, and de-duped if + * needed. + * + * @param preppedRecords Prepared HoodieRecords to upsert + * @param commitTime Commit Time handle + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, + final String commitTime) { + HoodieTable table = getTableAndInitCtx(); + try { + return upsertRecordsInternal(preppedRecords, commitTime, table, true); + } catch (Throwable e) { + if (e instanceof HoodieUpsertException) { + throw (HoodieUpsertException) e; + } + throw new HoodieUpsertException("Failed to upsert prepared records for commit time " + + commitTime, e); + } + } + /** * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal * writes. @@ -177,11 +196,7 @@ public JavaRDD upsert(JavaRDD> records, final Strin * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD insert(JavaRDD> records, final String commitTime) { - writeContext = metrics.getCommitCtx(); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed JavaRDD> dedupedRecords = @@ -197,6 +212,31 @@ public JavaRDD insert(JavaRDD> records, final Strin } } + /** + * Inserts the given prepared records into the Hoodie table, at the supplied commitTime. + * + * This implementation skips the index check, skips de-duping and is able to leverage benefits + * such as small file handling/blocking alignment, as with insert(), by profiling the workload. + * The prepared HoodieRecords should be de-duped if needed. + * + * @param preppedRecords HoodieRecords to insert + * @param commitTime Commit Time handle + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, + final String commitTime) { + HoodieTable table = getTableAndInitCtx(); + try { + return upsertRecordsInternal(preppedRecords, commitTime, table, false); + } catch (Throwable e) { + if (e instanceof HoodieInsertException) { + throw e; + } + throw new HoodieInsertException("Failed to insert prepared records for commit time " + + commitTime, e); + } + } + /** * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to @@ -235,40 +275,14 @@ public JavaRDD bulkInsert(JavaRDD> records, public JavaRDD bulkInsert(JavaRDD> records, final String commitTime, Option bulkInsertPartitioner) { - writeContext = metrics.getCommitCtx(); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - + HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed JavaRDD> dedupedRecords = combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism()); - final JavaRDD> repartitionedRecords; - if (bulkInsertPartitioner.isDefined()) { - repartitionedRecords = - bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, - config.getBulkInsertShuffleParallelism()); - } else { - // Now, sort the records and line them up nicely for loading. - repartitionedRecords = dedupedRecords - .sortBy(record -> { - // Let's use "partitionPath + key" as the sort key. Spark, will ensure - // the records split evenly across RDD partitions, such that small partitions fit - // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions - return String - .format("%s+%s", record.getPartitionPath(), record.getRecordKey()); - }, true, config.getBulkInsertShuffleParallelism()); - } - JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), - true) - .flatMap(writeStatuses -> writeStatuses.iterator()); - - return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); + return bulkInsertInternal(dedupedRecords, commitTime, table, bulkInsertPartitioner); } catch (Throwable e) { if (e instanceof HoodieInsertException) { throw e; @@ -278,6 +292,67 @@ public JavaRDD bulkInsert(JavaRDD> records, } } + /** + * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk + * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to + * Hoodie). The input records should contain no duplicates if needed. + * + * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and + * attempts to control the numbers of files with less memory compared to the {@link + * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own + * partitioner. If specified then it will be used for repartitioning records. See {@link + * UserDefinedBulkInsertPartitioner}. + * + * @param preppedRecords HoodieRecords to insert + * @param commitTime Commit Time handle + * @param bulkInsertPartitioner If specified then it will be used to partition input records + * before they are inserted into hoodie. + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, + final String commitTime, + Option bulkInsertPartitioner) { + HoodieTable table = getTableAndInitCtx(); + try { + return bulkInsertInternal(preppedRecords, commitTime, table, bulkInsertPartitioner); + } catch (Throwable e) { + if (e instanceof HoodieInsertException) { + throw e; + } + throw new HoodieInsertException("Failed to bulk insert prepared records for commit time " + + commitTime, e); + } + } + + private JavaRDD bulkInsertInternal( + JavaRDD> dedupedRecords, + String commitTime, + HoodieTable table, + Option bulkInsertPartitioner) { + final JavaRDD> repartitionedRecords; + if (bulkInsertPartitioner.isDefined()) { + repartitionedRecords = + bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, + config.getBulkInsertShuffleParallelism()); + } else { + // Now, sort the records and line them up nicely for loading. + repartitionedRecords = dedupedRecords + .sortBy(record -> { + // Let's use "partitionPath + key" as the sort key. Spark, will ensure + // the records split evenly across RDD partitions, such that small partitions fit + // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions + return String + .format("%s+%s", record.getPartitionPath(), record.getRecordKey()); + }, true, config.getBulkInsertShuffleParallelism()); + } + JavaRDD writeStatusRDD = repartitionedRecords + .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), + true) + .flatMap(writeStatuses -> writeStatuses.iterator()); + + return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); + } + private void commitOnAutoCommit(String commitTime, JavaRDD resultRDD) { if (config.shouldAutoCommit()) { logger.info("Auto commit enabled: Committing " + commitTime); @@ -907,4 +982,12 @@ private void rollbackInflightCommits() { rollback(commit); } } + + private HoodieTable getTableAndInitCtx() { + writeContext = metrics.getCommitCtx(); + // Create a Hoodie table which encapsulated the commits and files visible + return HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 5caeec23b4347..1bf9ca506fb2d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -80,6 +80,7 @@ import org.junit.Before; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import scala.Option; import scala.collection.Iterator; public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @@ -190,7 +191,8 @@ public void testAutoCommit() throws Exception { List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - JavaRDD result = client.bulkInsert(writeRecords, newCommitTime); + JavaRDD result = client.bulkInsertPreppedRecords(writeRecords, newCommitTime, + Option.empty()); assertFalse("If Autocommit is false, then commit should not be made automatically", HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); @@ -218,7 +220,6 @@ public void testUpserts() throws Exception { private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exception { HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig); - HoodieIndex index = HoodieIndex.createIndex(hoodieWriteConfig, jsc); /** * Write 1 (only inserts) @@ -229,7 +230,7 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - List statuses = client.upsert(writeRecords, newCommitTime).collect(); + List statuses = client.upsertPreppedRecords(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); // check the partition metadata is written out @@ -249,10 +250,8 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc records.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, getConfig()); - - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) + HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); + List taggedRecords = readClient.tagLocation(jsc.parallelize(records, 1)) .collect(); checkTaggedRecords(taggedRecords, "001"); @@ -282,11 +281,9 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); - // Index should be able to locate all updates in correct locations. - taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), table).collect(); + readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); + taggedRecords = readClient.tagLocation(jsc.parallelize(dedupedRecords, 1)).collect(); checkTaggedRecords(taggedRecords, "004"); // Check the entire dataset has 100 records still @@ -732,7 +729,7 @@ public void testInsertAndCleanByCommits() throws Exception { List records = dataGen.generateInserts(newCommitTime, 500); JavaRDD writeRecords = jsc.parallelize(records, 5); - List statuses = client.insert(writeRecords, newCommitTime).collect(); + List statuses = client.insertPreppedRecords(writeRecords, newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); From b8527a95f192f3d5cfbd15f95598879e06a9963b Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 1 Mar 2018 14:04:16 -0800 Subject: [PATCH 025/374] Handle inflight clean instants during Hoodie instants archiving --- .../uber/hoodie/io/HoodieCommitArchiveLog.java | 6 ++++-- .../hoodie/io/TestHoodieCommitArchiveLog.java | 16 +++++++++++++++- .../hoodie/common/model/HoodieTestUtils.java | 8 ++++++++ 3 files changed, 27 insertions(+), 3 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index bb295b64d2b51..6e1f14ab74a6e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -126,9 +126,11 @@ private Stream getInstantsToArchive() { HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); // GroupBy each action and limit each action timeline to maxCommitsToKeep + // TODO: Handle ROLLBACK_ACTION in future + // ROLLBACK_ACTION is currently not defined in HoodieActiveTimeline HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() - .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION, - HoodieTimeline.ROLLBACK_ACTION)); + .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)) + .filterCompletedInstants(); Stream instants = cleanAndRollbackTimeline.getInstants() .collect(Collectors.groupingBy(s -> s.getAction())) .entrySet() diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index d0333d077470f..3add748c995a7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Sets; import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieLogFile; @@ -90,11 +91,13 @@ public void testArchiveDatasetWithArchival() throws IOException { assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); HoodieTestUtils.createCleanFiles(basePath, "100"); + HoodieTestUtils.createInflightCleanFiles(basePath, "101"); HoodieTestUtils.createCleanFiles(basePath, "101"); HoodieTestUtils.createCleanFiles(basePath, "102"); HoodieTestUtils.createCleanFiles(basePath, "103"); HoodieTestUtils.createCleanFiles(basePath, "104"); HoodieTestUtils.createCleanFiles(basePath, "105"); + HoodieTestUtils.createInflightCleanFiles(basePath, "106", "107"); //reload the timeline and get all the commmits before archive timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() @@ -103,6 +106,9 @@ public void testArchiveDatasetWithArchival() throws IOException { assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants()); + // verify in-flight instants before archive + verifyInflightInstants(metaClient, 3); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, new HoodieTableMetaClient(fs.getConf(), basePath, true)); @@ -141,6 +147,9 @@ public void testArchiveDatasetWithArchival() throws IOException { "Read commits map should match the originalCommits - commitsLoadedFromArchival", originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), readCommits); + + // verify in-flight instants after archive + verifyInflightInstants(metaClient, 3); } @Test @@ -233,5 +242,10 @@ public void testArchiveCommitSavepointNoHole() throws IOException { timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103"))); } - + private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { + HoodieTimeline timeline = metaClient.getActiveTimeline().reload() + .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights(); + assertEquals("Loaded inflight clean actions and the count should match", + expectedTotalInstants, timeline.countInstants()); + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 74103f8fa6ac2..db0e87d6fc666 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -124,6 +124,14 @@ public static final void createInflightCommitFiles(String basePath, String... co } } + public static final void createInflightCleanFiles(String basePath, String... commitTimes) + throws IOException { + for (String commitTime : commitTimes) { + new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + HoodieTimeline.makeInflightCleanerFileName(commitTime)).createNewFile(); + } + } + public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) throws IOException { String fileID = UUID.randomUUID().toString(); From 0429caed7bde6185054657406983334dbef94d23 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Thu, 15 Feb 2018 11:01:25 -0800 Subject: [PATCH 026/374] Introducing HoodieLogFormat V2 with versioning support - HoodieLogFormat V2 has support for LogFormat evolution through versioning - LogVersion is associated with a LogBlock not a LogFile - Based on a version for a LogBlock, approporiate code path is executed - Implemented LazyReading of Hoodie Log Blocks with Memory / IO tradeoff - Implemented Reverse pointer to be able to traverse the log in reverse - Introduce new MAGIC for backwards compatibility with logs without versions --- docs/_data/topnav.yml | 2 +- docs/code_and_design.md | 38 + docs/code_structure.md | 16 - docs/configurations.md | 8 + docs/images/hoodie_log_format_v2.png | Bin 0 -> 223676 bytes .../cli/commands/ArchivedCommitsCommand.java | 12 +- .../hoodie/config/HoodieCompactionConfig.java | 25 + .../uber/hoodie/config/HoodieWriteConfig.java | 12 +- .../uber/hoodie/io/HoodieAppendHandle.java | 17 +- .../hoodie/io/HoodieCommitArchiveLog.java | 8 +- .../compact/HoodieRealtimeTableCompactor.java | 3 +- .../hoodie/table/HoodieMergeOnReadTable.java | 11 +- .../hoodie/io/TestHoodieCommitArchiveLog.java | 2 +- .../hoodie/common/model/HoodieLogFile.java | 13 + .../log/HoodieCompactedLogRecordScanner.java | 257 ++++--- .../common/table/log/HoodieLogFileReader.java | 410 +++++++++++ .../common/table/log/HoodieLogFormat.java | 30 +- .../table/log/HoodieLogFormatReader.java | 197 ++--- .../table/log/HoodieLogFormatWriter.java | 66 +- .../common/table/log/LogFormatVersion.java | 131 ++++ .../table/log/block/HoodieAvroDataBlock.java | 224 +++++- .../table/log/block/HoodieCommandBlock.java | 63 +- .../table/log/block/HoodieCorruptBlock.java | 65 +- .../table/log/block/HoodieDeleteBlock.java | 84 ++- .../table/log/block/HoodieLogBlock.java | 189 ++++- .../table/log/block/LogBlockVersion.java | 79 ++ .../hoodie/common/model/HoodieTestUtils.java | 7 +- .../common/table/log/HoodieLogFormatTest.java | 689 +++++++++++++----- .../realtime/HoodieRealtimeRecordReader.java | 10 +- .../HoodieRealtimeRecordReaderTest.java | 7 +- .../uber/hoodie/hive/HoodieHiveClient.java | 3 +- .../java/com/uber/hoodie/hive/TestUtil.java | 7 +- 32 files changed, 2037 insertions(+), 648 deletions(-) create mode 100644 docs/code_and_design.md delete mode 100644 docs/code_structure.md create mode 100644 docs/images/hoodie_log_format_v2.png create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java diff --git a/docs/_data/topnav.yml b/docs/_data/topnav.yml index b144d9a831a61..190573a3e335b 100644 --- a/docs/_data/topnav.yml +++ b/docs/_data/topnav.yml @@ -23,7 +23,7 @@ topnav_dropdowns: url: /api_docs.html output: web - title: Code Structure - url: /code_structure.html + url: /code_and_design.html output: web - title: Roadmap url: /roadmap.html diff --git a/docs/code_and_design.md b/docs/code_and_design.md new file mode 100644 index 0000000000000..3baaa97b449f3 --- /dev/null +++ b/docs/code_and_design.md @@ -0,0 +1,38 @@ +--- +title: Code Structure +keywords: usecases +sidebar: mydoc_sidebar +permalink: code_and_design.html +--- + +## Code & Project Structure + + * hoodie-client : Spark client library to take a bunch of inserts + updates and apply them to a Hoodie table + * hoodie-common : Common code shared between different artifacts of Hoodie + + ## HoodieLogFormat + + The following diagram depicts the LogFormat for Hoodie MergeOnRead. Each logfile consists of one or more log blocks. + Each logblock follows the format shown below. + + | Field | Description | + |-------------- |------------------| + | MAGIC | A magic header that marks the start of a block | + | VERSION | The version of the LogFormat, this helps define how to switch between different log format as it evolves | + | TYPE | The type of the log block | + | HEADER LENGTH | The length of the headers, 0 if no headers | + | HEADER | Metadata needed for a log block. For eg. INSTANT_TIME, TARGET_INSTANT_TIME, SCHEMA etc. | + | CONTENT LENGTH | The length of the content of the log block | + | CONTENT | The content of the log block, for example, for a DATA_BLOCK, the content is (number of records + actual records) in byte [] | + | FOOTER LENGTH | The length of the footers, 0 if no footers | + | FOOTER | Metadata needed for a log block. For eg. index entries, a bloom filter for records in a DATA_BLOCK etc. | + | LOGBLOCK LENGTH | The total number of bytes written for a log block, typically the SUM(everything_above). This is a LONG. This acts as a reverse pointer to be able to traverse the log in reverse.| + + + {% include image.html file="hoodie_log_format_v2.png" alt="hoodie_log_format_v2.png" %} + + + + + + diff --git a/docs/code_structure.md b/docs/code_structure.md deleted file mode 100644 index 2550c905e93e6..0000000000000 --- a/docs/code_structure.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -title: Code Structure -keywords: usecases -sidebar: mydoc_sidebar -permalink: code_structure.html ---- - -## Code & Project Structure - - * hoodie-client : Spark client library to take a bunch of inserts + updates and apply them to a Hoodie table - * hoodie-common : Common code shared between different artifacts of Hoodie - - - - - diff --git a/docs/configurations.md b/docs/configurations.md index 328c5a6e72191..2ff6077075ba8 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -74,6 +74,14 @@ summary: "Here we list all possible configurations and what they mean" Should hoodie dynamically compute the insertSplitSize based on the last 24 commit's metadata. Turned off by default. - [approxRecordSize](#approxRecordSize) ()
The average record size. If specified, hoodie will use this and not compute dynamically based on the last 24 commit's metadata. No value set as default. This is critical in computing the insert parallelism and bin-packing inserts into small files. See above. + - [withCompactionLazyBlockReadEnabled](#withCompactionLazyBlockReadEnabled) (true)
+ When a CompactedLogScanner merges all log files, this config helps to choose whether the logblocks should be read lazily or not. Choose true to use I/O intensive lazy block reading (low memory usage) or false for Memory intensive immediate block read (high memory usage) + - [withMaxNumDeltaCommitsBeforeCompaction](#withMaxNumDeltaCommitsBeforeCompaction) (maxNumDeltaCommitsBeforeCompaction = 10)
+ Number of max delta commits to keep before triggering an inline compaction + - [withCompactionReverseLogReadEnabled](#withCompactionReverseLogReadEnabled) (false)
+ HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. If this config is set to true, the Reader reads the logfile in reverse direction, from pos=file_length to pos=0 + - [withMaxMemorySizePerCompactionInBytes](#withMaxMemorySizePerCompactionInBytes) (maxMemorySizePerCompactionInBytes = 1GB)
+ HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map. - [withMetricsConfig](#withMetricsConfig) (HoodieMetricsConfig)
Hoodie publishes metrics on every commit, clean, rollback etc. diff --git a/docs/images/hoodie_log_format_v2.png b/docs/images/hoodie_log_format_v2.png new file mode 100644 index 0000000000000000000000000000000000000000..c392e2677d54a93776cb06d7623cb4c93bdfb45e GIT binary patch literal 223676 zcmeFacUV)|^9Rg+U6$Qtg+&yQx)wmCiPA%J0i)7Hq&Een7ZC`(=&sU1MFHu80#c=S zB1%yy0qLQHBApOg2oOTvbCaNM`#sP5zW2?%*lpc)`IW2lU3H;}Kn{%4y=;(@~8F#MJgWrF%x^P94j?PDb zjxOj=I=XG}QP34%|C4cHI1*7V*+23^FPDW!>QHEU=q2TSPHpebA8YT7kGdAwYsr;gWOmHGuZwz05@V~p05fWbh3PfFXQ~48DQEw=@Ar;9F$y4GbVP_!b$^{aC8G z4vB(JBgw8iQ*-@CI=bZoaZt+Y0u?v#I(<>*$p4#?Ri_lSC$E&AD zBr@2e){yqbIjz$k zdY%R|*&DtHTq_A{gCoa^e zz{o4W!_y7a@? zVjULlFNpH4;TYm=yRAnCZefx7l=`~NQbZ^(b>huK1tWEm67G(TGg6AH>vnE4(H=HuVCIpDS4|{%iXLB4pA=hRX6RG~=+Tv*30+1W;S`E6a#&g)3+8hw{KHVvu3eiDIf5I!2 z44}Qfh$Al`Ni&sxs0$74(3)htO3_1E(e}th#Wg&*zrxO6tEJ7j%<{U~@8!T1k%^m}wN#!H zdbKTkV`yg+W!x;uIPKYAKVtE+rcd~#nn99*8w_UeYy%guxNYYYu-Ugga;LrQT7YNw zOu6HbF3_Ro9U`0XuBEQ2+m5s|xWKsn(?`}4;p&N6pXK^P_{92PhY|FPNkvBL)@lgQ znk_OTO!UnMZ;cG~V^*sg8Z!1D(aep%DN*k15@jQr*sqtXFYIVIwE3<(1C86oYfI#d zn8+8Vpb<9hDUy@VQym;&B~nU$>bgI!Zx9~N0)6xs8O6ky^!23+b-<=pWOX4V614s<5^CRfwBv3-<%H7czRE6JCmhU!aL7|y@Lwr3B`>g%$)=2NDVqF1p z`^|m4zdz|%3X@wg-Sj9dF48`Bu*ABjqXn1e;SkqXKf>Iz{45_o<}USM1J{|Y*W z!0YtQ(hBWD`n4KuQ{2KrA*2}cXC>V8j!4bT&F6u&Hez~Dy{d9F?+*MMs zvp<{H%6d_BO82cgyS1}re(zrh7Nm}n!tWDZ6nd>oQ$~olc{pT-Uu|ssnq8AyGVR{x z=(W^^MfLKtClB35k-fI?nPmmc$^IV?Gx5q@sTeJ?ySk?6h5K22KXvXA$IdZPe$xH0 zXmmCwE4KuIoPl01)q2+(=OkSe2Cae7g1v}9PJ*E&^UHqr)z3FH^u&+{LKpB{o;?Og zFWs`TWN$RmZMi+8?B-C4j)E^Pxz!$#Z2sVEmesT!>KE$cbej%;b|Do{q|HWek#NWX z3q8F$f6{)zzC7ci)1$t&vxfPY%8ctygA?9@cT+2>cjp-s`!;|;Tm6;%oY(AR{E>cj z0&f;e5+;)?Qh)rOIyO1=Cu)=zf7`j%+59bjh1CfFT;cUPz0+qOB`bNqv~y}LSC9mO zp=O}MTEBz>@g9-Sr3zQYw;7XJMpoRJ{<9XT@2aEM3-I@+r?d!EfNM`e9Y8|%KoqR~ z3(`y3&z@P8tvt-)N?8DsA?%`6lw|S;5!fZC!{w#`+@$mU17{coM`jiYDSCzVmCxo} zAYfQ!VsFDa8^Z!VcG~{rMV)wj&iAsEpENE)Fi;ubPIN0PEF?GLMT80?2;AKA#x)2+ z%7&x%&hm_=10Qi#R@1?x9Q%?)+9yJ=1<){!KuD01*Xm^_S9c4{+|PBRm3F$L&y%*m zBt1G>oL?#}ZM%=Gy`J;{c z3nuhgqY`u!M6?#GCe=-Le=W3E?H86Ww8LJ#(>dh7u@>C}G#a4rtr35+Rj=#Phic1} zIDtMLPtNm9f4iAs@SYjR5AK-&u6ZWs}TTD7CZc zMh|yH#h%50wRrUIC+N9^Kd=E;<2UgAtJ+G2apwM>-4R)nC8<@Jki8kgPT#8ZnJ#=; z80ZGa;`nTTy!Qh`be{sVyt%?!;@fl(E{G>zqUmUi&X6h%H3Cman|ivo#uzJ>t&2_8 z@eWqk*f~vU#zoJU#I-Nj8~fUJQ=l#r?~60-yFG1I;pr zQJmdG2bWZHi@KT2?N9Ao2}}9){<`4Zoo%z#A+6uSa*!qoeJyHZ-?b0bvpdYCrY?ux zMLh2fm6A4>cDb7B1jYcqP&yC5iD{jPR*4<>0NNLyYB}wy8C%OF!HT?mfaS9JzgllS zQeOZOlftwS5I~-P4PW*WO8{+IwTXKWUkrjL$DgmWwc!djHs-n%QA_?+C*^2B8{rcz zmgl6ne*E9@;R!aEZHcK zC~eS}E9RUH+*!(MV-h6I*e3>ks1#EJq(9cgqd$S))m`EdbODauWRe zB3zkW4nVOX-U6wkr-#Ko6|Mg+Gs&*R*|BoNBA2p7-2JpPmX0n%+P5zv!7S#OpB>e@ zHP$((+qfJBv!VrF)DP%_JXvWvPW#|{8k*34{gvkXXum2eQw2u*mF~N5H-8ft-8Utn z`~PGZJ7T_MK65?q8AdW6KeL?b%^Md6ca@JB|N8LiB_37xS8t{CFF$+NDR9Cdt9!$i zaa~95ERUAR3q)n6cDVodnHz^qhQty&eLC$AUHv8NOxQcapZ6af4%2D04#Hr~~%TmoQ?9%bpe6((k8+5VQ|Ef9CuPKRvRstxbbKiw(v;)IwW;t_n{^z!_c*hLhW%Jh zcgn9m1yEw76VDycehe14+9(XFv0nHJXKT-|X{6 z3NUrf_YKDQGx3Ir)w4;|tl|-Y(79`?LE+>RNq5GBY!?3ksqD zr}%;gIt2i|($s}6!YDuSxB#wE$H?e#WMrfre6PuUc9r(r$f>oh&Fy|9@1aA75?m>F zfPbMwc_@OcerYPoKIkYlzp&69MN2j5I`ao8JY^bJPJVtQGnm?!8|8v1LI`qx{_^F? zoZMW63Qql633p{yIUcaCX66IMLrAic7H~CdiqF$Yw@5XI&M$ko&0tb)tdcuyUH|z zTlUO!NAgg?!l-+RZ7Aiq=TynDniN^8?d>VT!gq0?$j*YsivZ=@y9$P@T{{}{ zzLZ!E_TZAbdUw4%ZKNR4iz<*Ll?C<9&3T8)ox*c+au8;O%tYr)#E*M3uNJ1NP_pU8 zix+$P`p#==YS!e-A{GZq#c0tmU0Ul$ss^fEH85xqDy%*5RAV}%@4a=2j?V}u{= zsvuEqsub8Cp=L6wtK~SkxLENbCgrM{n(fzT{`S~a+T4z;&7h*ItE*M>GcyG~w6;Pw zntM+@d3WQC+KhF{;0X~CP2z#Q*;dO_WoXuzAn9PaG+Y&Gtf!|3WW~o37Z=x3?lc5a z_MB_G(8MJj9Pc?*%il2`SJbLsf8`w=kLV~J4W#6F#J3UvR)ix+kl(BUBHrCf3O z_9u#bJ0DJ?Qu#bQJVK4ZUO$eD<3l)mdwV+$m5Y?SU{W@2Qf~k##D$>hYhz!Y5v$uw zhh`60Mmi8nlf&c3k4v+?_ev2#v0c=yxCkJMvK6uKZ&<#6#BT?uz#ABGwu>g(D`1jc z0|W8d`DQgzmo8oU7`3;!&J3D;{K+O}&i&N{vv_DZIbB){ZO`930yDJkNP3zLB&`On zMR)B@`$w8WU((fu;6uRvsfNH(?OI1^#vZaQThQimG)6!(thu>a)oJR(6G)d45__ZU zhEwrC%W`F+Nyoq-E?dO7^u&E;zLYQ5jKXOK_6h)NysVpbdThJ3RddAQFUubLjk?KK z`g(c}K6&ya)EMjy1FzR_SlhiP9ehUF(%)|W_hiC4DdK&Q-W1LbH#0L!FSqSV=TP+X z`m$=<`_%b9{w$cAC4=$eR_DK5BJ({}GAC{A?ZZBN_@F8h^S*H(v;2cED`HZiuIOh0 z&4h!HZlOOs5y~hnmH0AUZrXz~9RFJ|#>$joUh9%vCJh!;vfog1L@_rr#?Wo2it>7x_qiZjGcfC=7oBkfPFZdNZ9z>;4exO zq)&sd22M^+OWlZ_&8j7I5~W?tBMzSx_)2_tXfEF!zJLFI zao|l2hGedL}P%jI|7LnPBGY|UiTewgK3z*^NS)NC|u zuzNOl>e#5NHZmyr8UIqZ93g{WM2492JCT2RLE6H!s3Uzu`V>|dyd3)L2UZf3i?H`h zRaH$TuG=Q{6l(v1T+fpsyt(7*mpbZH4QF9-sol1&-B=}S^=R>|vx|?l#T+|oIW|^& zf@`2cZN_W+*$ksSyM@1bjS_8j(Gz3bO~vPTR%Tqs9=yd}s_-Cf3~ZrI zoT@u25ZG`VeaBGtz^Y>#i)i-5{`&0}HzdxR&u9E_aJ%cR`^WGNELm!UbNV_N%gX^C)pGAkZb&ghhQsRDFQ+5A{(Oer7FdHGRD;xE zM+sxDtgjH}aU@JVNqzF{T44lZbIM50+9Haq7_d=S=1NY_N>G@tbIYss^;cEBYOZuf zl$glOQ+zIMK*2sTT;P{ThuxuBm6VKuZS3HK5+yRx5I?!!m5}cdTq}k{O7~+PMY^VP zcg;^{XYu(c+LNaOo0pO`>7$)|kFXD!zvE9?aB$}436%8qD%mg9wAAjD>n!7_(xJ^> zcyhbsmO8dn-=aI53sFkMy??l)!E*{;l*bZS+h0+*yK}vew31*U5Ws7EJK3b)C7Q3@ z#5-*TdyPQ}w=ei{lU{0F_F!}NA7{B=av}Xa9w(0yLU!w5oMe4!Sm&Hxcz)1~7bSKrghVu~hwwBX_8Sk?bkREhG~L%atNEzmp?QG^ z1Ya=E%{Q}Z4~OcB4_chekP6eLZU*6r%3)Q+zG^0y_^QAz-6Ak~CWV%Ulz`c-%ChRr zc7ipbBa^bcbrlrfY9*vivIdw#KwZ%rYo`7NHN$-EGVL{B2rpku4Ys&%cLz7QkXkzv zf+!xW-RO;X!<%7aq2s)uQ^fu8a~|l2v>IDm+YkV-{PrO9n{Im=gOC_I;SH=c(WxU| ze(jNrgO`97tR_%ZwRWpH#dcs7aCuQ>YFzMumb_P6ha0J!RA)ZqznK1$id7>VC40+d)Y(V6+^M|@i7~KP_pSkdZVGNm zW^EzNnJo`MtF=(XPo|?rULU#X+G`^gcIS`QbQdE#iTYw8sX~8AeUXvNw|0OXtYml# zS?!_}N2VQ?=G$drY|Ibr>yWR~bK6e!xsn@2ZZP1D6dOjgv)GXOY?ef}&yH3E+L^jd z&ThVbt)#RJ+nu=2)4uD79UVLvJ&bqs^J`A!M2MwsjB~7XxrL<=A`LLY+uWEPE6+Bh>J83Z<_P112%WPK$Yng@sYgt*D&Xoy%j@h3^M+we7yRBs5Yy#GTbB%A? z9YWf4bA%6uLwaX6hH z+R^Yn8}iWOyS|54TBe_d7b=PYTdbe$INvULn=&(Wytwx?f2crsNJz{!UUr$=(Uf!r zGSJ3JfGbUX_@SX5`R?9C*Q4o8`>(?%z}S}a?N-sV^6iqt`?>tBcQBY_)6Pmgu%^n* z`8CVHUBXIJ5k|zeS+4T1Gw@Yb%kL&;I-D?O>)wi+1d5W<1!c?(P;*>&hVCrwXiBs- zSbS4QUq2So*DbyZl5Ydo5^+N;a>yXCKJD-f$>#|-m&I5*&I~C>pr6 z{MiEGiu~&FS1uiVK(3IQt2&K6DBc_k9&!@la2@Wm<%^xBi%Mq2b#RHP-vhae?da(`!!DWf{CL5+^^?7F=D&S{N_afPNkD zQtqFaG=hT7sLOvtlgvn=|CzM|eGiwiy&AYxm|<|SRgv8eh?z0rA_C_ars-gQQ6lD` z!|O`hH=S&_ot2ftR+hNHUD5j%KKe<%GHI;)on5;Da!Og@yWnPTz;kXrzY3@YIAA^Sbg)`t% zE-u22=rsznRH#M#$gLyT4FHCknjKpn!R-e}f&JBClE0#$A^mN^MoqT=nXF*dRW z%_zJFDSmY9M+pguMiA^BCk!%L_xu(x&~KEG##(2qM+$9hrg_ zvdbhB5XDTF6|!-6f6NRbn3A73nu57A*)@C5-B#hy8kUKMll_OSJjCc)>13=oqA!bZ%a8?0*?+NO)JsXrFKQ zXZYqMlOA8H`1!L`9!6REdN4#41WA3ESrd1iXoNywRj@~oAA1eb`#fToUObV#pU2~T zIG4oe*@=vqgA##@o9!L@Fvt=d`CxK?LT1phBAfpDcaV}7@rYp~;DMWtUH6~4iE$5% z500YC3T;V?%*p5+4+wd?e4L3(`v_Hrlc-l+Hi%8h^R~@)`P%LI5hl3~Oc9=7!}XL# ze+4$J!Ci54#D%#^t0=XQg-<~c+v-rFLJ~z{qT&$c*CZjWq6Aem;wO3-lQ=XPwlyqoJlc41jNPA{PZ)bgkheG>$#YY5@}$SQ;9s z=xegjzK+%s5yqyE6b)Z!^&IN)1^QL z&vDjT1KkyLuy$oWk-n^aF~zkqGfEmWpnJIn%uuDn$Q!lowGs!LxTUx`-u*LTzw7}B z*(Y=0jTcGVr9%n%lDFPNxa(@cw`*KXjXM2S|0tB_ zw5Y@;?3-j|UUM^E9Obh>`(+G5$~J<@cF?IO%s}D;u5*DF?q8}h6&28ke~_(r^=ji? zdPWd~ySoJ<-nizctWAh{HlA&*D4iYHaC`^(btty89GE>jiK}Bl-*?pp0?6#zfEyqLB;7%SAsTxUTMz@NDHw5AJ3J6aO?*Ipt=W=E z_2feDXVxCJTQkmsL>4%d6794s&Rf&(>C55*@W#3QC(HMB5DyJKv}&!!@Rlub%RCo` zyoDOF0LgFS4o`qQN@9z@Sb5z8n7u3}a4F-a!DPxYi|6+QP_@rz>rz1YPy`~)jMxmJ zt;9rfPTEE?3e!Ka)r3g*shJzuV|s!PddEQsrYzK!q5uPg9&6cDYaHCj$cW3}`|4Z> zR`E&6g>`zDp=ekf@fex!8OkZE&yF^`eJ1WoEp z-g6b!8(oHBlXe;EmB15c0fddk4{waL7kHBlid~0txPGpe`v5ev5#v8;MUMxurkp$b z(`imQ;~HI^Nue61n%$Rw?52;ch@i@yv(iCIRb zDP{v{MIi9b^ZnE>Vm^uW$2F6}Q%+?wgPgC?tLMoK8@C(IDL2uK@I1gj9Rzdq7CdvC zign>qytS3r;lU{c$miOFP_9_d;h--QlX=BJ83}}q`S!22MwR5bQ#m*CD~VhlP)Zp- z&252(iSyDNaeDtN5cwXfj2Y1_d}IYBM;bklPXiElYoLmc+tv6Sc2`ZiX^0qxi|7I9 z4b~;Nu@JN2L@`Rbv}7vvE)xMr)IrDcG;;$3c0dx^wd|9-W6s~g=Jdl$QXF}3LLd^p zahVRn+5US$0MkobWQ1BH8rKTLsiS~YTtAXKAcR;>bCgxH-D<; zr>7rf6E*}ST-rW}OM%|G)5GQS_fNn`+L{8Bin2!o5H0At(jzzki@?!vbATMfXFz(I z-9gwJ1xfeQ)3}39dEVqDArV=WZwnN)1rKKsG*Y}Th9iQi0^UiRlr2SK>gBpBuYPt; zHMja7K~hKW^f<BmmB3cAiNDF5q<_e{;(piZ7JR zY6TP(t=d@f$tfF~3bKYpz5~1qydNMyblNtm-xV56UxVVsY16eV>)hDgB~37y#8oPY zJy4`Z32L9#w%%DYdZuOP=7v8yAoh~334cmfYFcT1ZgUAJ1#oOm8gaE@Vp2N*g3+bVf^ve(~j#;R{{e)S6QV06}^3vq|>qMj z23ezBcimwqkMA!#3q^4bI@2J@)maT z{{udw#thZlYv%TLHk~kk3*zfQHl}#0kZ&+^%{)+D%GrYCy;m(WONw@*2beAMF6@wy z<@lR>q3ArL63d;<6+nDfKY()24nAF8RNlY=WS(!)Aded~9-~ljBY#YX(qe&36X2Gl zo$JWDMRx5%Mbi!K^fei^7!YYT2bb_gu*4;Ir@QeC-zc7xhFJlte9H?1{;-h&p2S8v zBgor(KB;^(5oBdnz@IfQtD~S)dSrf$8q*MgOH#+mJcp$}$-L46OZ++FB7b?THQ zX&bL|?Fi??# zzYolnIVrw@c}KUdI)XDh4$a8vF{SoZ4SCdlkwn*^3>>gC(B1B5D6S!h zS1hiz>G4V30}ySZ9xDtIL}+5rEBOJXa*+oAXG!ZgmevReg5}?713B&PY*Fely*)ju zplwmtcPYTXGjF%V5smqkZsMlE!w!Hr6+HQSP5(mJ3$I}FJ3tBQddHA?V_ z_-hUD_*W7*}&jO-+3% zMWuyPi*)Y3iBe8sRcgkk1-aqP9vlFyo zw7#n3@q!xBSMu9WU3PWy#TWs#f3FaY(6+(EsMgE1r?;2?Un)e-zf%|G^xxI9+4Hp7 z{QlpwX{XKR-v2S12&%TXn*1NLp_ZCDU;H1k$)wK43~ck~D(By16G<(nR_Fa6voXAg zrl=75Kgyg}irrdrS5})n>{FEKIN>kxUguXwLE~xJvpksX2XaA)l>GLDC-^1*~ z`dl}~7EP*d-=s7GKtvA@ z)fRqJLvC-aZZ;_SXQl;=KhaH}C{&C^fUb$CyW4?*5-I*Sj6jFY>^Yi%e^*ga(MXgM z(P_0V5x40`qAFkKMZKNgmI*OI4NVxRU&Eoso~TS(zt1fQI?hM)CPOt%s4n^FK&mvA zF#`wix?WH?v)T1b?NnY$jkWk@YdR$^*h7wWWvb0<<%613tf*N+c6*|<7$__Qihu7K zpA%G5&Fby!{1x=?sP}*diMFR_1;0SY`Zd}xC;!e_UwH&AnELWLO&dtUe|z25(NUu( zR>Xwu=FOX5K}RqZ6W!$i<6Y!F_fgepwANePx-I^TDblRW&F&vXiMFcrH*Gp68;#@BEdR+?&op z+7+h&u1Y0pLfsco{|kg+@J00W_Y2x9Z8k9fWwSc1Te#gc&@4gZ zRS*L_8|g2lK*@uILqhtEZ|*U&mI3&0+FY6J{@mz5*G?@+KR*g4_vcTn2g<_2!VF(s zJNE@b#~x7C4{oNI8to`$a=8$SkiSO!*GIgAA!_4RPy64GE=`U0U`^0K^%JYWaeL79RdlW4?&qCWt}z0gNnpWD znG{8qIR$FwH%bQY`hd=azc@*>S`l;Z_tkt9{d%nXx`y{6(e@6T`Rv&d$gw~d(mY@MKZphD#X*RN zKqV&{LjlS~9i^H>@YB652bq2J*L8m7nv)v@*EC<6O@v4~^!>+1W37Q<(g*S}R4Y*j zt!V{W-^~w?zsmY()3ueR^+6zh`Hh~c-iRk(bu2OP`JJY8%TSUEa1(aAB$edYfN8mt za*pb}3jWsD^Tpeu+o3=mKp|M)l?f|50Qo_P62O1O!#thS_4V`)fu6*g5>ak59 z_oK>&Jo-(gWpm*A2iuJ$Cw8)v6I{h&yt0f*G@A96s&IPTk~V z#b0g1P2VNa3D9k-V(Hfgh@{QkCH7r$Cfe7E38FxgQ1IML;Vl9uc8$va-{+ zS;I#b<|HgCEEcIt%d>c8yMe6v^twQz>4uj45k(&@Z{d+;(~=fQfIDh)sZ9ojm3b;>q?QJ`o?vQV zecTxdV#5>hl^r+O zjjV)JXY}oNE#$dzJ0v>) ziJyvA?=M)*2JJF%UbD~A4jmrN^xCL^S-wdwqj#HD+*Z{qYXrhk(6Ctp*6TfPE6)*lkV9 zmbt0i01l&!CTuu?68RyNi%9i?X1hmB24eehy5>db`XND!!;<})qF#Bsc`a?8)u4cynjS>rOH_w zsA$JaFc28R-;b1K!QOj4;;*AGJac_E;sEt4lhZ(7*mP^9k$Xl7*9 z4&m=s3>Zi@{ys&<)SB*dqv`sLL3CaSW^q{W6=Qp;?^1N=Oe)ul7Q8dU8DQhj z2#JKXDT( z_;WTAFc*O21*Fc-#^D!SDdA$Kt8u|e{Z}+`wOs1V=&k8yaXvuzM>A`uRJfu>2iDPC z;yXRx<;$u?yE{@cprC?DUkuUw-wn%v;vQ|wfbpbj^6QFm^yPTe!(O~%d1;uTZ6a@R zLplk_hWiDqr#8!CQ2%l(6Ys&Bf{)9+B7ug%5zA?Jw{zGn6c>kO2h|yu-p<5{MDOg9 z;C@cI7WOBy{rxsOLqa$L=7a98?Yeiz9#gc@rgk;jKGaNUg72WzDh0OsX z<^BM?D5K)5Vj)`}!bmzkb}4EtrF(a$@vd>v`PA~wj(HnUMOVnGopkBAp6BZUKj$>D zrD9yCRV0fHu$s9(^8{D`kh*|9u)i+R$jjvQ<(x(t02iS=w?>CVW)$4`{L{@i?^MI6 z!P|8JHflF-uBxT}oxdr~H5D~jzVpPaDT)m0{kZ@~K*dr>q+Un^GK-}A>-~v3bvKN9 zoJ592rgN_U5j|mcO0=TlWm;b$Dz(ffxxCF=8~-j0vcGgGV8mU(n4L^lqWkrLN{6Bw z_RHX~v1=HTDMHzrKo3J#0NAKbH?kh6`Hi%o zOJ)ID?0Q4`RbmM>8oJO_K5h>9k6-u{~(&Uf-< zR!W%n+O!&Yp;zQgfq#~2cL2Z{>+Dr9mRn_=6sG%wDoJUWgTKz_h5viCvKI|62a=*w zbOYYW;#>v}Lpo#@3Bh`zr{MNXOR7eMkk;W}0d|SXERC7DfpL~X5Yux%Q4O{oG{fpP zNDYJjF_6nW{2a{5*fTkIf_8TUCV9l0p9G)t;DW+YoQ9yFwj_)j1kqVKzMGiR<7||B zK8bI^CQ$T))(HiBLRN_nxkG}5h2ja?l7bD&d`2`2>vUdi+?;s~2im|`-!X?NytX?Z z$N8~dGTE3)J_L2V1ZOV5nF>lz6|?-Z^5=BqH$9XRdFKv}ur$IIxF#LQYOyoHZq=Kk z=Zf(ssDY{)nuS@W^2p=`?l~)pqLVoW^4oEWz7Y zf}F~0-aD6))uBq+)5|^2FUF)p#2}T7*uJqx`uUW%@cIgir%2dc0e!#IuSMlRSpRY{ zNp}?YDIG575gSd39c&xyisBme_25ki1n>!HNe$=Ex!_%!P3I$bAgy z);gI@+L7M>5Q9WERjm;TEp=Sd>^+4tghlv2t-HqdzC zGtwZC?m8b8I>RixI9&Wu5ppa#3dKiP^uSWIMj}{QTpR43-Zh>~sZ--}Oek5s7ZBcwH`5z~JTJDB|mUj?=mda=sCHtT!1R1cLjj7bbK^LF{K5a>|XDQSQ z=$rqNV$Oq3Z6M#3*D|0#2O@ckm@1tn4(Y7q!w!^f_hTUU90|qqi#xKd28Vji2&ViR zNQ^u7vA&c^&&L!O?=hDRMIfsF8-EB*t{$=BL&U`c$X(H9W*@BqK>{CQ3644`W}NEN z@M?29M^XpE8hMeGLTwVj%F!0laJOv&C??twhEqBATpiE&2sVXkISdk1JnYs3w1NEc z>wTZek|3?|xvW&Jh%Bw-1jO?{$FrD1prGXA9|))0c5v_dMwHc?? zRE6UW4+E#GMf@l?yz|o}IG+Z>t;jI)Oq8MbTRh~Z)Mg|Xea@aIBb826!#a{vD>K`@ zFZtlZa9U4)jT5dGdN^|{Vbr@~c1=dmcka69sqOJV$B62DF~N?Cxiaa5x{VYn*?Q|W zP}>8k6Ra#nppX7)#Yfu7i=d8AN0G)ugD&4+u z%eD7agZR#4C8;b&A5x^3*IHjXLGEb!boG1q@H+|X&chFRJ-KU}RQ(JzWIa zWw@fW2%Bboal1ngNJx2+u0G+2=XRk`?3xF{uh{_*mE28^0(W07?aHiL5JMCIh_j7z znA4)@sxou`f1N$)e3@rnCnsLNfd;g$`-QjKJiiW2j~-l8L>Pt7KzU6{!qyzfeEEUX zOy1ysp8njRFy3`8d()y~*Z&3sv;9gfq}$|bJT3!hL5)p<(sk4DDB{J3JbupHYY+7$ z84e9;&RC>wM^FOmFcgej}|<7R8-O7T7*K4{C%cbO!{xZ5DVZb!R1&qn^zbJVqL+{``QULJ!Jgdgh8*UgQ8g>2~&)26&fXn%Ed?vZ0 zIg*QYl5!ACWQc`+Hh?xyPz?NFS*7;9IgMTr4+j6=73v8?pR~E|EDDoT?l@RX z$=mmQo%EFPV4sF91mul6e_4Z`t>bf8G_~Q zQ6Yxqjz$#lfJz|HORf%#M$tXd+T}N;R^KI4>G;`9uo6b|gGqlsht$arc9UPcAUk{c zNW8KtrEV4E{1U7Ud^uQbHdKo{Yoql!4ohn_tn=}(^uv|`GUTV(<5W$Oj`^!5{e{SP zKTNaa;Yy$#$y8Q<)*5Pt*eiNCKHZytekXt^=)@nIp562TLO3mExu_DXt1~jsu;?h8 zsA>E^DF=2f=F%WZ6TXYuCJilrqi$$PdSDRqDoF?O5$(-S+%8|W%Aw4 z^c;<3NG0Nr%&@^S^951@oq^XQzV>QwMS_z9J190D-%4Mv;||=XS|2L5j69)~GG+xp zKY&L3=_5M1+1VkoZqr|?snto~`>pt!Sp~H;U>|i#rl&B?P;$6!oghD7Ue68q%P6na z=t&4Hf_3a!5p@K+2XH4KW0*YCD$mheLr&wqB$R zQfL)Ml_;BH zRRQQ+1-TO0!9}+uQUb(P3lD0lg#!Q|T1$Q?HIoPts4OeJLZa$KeV#*%sivL-$#y9u zY!q7o!Zz4p*N8A5#fI(cj-eWpsPn%x!$G#1+z3NDgxT;J#Ug&83*yZM!o>clX7|o+ zv;?J?$YB7^vtGPN2qgL!Sw98oUZX<4X4JVr*V|Bhb@V92uR+S4IpP-BumX;ShkETa z&(Tv-i$UuoNK^FQ{?PJNknF9hn&-1JRcX~O7ACa|Nt}bwa4tDr_BjZ?@>6v6jnO%i z(sdi>A@5QW9cY6;3-{vzjEPBS$P?L>KO0FdOWxS`)np_@J9kDx9TR}EN!QoWVPRww zzVNxx!QCiYVZ+eUuUF6D1<%X3AXY80B{iV@&HzuAl>LP2(zt6jNCj!{Go2S&p#+~H z!Sm?uz_KeiMe%cA2cYKiYR9+l-j5Ogqay@LuqE~aUfZ^h`>AI!2=#8C=(xiN`L8sIgg)M;lo!V>OaDNPP z9%=FV&57{tsGji@m$pMa)@E`!(Ag8`%FSast{;N30gnL@;ZV#K8(1zEB{cU$#<_u2 zn?2cr%q-1&n-B*@U>~>&#F7ZxTG>jZ_4 z$jCr@cW@A$!u{qkNiU;_Y&}4KMW?kn`WFjM0Q4%e%}TJ;M?=z4YIUib0}djWI+xxf_&cK)&IA3HZX{_M7uP zFYVxRr+y%*tt8(P7{O0guCW^M-Ofr<83g297br)Z^WV5Bg7L6z21Nky{#dUDZ*Pc{ zR2p11ZNSd}JdkL;nsny3j@Hs$71O}t@v6J%DV2_>kdT+`zo;i=0ERZfiD^D`PZ#9= z*)I)QBY2sOLX`;)3)iMY(3GIi4^RmwviqI(eIx7JT=ZpN zQ`?uSyUlNF64L>}6-@G}7w<2UF5lmrVwS?81kxt@7DsO@8{jg7@0ZV+dKQpwTABjA z+P7aiqeDJ_Y!lWepSXdYlc0Vjh1rF#>%0fZPxo{ceiYR1KfX|BEn^g`J&iSlw|i!3 zgs8KP=&culnuq#g>=jU*s}=fy7TjCZ#oP zzU?}Aa&Oz!3jCC<&&-G_Yqz$v)5W%}&E-DnPmVs=$bBwB|AZ>kr+e+It$0#4El04~$A z>ywbxG%1>UV*sLMp?63KR5b&<0HlXHA9MS!&zSr5y)G3iGpqF^iAa)jTy@xjUEYOm zT3NLbQ-gcOBpWx^e&FN46iu@;)T01aK!61RqD0S|$$36BWOE3~{g6Yu?wVoxk*u%S z-C#3h4fs+CXlBZ-U*d=aNulOwzSPfT*DRZUp?NZgg)g=YZ2!EqsoW{^s;O`tAXYY% z9U$KzsQo30kPJ!fEiVSJmM{FDCXWg-RgvCP*p*Wd%U{-=)kxmj z5YZtMq|ZYo36NMn1Q4y#0g;1{^}H=0FHh&&Sy*NnfM{kSK#b}$qeI>h2`l3&CUv{j z2A;aNmt@`B#Bbf}N3 zX6!CFBl9HZk$Iiv=F5H18%qCGRRQ7DRDv8~!5EP=dYt^Lu4#?*#aTgN5rpU=A>&!U zT;TWZ4OxS#DQ|iAYDh!wQ3lRimbp<%b-0W9a=Q>sUlpzBdaZV;mq#3=K&~_BPfU_3 z4C^l7E8V1>MYH{Zn{q{o%`?s?a=Fo$5sKGujFv$$LGtj<)*7Sv@?RqY@Q3QiNG@rC zX4==IMD@KRDaXzoWC)<2WG*KH7(6l8o2$1V1$Z*3=y$Hp@DDw=`i%kBmCeX**!niChSm-w z;nJQUDek#6teOuJnZN3uzJ9*(6AtYp8FB1|WOG1+$Aj9;&-1{;V_y?M3E}*7;|*Oc zz^6a~RQaYnl)zd!ZyWoq?81#D3v;;w>~DKMgRB&aL{y#T>E9aHEdhQtO2!22K*n6M zr*55+9ko><`6{J)|Hwqp=FC&W;u}!QUQmWDI9mz&e__yZC(SVK_3E_E5w5nuSI!%*)6sq?T= zqL*LK)2uR|c3fUSrde?r+)CR({}H8EEbTG?O3*_|X;kW`RGD_&c!LU{nNBcfupjD~ zAuo1|gM$R&ebC_`!{w?^d_mjbbZL`=H6V}hahl?OU7 zm)j3@Pjjq3ydw^}%3U{twLp8d2r(}YI`9CF(S(71bcIi@P&;gev^Ok=4hBkAd+#(9@X-zpMIs50obd*HBgH9ZPUTwCX z3VGrXTOp-?yUbM&B44eYz!49qfgYTD7l6+CuTW0BZ&ncMxX3&`jf?{$5y&wRh%*DKpfQcO&q8q26TSqj)yk&!K2W#ob^sP&}~2F=x^hc3q4 zsBdlW8K$99+ev-nlkxRoivk@05j^fgIa10>P9U28K~G(5>NcvRP_?4hrMrx2LT0isstRJ2ELv~~e40M6BCm)E*$ zy1OH#g6|o3t7+~BmY8P|i@aE5Fzci$vt74ZOLi_WmJqgGk1e=ygb`6OK*>R@cZGCB z)u8#&IWUlrm~sv3$(%d)bc@&ps4*rLpn70-ZhsB)K1jpaT~vS6v-J96lVyb0 zJ0ClM=Rs}oZCE@qjI)I{(jZ;|*_FF)MeiWa@XA<>)vx85Q(o5zwvv~>a`HJR%g(I* z3VBzTs=fQSu+D1{)d|m9t)xJ-h+r(H<#`gV_^{t=@2^D>k zVO>TQalnNWNM>bYA})><08Nr@Sx7hp&N8LY0<4O%0-~MBjkEwE*2oXBwSYN)Hg0{) z(b$u>4*~HJ#|2Tk5v{>z)V#bDC=QY>TKuDrSSqqD=B|U-#Bc|Wn?3M6{Ha}G*6T8+ zG%B}JJBLxXYlULN@$qFt6OhWCCHonrqfHI-%^S!#|{qMuP)LTR)6F zsn0ZTxS;}z3WF#ZhYe14iS`SM;cqQ21lRbiJ@B-GvKV_OClpE-9YFbHeu|G3N+%BK ztoM;by!vD7paH%?6$rw_yM`Vhy^^W_c1sL(5ALM$)xMfdnN2F;bQrgE)GK zY8NTXOij(PI+FFc9SGvQj!ycPApg+^{7st1ZmlaHNnSK z-1T*YK8myxGXQD)efTG#*s5ogr4o;3 zWvny*c%{KBmw^3nAlnu&luj66kk#d2dSdhNtKNB#S~+JJBRvJ8x=tWWy0p4x9WEvV z5i1f)&odaf&YKsW4Q2(TN74R#Bn5z6*9~+@jVo#J^OC`##nV9SmU*Jv212u&zw;)d@ESz{0 zbT6YP;U`GM#yHcneFS!v?RpIm%4^0kYgo4Q5_PVpE07#Djfav&i zsP@f6eH7QD4E~$f6J8twD@D$DlNPv?hvP)x0Xl!JYbe!jovu&SVtp?LWGv4hN#2ze zzT}F31hWPNk$p~RrUBh4$ztFmZb{){-;J-#b^*DNlpH(e?k&pY?sWeSk;cV2y9M~R zT>vn-b82=qLpjx|U0`*8jp605U3i?(Sg!A6gAg2K2D|R%*mcuP`|(6#w&KW6e48J4 z5>-xV7vqTYv#r+D#CgV%6G*f0KlnBLMk&ia$#@$D9mH<6^9wBgRAy85 zp`eikf1*puw1E{5dP0^RcFc=w<1uzR9nE4qA65v?bCMa-mz|8vbT1;k6C`|sGKoC;V{DSceSCZy0&kZ}gl`B2Df zKS7d!%#Q`w2$7n0Dd*6c zJ04Z3eQ(*;NELm!K$b{%RL1WG-|zUXkG*g?|C96FAdx9rF)EFCe{5#u2#$(G`275=BoW|AXx(9nS@m&QF}2)p{9gW>lH)F>LTeKTadIPtmC{Aiit_ zq9Zhjj)=}wWcK*e&POp`+QNldcd8}(JhBbOx0K1#9by4gsi*HGX5{L)O9;}#040-{ zlClM0$g><*aVST^g(T6KId~GHlzV}$3Ga< zO`ZXFJ=-V*^wK_r+m3|e44QyRIy!>1sh+nZ?@QJ3Z7b0PlGFd11?KG~`jH0*{w5@W zpsXD9fTMnb-pL_ng8xa|qNS2Pbt>+DVeItNuT#ZRVf3fwMjH~WycUN9lFbwh8#;)pXFPV)2V<%K<0nxv~}-#vHIw z;nmuiaoaAGAb_-#EXp{GbV>rPDPvnHFUfZj5|xwDLwklgf_9ej;F)ZOYH)eqsPSCH z=Xxj)RX6DJUN%J(q_WMw7xS&mZvq_xu^Ac~LN$KG#$QHIjI#@@9i>?)HWZ~7mj(OT zb{78@zc|=9iE;@jm#V5OO}m}LE%b@B{1ilthJB#bOym*e?*(`QI%7~<(WXv))2yLQ z4s)2-&Rk{+1)q8`qYmVEN|CsX12RzH^0q=zrQHCx&H#>~atyiWw|a2Dc{ULzqGGcx-X&23-}%raeN|;T@6a%I*=A?edE0 zmjrqjij}dGNUDsvM5tBaNJ1iVQk2Y{z-Qq5LC9IFKAtbaEshqS#5`$7RMg-}p0e|gKuu4y$I-%vZ z*59Ce{4IMKQVkM2OeoYG9sp&7>%vI9j2uKBf||R2&2(oPb|DHhER^XC4+TWUf9eCA z*x;+h`Mxkuuu)NkAcze}y6rI!QsP5l&S}h?!|P(P{PV6d|F5ASZirj~(v~T5e({WM z*MhIVw)r@vYp0rCyYnv_vtHq)Xg||oTZOnpMZKK`rrh1c= zMJV@ZRf^$B$-Qfh98e_1`Q%(*v_y|2d{%t@(avlrFZnzq**u*4<>A=BPwP8LOWP46MgE)b-~aQk9F!C(m@DSZ%|O^Zu%4(-WCVS8mKY%925r2dUMp* zRJcv-Sy1lsI2Xl4az+i>*wEF39-TCUZY*-F-?~66l`<$7evQ}#Kz_%cuYf5i)aeBB zqPg=V8}9XrV15QC#H_A>e^J25@ZCKz25O9x8PRaVPXNBiFw%q(-U-*eIZMoB@26kxj}w4Sm!}g05??P-56~m`USToY-Zu{0E1B&XBjuRqu!U%SB&yN#j6zQYF4QHQVpN*%ne%5wY?^2EYZIC_AW3NY| zDWAwmz^wy}aOmYIk-n7bcwxNblF;c5PrCz#!2b#^c74|R$wKE|aFyPNsITfH4Yv@~ zn@P1UVh}+`KMjBML<0cn%xb@NWyMVT*xt9BENX1tATkXq$1ks>+v`^{2=&DVWq$=E z;OX~VfM!5YHEC!$vK&JUi=&gps8<#~U%{p=+-uzXjf8w{FZx$9GkW7^&y*5h3q{Ki zCjyKo!dA;Aq`5I@kOF3PuyN3N>?0!7AXw-QR6ZhvAnGikp*iXvNVb@yhB#jFB_dDw zFa5*U{m%^}TOaTdnz*u&EeT$M(aion_=>VXe@w^EIW))Emi0n$t@Xjo8nh!zq0I^) z#Ocou5ikUBBRcA_+0w+@U2Iy&?(U7X4RBhEuPcXzRaX*;tF==97xsmC z)sKkg4GR}i@*6%8efT+L=D3($8I`C#2r0uT8TFMCTO3xN`tLSEtvHxT88ivPP&}MI zHfU<%2>iOi2LM+cbaJTvPcrsrg7AE+&hsfH6-__eW7hs;tTa;_n(o8cxJjQ9|Mz*g_m9#7=uyYl{{SUvNz z%9B&4^bahE{~IC>Z*(LtBjb7oQ?a@P0mQgca~g9CTe^=>HP73F2NDrXhv~+I)s9&) zwVXH!gt%#^&I5K#W+HI%Nklea+Y;r?$%m-_*1-NhSS0>$MUxi9$laV}#~-1K29E$n zmFEser>fJsO_HtM@YJ#1o+|Y{MuW4HcHaaLtC%)flXPxg&6`GgbNAP#R11zKk-io& z)DW|md}tm>TYu?l12<@mYvU#=GQptOFc(42P(d4`_E-EbCRV=kY!mw81gx@>K0Z0~ zx|KwC3ikIn6;FoNE}Ezz?og&-6@r9vu^+bKHdn$X0DKg-Y6BB)ST>$X!Z++e zcSaBg@j%chfxPk$(((YXhlw&53_`Xh?dgjg>-=UiU=u^XX1Ha6Mkoymk7+s9u z`5_60K(_$~0~71@M+lh@ZoL?{ZUi1{8*!$zLH@)J1&cWi8gI)&Y(&+ZuF(19@WY1R zA|#xD2H7DUqwtPp-+_fvs=Xv(>kRK zA)su!Mw!`C3p#fQBEWms{2zW=E_K-p6*G%4d%=FQ-5 zd;F#~xZgcSh^tgwO;CGP8U&{TO=8$Y7oZ6!OPDs)C&rT>d;ceZlZ-L~bq$cVA+c-V zd4K}Y1h>5T#cnvR`v_Fga}9X`8s?V5`e(N87l2n2>F-3ufw{8KXD`h0ujU6DsyFTe z2~mQ=?*Hm9gCjvdKhFOHO*}J2yUcO}TUe-6rVTKM)01|V8NOoq_QtHr#!Gn~7Xa>% zZe47Z9BTI)WF9WsQ{4`aZ~%>K+}3*g`8gD6h1RU(^Ot{ON@0xu6=)#Ru(^^gDg9SgtiE@$lFBfF}(cW-2Y63nIH-X z6D#S+H>X8X?0`Pe7L$u+`M9OY3`99d4)h0WL(uibCb7=epLpjXjo({#6+Yo#RSn44 zW`B)!U**6RBNjC z3qavNd-jcI+!MGmJ1o8U$KTS~{4_*~LQsv$SfQWayKH;2>&`u|abAh;ZRN27Bum8+ zCD1bd!Ccf54G0{W(qCaGiai#=>r+%zOM*edw*Fyq#BD zb6z~@GKbG~zM8DISadO=sLY_nD*n%fX33|?38-K(1a0pk=mvYmLyQ@HLM=A(KrEwd_4x?x z>r2-G^zg2s()>PXhqyA`{kzK#!m=9(bDzAyPzT`+=^e6gJkWz5Qw=ks8&1+)&46Ek zzC~PG{lACoXrUNsCJPHVIM?u@cmVx_w8h+?qr5p!fG8kToBZ#JUqcUs?D7xQ2xoFL zEl;cvpWMS?EWK=BUt&t$SpeVj#UhmxS2I?U{8rJsMxyxEzV)j_#bax2^yb1-eHc$7 zq!%sRSp-<8({@<`^)2}RNl{2x!aD;0mACI|n{SudW?Jv-u+fW0lgMWhxyf~&Bqv9@N{Oic z^}HsUXt8;|Asr0dec-3SQ~sGuZVB|sC)gM2H=d*6-H&79mt4gZ_V&8gBAE%S4xZOXP(>x@U0(` z|L~c%&h91IJnesj4ba0{$5}~f98kfy$e|0>j?nylGDS#qg}nU#8l1sl0&{?`>PgA# zsSJcG(1t7!Iu3!+8F-9tfxR+)!M=unF*IGcP%tTpuIu@~T5K>2f?uth4a4?gnq4@M zXw`#TzlR1P^KL@E0Hp%De0u=NJ=7!72_$c_as3h+*`gn5{9BGH-wb!{_W&dZtNRN| zCQel^(6Rnw=UC^I(aHiVT!ym+YzlPW)R2_bI32nrs@3nw2Xxul@7nl}DxdgSt+rXu zhq?_Jb?dX!t+@q9$NAxnBPQTTZ{WW+JM4vHwX)uZB`n5`2u-d}2CANuQ`oY7`)N9z zYm(pS&hlOQ@mN!8g4NP$+nZVw(`Z_T$a8re$M(7THD_rwyGD99Y0~cdv3aY%;}=@? zoeR6=#LuaGKE3TROO&72ft2!8oLFa6a4=W7%!Tsus?nu~vo~;IRtp09uaCY7UO*s& z@Dc|g$tWo+>&2ON7Z$Y1i4h|LER1)2|Ins_6<=+!uS1?AhmP@nPaYnN!}{!(!Un2639$ z5fy8hQ%Y~OcN;Rx_op;w=qw#(g-=v-2f7zSDbSd0b8aQ#3a9h&Y-Fa@e0pNtaRhy0 zDNZKtsi>g=GMRexMohla0RaKFIUDp#GTG|0UMyE@RtUBt>l1hJ?eSJ2gopiu9HTxH zcJ0n*;S0pS!6jmPbn@@d7frk)%jEq2(oNa?b8EcASai;~865QRCWldNm$+i}=2ran z_Px07RLE}WjoAIKkG7=dU`Yb<5brIso?7f%7CCa&VWf&e?Yzx{T4(EZTjo{Y^(7Xp zJQDTXdELw`t1z&p;>`C35!9cwWA|_?cZ|wOKBN2Q$9OX6T`ZVUQ4EHlwpe zENb<#oi(muLJ{rjmyhP%qQ(3tPL!ZEy7--VXBL?O^$Sl=kZ3w5QPnqfnv1JpqM7*V zux90rr@Z!mPYw|@9%V;dhbzv6huY3y2-$8XFu^f0}DKuU2a2? z?A|^<-Z5Y;IBxdaW&^+aRchaMx0tZMbGhIPe|MpnDlB9=ArpSlH7!KgzW%%|7T+mG z_L)|=7riZ6Z4#U6$FH;ZgFst{m6NmX&m&#c$X$6#D`PLNt+m{#fE#>7UE$!YS)iAj zTVSPvl}D;!asGI$9YZNxJC}|*seJEVB>s1GD*l00z(Jg2z3;T*s-yh(rYB)5-zSK$=8mz>+=la^ z{1%aTwoL}b?}<`ou%@6acveIN|KIO#{RHANnd=Xb<1?b8VixU0Mt1kNk@0SRfiHxbpTebT7K!U0I#Xzi)S)_@HE_yNT~{gg>k%7M&$Fv~St)Ia^7mtV&8s%6-^h z9|GO0hT*?f{}z0f-`dRF)0kzh^+L^Et4wKF5%|o{wjNZ>yj~yzIc~N~oWbr5o@7u* zt}ZPtojrZ}e50kM<=fz36FXzl6Oh%$?S%MDMrH#yLI&LxzWZe>R(*4A>BerfretQ` z8Xg{Y4=n4e>&&(KW1xp5yuQ*&PK>Q!XowX(K+#NRT&PS4CJDJa~D8|d#pEg;bL=iJ;N zS?ZSs=sD1p0y#4qe(?y&1EfhC8phW*G*IyPmVE$l$6r>uoA~tNi(mq~_w1p_g3ISR z@U&!mH|)FMKNp74Hiwu(z~gv-IqPLrvxSAUC0YFT(@}TFY_UYou!rocrW|@?*OUIQ zUPTaHFC5Jm-riY%P9?YU-Qc8_mNPGQ(4fP4alEUb<9z4^-4|fhuE+H@9w8l_j{^tj zj06M(uF1(||M~q3B&)9N9~hu7Ws&yDgg1V9`3Nb+c>??47aD37XD`fD6(ZE~XQj$X zD|M-FfiN)VK#_IPZF?KM@|D+%w-A_z%wXRmadEdDRJ@HE?EbLc13!F!@0wZwI@F_Z zrc+8q=ebEnwRPiMQ!UVu8Sp+NiDhy>_wpi#I}_=R;5ghcF-iJ!tS|d9iiG>GTfB{5 zD>(Jj=!?9Io}D9nuefKr)u3E4qA-JZB5N}Hh$Z%^#7A@NXY~Gc37*B>xyyK!v<&Cj zH1itIGa;|dwn?04GqJ1aDg~)5x68^y6hI8%cF#YNk6KBcc-^}ALnqNNq6cWR!sRY||s;=*YxB1aRs5Ab*){ z*l>~Hr-$VrrV6@6U050XF}aTUa%G%+GkgBV$bMEFT4;i70lyu@n&%G<+p#f z{>Z%~(y`TF0|~ZekvkjfuJnJxI~g7Sm>HQtA`G*Q>`=V%?oGi?e8g_MS`WA5s$kU( zGT^ibGx^4A-M)R=!_7AWy$SrN6;_Cg=gP}8m+i2nA3^S6$(SlmqD*Ma49aLSgF1+t z`gE~TQRrRYpC|S!$c{w9?Pn~hEyi&$`hR|^iTFwIk7-E9%6l@zu8Y%aS3jA7&qHp= z^X$7|k|)Y+*a!3Iinz_iKub0p4x;TP{;I=eEhjWf$Vm!c!{b>T)wUeAMU$TQd&sj-iFSaTSG03R6!#o3ey-%v2USgr4N5Din4V z->{ijGj17}A&lC%P_)q+u>STfQBlXS9Y!=SYP1_C8lu}5%2ihjqi*S{oK5i06}&t; zxxbe?LY_lpF$62Iy!1t-r{&FP_7m$d$GpynCqjZF)$cp}8u2u<&gSKzZt&Y}&q8M8 zJt%KkI>}dRd2}{O1Qc&?%Jmi3jiz*QkP}*;q#qZU3D_+k!g+3C_C-s9<~^R}P%O`g zke$cu@M(Z>bzOZ18o?S)z3 zM!8>eR68edl~K0iQRZ8F|9{z7cu|V;K+C9nbl?|$JPt3NF|+pOJf(uXd`d^;;}2zQ z{?2P(xQ^m?j@UZ~TrG+ek zx)u!|5+yR(&pj0EU-f7=Z<5WGm>7vLTwQjjS>9E;ntGjrZ%tX1)7i2%NQ`AJP-&?@ z_Y|AC<($%@n^w@Bjz*8BrAFH|o9F#4cdA4mYZ4R)$!#j9JC;&4TZBA(&YG|9xQI(l zjG7-+AFL{SwYHYDtk(KqMJJj=TP1b0{-x=o?B4$JH}2^j3kx2&Y@W!5agV?GaKsDt>BdB;o2AXO=1C^|D}58=N~12uLN$4p*pvrv z+Jr_tyjOLy5~IJzhe_qy&9Il7C5uu{fxn@)_}O}Kbz;!>Y5v+P7U?lFdWhGOqaZtk zXUjAMPtN39c8lZOxwpkd(()UJi>(9lRfD3pJ7x!>YbfR%ndl;jaj4sEvv=<={lSs^ zS9K~qsWUOF&BtJE-pzevr8z5#x$zvIcX{?9U)Nh9V>~UGta07f8*`Kt2f}c4vr^6e zwzO?bQn!^Vy>XS4n7E}6(yi^`%cNl$LVB`Oe_VBJV8aJ7hj=r;_Jgm63 z5PSLf3d2<;uFp>xD9>VVMoif+P0zkaZLx<_F zMy`GHhza$jQ4=)}Wx-j|Fb6RPUVU^Knrq@U0qCXp3@7+p1^h}c2c}Ww@ql#`6_4UAKk%!l_^T`|H!m2XdEByi-f^qM2-F^-_#JtMPgrE>b1L zUGItKx<>HLvegI;Uuj9H1Pf>2R+NXo9a-tDoYuxN&1cIewMB{R>V63BND!LAkqqs!3BTWT93O+__^c>?PqhRxozK0Ga ztj+ezcfNO?J~3yB7A5zTJ}%da$v#N2kYXY-p#;-bG(x^2J9{;W_pH%X%?zQ6gt;aW&57EwgSa@Gf(Xy<=2JByhC}h$qj%=Aw8B!E$zBaC5M-&gj^nX8 zc+;#aC4(H>ox3%J?UUTQ51(pDu6yBan=oscD|&Xuu%j%NZM}+8--p-^UX!Va~pTDnd(ki zAvO$eTp;IzZ<->r$sJ~X$5*Fnw2it?>AQt?cqvVu5z{fcUACb0DXytZj=O^VF^|Nz zi4;w8=YxzQDfxy}l1?Y{n_qPACX>AnO##bwU(Tk!?$s4e$n0Sw4}Bk;F@(!G+ygtt zn?V5VPbP4vzcYUSa#i1c*}K7*D}?7Uq6II{Q`TPg31QuH=Ke=NeOasg)}Lmck|ZE;y*ce-Opq>>i$hA+BkDw}({RQoX~P=pBU#?@ zMdhvA#OJRIb44js7s=NI7{#wHA4;i0|Lsxm_q%Ry1x#KJxLoJ(A5A%^#nZSZdX#-E zNXxkTeE-Paq4IZp-hn7W!_!8TV^0Rx;o;4@YLLp6XV)m%|=2 zgB%N-r_YwyK#`_q?KOfHxLy6Yroqu2;`a{=*?vDi|4`WSSB70HgzrGDJ`uU)vU zEl>Q&5#KVMgoCuS)0Mv|)@*9!bz7iulXk{y2Bp63HEVvbWzU2QdMWzAI<{)sEezi* z{v^uvICHC{z9<)-f2nm#x015=%L-#hD#_VT;DHU+hN$z>ei$~49&>iqnc&D?4|_R3 zx00>to{lv>SOCUqhyk;A&%|BXeM|YQV`4fkpnR3badC^g8Uz^lJ6?NCm1YVvFA;!{&lT=)hzkFXxl9geh1-W>&mCf78<|zuf^omtowPZjG3P-%&Nb2 z9ab2!3F8p&T56i;6|eYrVsGSDbDC_{7<)$cn$r1e#@9w_5{@~T!SSeXNtLb&W5pC& z=*MAAW;PlyBf0>_j4_PlGi{GTMdm-PCWn9P6Syi*);+rFp5D|@o_J2|-ilDk>{?Hi zV8hKu@)HocSku)HMjxN!Y-HeimNrl}{I18NQ)I`ea7>eo6xVi)N_w4KF80utgJ7BA zbtyXMl`|G(u8k%4*p&s0bgVzfD<>Gs!M4fr?Ga1YQ!H@1R+KWh%q+UE-h@ouC72x? zCPZ@O>i5*`LZcqb7@P}xB-bu70|(Ig;qf{p^Zt(Ab+(&bE;G1z0UfC)D@*~e8_Q-x zIaGHDj_BOHXRgS$kBgc2*6@I3<&lOOk>X2TWsvG|H&vwE8tO0MFv3qjwkUHq6g=@Q zhg{)fgL*WVQg-(?uH@D`_Cz|-tUY7Q$zmZ@zdO}$+<5gdBZDFf?kRxDQuROeKzJ7< z*;Nu%r>xaQo{K1`#zTUZSO(6iv2VU(%uJ*gPF#$_&8GbWJMeAzOqVKEw__!+eQaWK zIU3nLi)C46##v77m~In6onLkoHbQz4@d6nNN9X)m z!aZ=|ONpBP#{}=DhK1r>)lT>*XZf`&2Wf?qAzL^tP^h2Gb&$+p?RP=;=@UOB%oY#< z)!ncC^rG80Ey{S~h8lO>N}IvB$E#iaPD}pA^P1smPPcw?HFauYw}xa@U4wjBap?B$ z@xjbMuASHFJ9kzo>kwDx_Nqqih*5$1>057y zeJ2nAA`gL+wF?)VK$-rHKm}|Dv7XClstMy|&RCMoXLIoQPGWKR2 zfP_ysul!-P`*_Ugz6zUAbULcJ=tZ4)g?1(N&aKcx({nghwYEwac<_sw1Pq1_@IQ_81hJXP z4!_#TM-F4u;u4KcmloQoLB$<@?m)l=r9o}y~SGXHfJLgQ}(%_BdtA4ScVf-!l zvtq#-^N#4lTwI5y64tz%-i(|&u{OTF9i78>IIE$!3&x#k?TiDsNebiD=gyPyGg9N- z{%^0nekUJxZlUse;wY3?nFcMQ#S6nK>yztEiRSv)cMNG)wF~8jD_h#|+iN3DJF=_J zhl)Dq;n}jTsI1+ilxz;3V>?jP^@d?OLn{2^%|zdj^HpT0uP#{Uet4exwhgk6^G#ot z1k#Uo+zJ?_kBOF)zGWL7nK%D6P}Tuc?K>*6KUis}fN|@Q;x(f+4OQFdOUNPf2q~z4 z#ot8fnd?h7jZX?spFVYOhjIP;V;k)OaZgDwHW#ob$Y@cD_Bu)8B;R&Rjl+>NEu1l3 zkAmU6e^C?5((XRFtA8<7GqrHLlHSP$zD|BsuKeRy6MdM>Ge2L!O=!wy?bj^~ALW4D zE1@ZvM!)pOBy#dieF>@tf%ksC3FO)hPLRP&(59=i`d8NJtM&vj1L{Y?58RH12`}6a zX~`cc{qfaI^KE--YENkweC=D#AEG5TUaz7A%^TSZG|W^}O@-F8Mlb3+e|nXniIv(D zy~#j>*j{wqv}MdJUh-og&w0owTM4`7wMK9*n(4dOL>(i)>(i4LtTZVVeA4kfYGC<6 zfW^JlrrgNJkzg)$Kd9YBFYgu0*=wFBeJ-r1PZT`30c)?&zB=I$;-<7S_h-^uWigdP zb*I?sC#U{%E_w9_zbl(SF=$A~D|+KHZF4o$ zvP}a1wAxzFZrM~Y-RB{5H&|WBra9Fvi@EGG7$a6L2!XIjwNbdX9s-Eha_vh0*q>+F^oS z=h;cM)l>ZSxTE*WSFECsA6}W&5J}bO_)Jdf_o;2$ya|>$J}K$DLN^YBW26*SCQe+S zrKAe0F(AsR%FNuV8`ljJub-(R;gp9b)3B1^vK`Ql(WPWKK3Fy0KhYk&{Idv3m42M9 zW_3`9;kXB8Pyg0E@lROvzwbQJS;yHwH|dpIP?stJ5xT+qO$c~dOt-OG&e^ZlAE0RD zUCg^JU{&s7SX|V*p`O&WT zP^vXO5iJs&AIf)PkEpv=rPCT+k_)bq}c3m zN&LbKy}k%hXw99l9gkd;_j*tHyiNG((M;#-3AN+^a#Hc71XK z`Z5&u%&=$=4v=a*o{pC9W_)dU;(~=Q!Oemz$z;h^rNc1p zhNaUpQuAh*%yD*VpM5fUWvgZ22Z%I>@_pUHLOl)S;h}V&-RaNiV;E?C^=W%izPi)L zBEoIwvCALkT3$y%Zi^2Vf#wS&@TpcEXBOsoo07g5D824;1m~16jDdFTn^WGCm1a5W z)ehe~pQZ{#MxJxyvf`T{w`YddMO9lG{#K5|WJoUaz4@~X{9oQK{_Agb7U^y(o&&+1uZCLblw z%W(XJHJwLZ#xB4JvWBJ~I`1^&6{eOvDY|@{a-VQ1 z%AW@_I;)^cKeeWI2r@M-@F@zcB%jihJ05j;Mi$?uZuj)S&rP*?`Gg6q`xIPz2*Ip` z;Qy*TMh(?W1sY8C^AMHNR_u>Mf@T}0_5ped6%Cb)ye{{uVqUJ}bC&79~pv_)+`CNgG|QG@3-ed z-$l#s9wr3+qO286(ljP_mLv7L@-qo|lo$_-*#0tii5(YPD%^*)*_CIyQTHbddePs| zkBj2@7Bnw{+Q6u2kn~<1-<`bP!*UUmjcQt*66`y!-DVHRUO6oE<>!Xb$};Lclugq6 zkv>v3(%siy9Z~W8LRb%7*Pep|iN9ER0|OF`diiz}m-7uk?-I~_Up@Wa)85|RKMj>j zwpqO};thWq^tM%K$b0p}WNydA{4sW@i@@?^=brXSH{19152U7`RLZ>_diZmcUXX?T z?jgV9bIyZ4{way&pb^LuOcJVOCQ{*jZ9*NZS_JKCF0uvkpN%YHO^j_86LWY1C$TwC z^Ii*k41G?q0sAXffv~w_HCy_pE)TLtqrE=7#Fo6 zqv+ol-@(-bEmr2A>ZXU^47k|GM<_$op78fKg8dz7DM)m%nTUep15R!9!-cSHa3<6G z)X;#dnj_C3ybZjpQAVc==}#Wz)7a|MJtZ3OjMq?e-5hV5czoCwvv!jpDqy@@TBZCD zZ(PyE7)ahPE-=fa?X^68cN{|>V6s~`SI3?nG(;c9p zIU^+0p>SPR7UoQWf0C2iUCun}Dy)UvjcZ58PX#W|UV`Dxo>#5Oo{;adnTs;I*lAq$ z;nZouoJDzL2WrI#N-*|(%(J*)+#3GU>}D_|WoKT_%yRVCG2sL^k!|fSoc`$l$mC)LcQ>vDs`~GE$ zC2P5LR|Ad*3Y2sDRJ%)?j;ie`^EG)O29KJ&bn%5dyNgi2Y0{-{86IML>$CS0x^xmC zYnH$AC^+z9M_$r<(cJ1EwpOKB8nn*gfi;5-jq#=09|5Mrrk?%%pCbQ=2@es4k#cgS zU*9%o+Yy;FSDj_}petcLsnAFxyh{&Fe3b8R5dfd5Yrhup4BQzc3U_%KNDb~$^3zX* zGRh$pVUxH3QOA(?Fxbxt+tGdXF3gTUurV#mOQzUN9X+X4ju8smFm|4%kM0z^m$vum zGc)StJz5Hpke0|3$l`_L$fkcZK~}RUVEMc9{t6dkF8!}B)3yje4lXiz_Doj(b}&Mx zRgt_?na%@V^byb$Xe2O_aThctsxWNyo|JV}myc8ACQ7$%3O<`cS=YYyX3}HD+}^kc z909Z|6=5us-6h5vGe4S$x~pUQ&Ham^P)||Chut~audrnt61o5%B~osiotyeJuk*eg z5w+y{lZA~zR`1C($CP(X(N*|$y5~5(=%xB;%=xpl?4@ca}2#94AKszpO^E?H|*YHSOD++ zU^#u+o`COtLoE{4>?)K%Je#kh;My-D*y+EV;h`t+&L82Ox6hJ`MAhFk;a^LSr>#cy z(xX>_@}DslWue<^e!K?fAG2U*+*wdyq-kzgN3Cdj^WJ2d4-Z!^&hVWdMVT1oq0PJf zn#M$KK715=Cu6Ul7Q2c@+57EtEozXW)PXaDoj8^>o?u27RvYUNy}}Fq^lm7jg??9N zaQZIsnZcPQ87hharl+OU(C%ekn~{N}Lk7(^+1Lc{EoqGp)1*Fk#(jX{+S6I{pbSl+ z=;obClwcwsLOhzL4t+E|UEqVw@rUMSy?YMau#FhkI9FGItJIfv#lZ|9uL-HW5d znRt-+=#;m)Q}KD127<20!Yzz;FyU5>k8BbX6`hchZ^5kXn*|Gl1|u}6Q}V^kR_1U& zaJn|PXt9v&KIh9Quy48$unu>cak!~+HK)d1@)j29oo0Q^oHrE}{ldb`VcH+4P*pRB z!Ovs@(g1d^>I{S1=S( zBYDyvTdrsrLy<_%b%DqFUF?+w!-t1~1U=x?{)0DKc272WyIfay4L zjs!*tr%?ZR|2eVB`Y*(w+;4c9Z4NSwBQpP|&9F)QHsc0*Doq-!O0OzV7h!^Q_4NM0 zpc)e{>{Xc9|7LFLk)~}9s+Yr=fCXjn8?5ofn}j3{r7?aONH^XT)|OH@-LW%_;dqrDi!nJDzPe zctvUxi8I{%UjW{VngF=`>5Z#7P?`&9O9$H~rYzryJ9eJ%S!9=HK{xZzZ>VM5K@zH{ zR%&t8#rqIXg@P4yTyQQ0f07yn6?RO-Hbtgde}7TF#!34-Pd;jC$DZ~mo}fjW^W^S$ zdzSkYkj2N$c9D%dB_Vb)ssDyia*cDLuZF;*=Xo9>dTR6F>+B1(B$DkWyU@bST_SgOol(L5Fv3fHu|Af8)eDa?JgI!zMfWC;92?R%IC-a(Kr?OcP>PR_*#9G_GnS ziT_w{80lUwvcy%-kN4vo@RsY!X1;~03#B%%XM!-Xt?$)GG;rS><-)X9E*xt)I$qR8 z-G*Ro?YtzU-zWgWWEReUEh#~uw?+oiD8L~4H?R=oBW^!`{+!rGypuS)E$CPt4mV-1 z#*xf_dUBXUrt;FR_;V}1TPj=}!yp@xfd(Z)KBAu72l6=;WNd*Rf1QUUmH#9X4sHejo>31T+g* zOJNwbuCTYl3NFw;cW%rM3l!-y-#C}7`*&{Te)#JS(PfK?X;bcC-U3E;NJ68f<1Nn| zwnB5}QH1D1j^o_OoAtfc*i%;PmA5LFdiIvs6tG3E4x}uJGyAzj98QgDb&f~owx6`k z9SVQE=-$hJN+-xRzBojSUsS{_Q-4DZ8lMsUN;;;aP z=1TU!N1BFHtm7_rR|*Q!C!gshcicSnLvbR9 zy!AlXe1Olu$U~6u70YCxG!88*b&fr86}SF_OM*&ix35#o%HdOf^LHlI78pi|I*+f! ziTFnYh7IIyi8FcC+te={3pn;0bw&@1GFNd39W$V5|0aY2li7TDd~lRBFdL{g{BY zFDDhp?Zg;ZJo;iBLZqAc_b#qyE-zqegHPc2wpnSZS)^54?V57}3vzXjBj5U#@b$j4 z(;fc^7vtFKK-|2mpmqL+C%#+@H|$XKy*;33ZSDheP6W5P`_#e=nwFLtMJ=X@0WlQ@kpz5u;VBL_^*giXoW=HbS4 zE8X@bOpO7`$se8qBuy1P=Ves-%qyzXm2EVFYeVpJ)e$-uTM}7k8{R@mOEa2(BFPj^ zJ4@1BH=C%c@sZfYm6G%Gw4EP|nVa*3Q4@nvtbW?YwDR`>-uqt%com*9u-;u;u3qzu zy?cA(An(vvd)T+0Pgfs(UXK;H?qc4~J{cx0o#|`1{=oMPOjQb}Qx7og?_%~q2ja>` zz!CxQ8XA$Q-Msl4Z(8&PFm!=Np;}W=+Ze@xJt5$u?6omA81u4d3+4p)1KfsB14Z z#?M!Ai1!=!j8D^Odo%ubS?FH*?VEzCYCuG`Ue%%Ba9;7C{Vj96y0WV&&~aH}O+-fmN+mQ#}H48WcC z={{=-y4V*mE(2RUy-rXB;EkzTAq+dQ94a4R1ya}0Nktt?Ykc6vKn87J1*n|9b2rrK zP&uhUdyw8JbxicmLbRdn#%AvE4+~3u{CKkj+usLMk8Nd2Ay@E_e#BMnw5uht&J`{l zi#v{?fVV)Px}5Lf#;lo^#3h7sm9Yl{(+30j)dOeY_3 zAv)CY@nJb;4=7y$zG2dC=NM8HI2+p8;Iy79Nt$5?w zO=gLLUybxc5X&?-E9FK;-l+XAgaGlNpC;XMo)^}&hWGH&k4!MHU=4w7YvRS|^G#H& z5;K<!+b->l!@Bf3$WX~e9Lqt=WLWLCVEzusd zhlW)=N(rS!J*B;9Q7VdzR7X>#lJ-WO_V~ZQ=fvaQ`+v^sbzk>hbo!0Y_cPz`_s2Pl z)9QmAgMwsM^8dDKKfLPMg@+?x#Li;nVrVGv%tZfxEu`2BUnXD0uHU6)f=a>qX3Pw7(YbZ zgouGnbPHTXltXX zw4cfL%Jr)?J6+;PB1+*>1mlJd-3Lc(2S0x$z2N)aFr6YXX9~McUUTfrN}o{wG0+!t zFL&y8oj)NAn!S6juzeut?RJ2ar+fyN%Q`!9=o0LHD2t$N_XFMTANKa|40-qybacp| z8mOBGe&;Rf9%DAOMFuV?@bl0?(p>e;%sTY2qxER9ZKK8VL=>$#r-pL7zk3{e7OwZM zt6{BG7Mi^7slBgr8e$eTOP8W$%xY(m)cFJlS(3_$Ilf~N`v8!qbc@y14<^tE@!RNl zqfcU?Fe9esyE&a`$R+RMlCxmJf`w_p!Rxke-(E`}nV2t`c0ra^m(HlC568I5n||by zEtM)>>|xkDw81C;;@#;RJ`CiUkoTgyIxItw(EBNx){6h+IJ)GUq>7jOOI>f8m%iSF#Pg$*eByXlECwx9m{`?7F$ee zg+C&9qrQrs{ZV4eNu?n_70~pF7jK~RZrj81FJ1?gE)3X9*DB7o)~->D$lJbO{mAEn zlapOl09PEvma}9`ctP!h~Pe6 zjBJs~a`IRGAVH>xM*A52sS_P&9a2vmsLe=;7qCue#j3q`PNU3{MP<2B?$2`{PYylf z{mQPAt^ZYKSlRPiX1#vHY@8_ZTSpm;l$99h(-~%Dnwk#u0w!v3jVjx{D;i6R32+s? z}0?(S@l>bNF+;0@F%)7n6#?u(P#5?<$l(*{)HNWsi-R}`}y9z z`%g0gTt&j--8$g(lF>K%d}icli~A3#eSx0G(tP8u(|BlP$c0%4=ah3&tLbO+zyk(| zHx-xHb#$%OG!OoC#22Ix!vXP==y4vcbH#|8m9_@q`CSD^=IH1iE}_5C(>5d_ipD-( zT&LUTRNc5(mJf~gV>&r|B6_Tb)|F#~ePvE!&0j%a;gFBK0L%e85p2d7)!tgg_4GI0 zCHXKTgRdo?p=rjfIUv?-+TG)lQ8~WxjpE*{(;PtPR5YPEs;AzTFl)m#oA|9xi;Q2T zYauBxA2Je%fxoGk(qi<&ciNS_>I}>>@pqgZ1mug+1q%QjYkCXy4o*M2j|d;=94TZc zC(AFIyPVF)#`PJxOa}rD1XbN`%jfhWMkD$N(qhu79qDtTHD_`i(?~yh`NIdHjN`Za zIML+>KC}-R3LVMShM+aCr#y0a{0Q=!BY3+VsJkkhr!p1s(>*!oWWN=N-`+A&=5FTx z!}C=Gi_;?@ssi=-V!c&tHzW^z8ImxjoMPIPKv6;r|dNUc^E0g?BAf^&KjC4e!9aB+_u5@6A zTe2^FdXdm?k&?c}wrdk|G~oF3RNEReRE!I~%UW*y6`PiE%3|CjtQ8n0%#=<8Bb&8X zE1V61E&o|7A0}58pT|-ILkuyd3#3M=B-F}YGI`vXwny0JC}9s_m{I5do`0I6TJ-tf z;QXOp3z%lI9<011xr!dZC`$aMpU+7c>0n$t!0@i~ z{)}`kN!2Etf;;_x!HpMA_@ny&+>n?YTy(zi+8tY+^lgAtoHq8MdLf?zl2Es2@2_E~ z)Z9S@*tq(X0UuilI5aiRH%;F#FgVl`PKedKw1 z2}6x*#Y#WloKKke9?{Y* zNXA!b(;x$D&*?ma@Mapezc#`wYv;(fr=mH-*B9Jo&*-+n^Zl!9jo#4KGEb$c7}Yi| zU5lZzuvz!lHSjdkqK51=v~M9U<%Dwvs>fsi~!r`*AJm&I{;Ze}#PC^Dt*?`~(&BZ)oF3pHWtz&D`+LkxOw z#)~Z-P&4}R#FaN=Y=Nn}+MXbuvo{8VM^0e&^}z2w5DS1ANJt7QSr&L2^3PZj*FWzK zz&-cksk06Mvy~DrcdUE;;FCuU>4l189n{MlW^4?oq~yFpq*#d10=L}mt$%il?#Tng zetGggX)me9MvDiPYpym<{rGziW(wBX8D6MXuBIo+x(qV47|0pkb4EZ4wfQk}$ug@B z^xYK3n6c=~uPh!s{Uv{6xSh$eW5X@i?o^~~Q&O`X z&gUP=lrH^6|K+a1G{Dlz>VJ{UsS*r}x~BI>e^pl+!6BtOCGG6!IiMXqQoR5u?Fpet zIuDodtYp&~GL<$O%A$=hm+U|$<}*n6TLgT@&o=54L7e8As4cYlP8;a6d<@A~@QR;* zW1RjFD4gyb;dEVwHiY5Pf1|>sO1#PwgkA(Jp1~Rp4fruO@zXKeUA?dV&Z7W!uR3|cAAJ12C|iV&(V z)y~M6ie!}g_xf9HZF>8V=fn?-!V)YcxC)zYfE*O zj}AQjB4yTQ+2pv`rL9-p-y;vNq_Ascbo|0ypUc#zI~e+#rWvrf18~BJr!_A^jqdyT z8Grx!Dw6von6Hpwip(9$%1y~SLifNQAse(%&--n@$YL%{I89vr);NRk?grc}symi? zS9);d=dX!Ty}m(sLCd)00hj2miu98a#bF#(nQ718_cDHyP7rvs9i03AG+pv54E3P; zWL>3Y7mz0iRy0bPHg_TN7k&_UG9w$gK9L^tr(d^m1|p3gii-HkpFVxcYyL&>_tb(~ z0Ca*;J2foQ@#M+dSeRtN=*R{>3A`LUP0HnEWrV*oJpUGk%@#6TFO9U8Q%nmOtz}-; z*2VPOU}HcM13DQ`@!hYd*wNh9whM9viaJ1|wDj#GCC%5Pe@FsQSXVLzTW($uC5ooY zo0LV${s)QN?A?{w%t&3FhI+Ts`}`ZX6#DNK@QjjsWDbr$1Ow>e5N@(~V)sHq%%UGh zMjHRg2Mc9UOFnbC!|r}*m2Plz^Et*4g?eZODCY&U<}6GGsoYSJk)*$zBl<`Vl3;v< z8IISk|GULsvxJd&ZwgCIE$0Rfm41<2_{r&#+btbzY9iD{I-R2obosttGA&KaOfx^c zd&e#Es;0%#Xlil-svpzV#PyERrn|QSd?>f0Z$q@nd#1kr{Xnp&b^J18&PEZxJC6iI z+xU*5d&tDKll-Gp;l?QQYTa4b7V6Sm3`j8fXx`nFX({&Ln1b#Fka`K zRr8omJlf~$>)UP4OkrJrnNIFD-1M){fWmkNtogsaf0>b;28G%_-CaSednSI90%;}X zxK6noShtArw8vzKI^fa!-;m2x zOI)3KM(w~$u#dul1NlzfNTHa21&$Rdz++^8HFXOpscRsE1saNMAQ88qZW;EXW21PV z{@Wu-SVyj=vlU;HCFkD>UI+T3o?B4B>Hsr)4rV~@9{JB1mF$%UI>Vi|5>UcExGe*(4*-(U}WBIDolBe%coZpbB5lvgk+ z^5MNbn|aj({T{!FY$xcPLox$XX2TWNE>K=-|A(iDc*aHtylgW3L(}Kuy^oBqp|G;R z1jIEC;kg8QqeYl+|I%#+(P@zVi!sDJEd4LjWjE!&@JK_sHeQ=WDU$n-8sVR-XZp8^ zs^N81*8BJGKOq+Zw^EbqDwbk9i=r-kNRUE(VfN+Or(Yr=hL()8v2iB5%djq8dg9NK zBDsx(6C0OMv3EyFRn;&WI4R6jA0A_YFaf`-^)a!AXLrdPvzo?EQw(oSKiOGytsalQ z|DL?3LV$y+N@Nn$rV6^-NGWN03Sf7VcxK^i{YX!MM~2?MhV5ldhqchHyY}pX7)r`Q zB1iJ^Xx51L4lxlndJW zW2C&5)ZqGh8?RB=axZ3}C%~0XfjJ+9_;8iWhW{L!rR3PCS|M2k1?n++_{IYl z7E!)hOiK<`X4zRlm4l-3T?5Q^_;Q7RHM0oJTZnGS^@}*0X>c{E)@UVtWA3B z#7}juo2Har;-5j)|JCEC`u?l^9r@o}17b8+q8GF^BPW|0qgfHW7HJAW3@Efh6&sv& zg)e@C6xK{i?e|*7Mdw3+%XD5Eu0rv8m9DONwW>x%ZD;iu8Ce zouK^wS>TQpp+&jB9GV{cd3b)eM8tR9!8l-Gz*@l>QI;MfAzP%*l6Qb2FZO4iVtRyu zN9g=x76fqy+HWpS=zUcr8Q>tw(*E>9X;-dWq40$LyXhPJ(QKjo{z9KScXR_S9sWzT zEOg~Y&hmPgG)B6t4Wxd-g?AaR+B08S4ao5@IWc*wZX7O7e7`63u~W`s~@W-PMuV{@LgfXP)jd9=HD4 zER+Aa$Cj?!wTkk2+R1z&_VEA7NQpDI_K%c@OD$Gd^S0?nJA`Q&kCIn|;bim*7Lp`Q z|CKJmQ-<0cbe(*0K3gNh+%}_f$A`i~8(OE5Lbg>~F=~lV%!a*ikH2KMTH4zBMd;xN zKUj}~>Wg^>Y3$ofXE9?^SsErQ zawlm?X=b(j3KQ8w^(JW?TSEEXBek56xCH$r;eci2d+?aE8E_@4?|-E&MKg0NW>8X9 zan)(@Pzuw8u31J_w*L0dG~Wmjr$N=qI`a(ktR8-u(}xaS$+YV$PMb|N#^uL_hQ5?9 zeE9$Ni7z1giG*YIG1Xb#@su4kD+;i`P$ z(7(H@FMEhRY$@}rL>04meaDXSn7bgV#zN-aDr80~D@^y8oSXnTun-hwAO^^7sYfG& zV_7d$u2}N^{U_{H?lC9W7s{ch*maRV{7Gs<8cFB*8PZDBr<$48G8Q=_kw}jtp)y^K zG>caiG4kuHt1KkMKziz7`!szjRGZP}GvqR!W9hzdEwOZ}DZRv^4gM_LYtmPrO%P}- zzW*k8#kGWCunE^o+IZ>|mablHK+|B47X%6bZY;9 z1HSkFxFtk}LKp^16G+Zk8panVN2_vK3tVz_kPyRtD9Fg%6rCK_o0uHRt@;?i<8WgZ zbtd7XgnEq?S-6A`4-bbHmINONpEY-pVatd65>*H6zlRohuGJfgp@1Ed0muJoG4D(?3ecLp@tDrY_ z0tTVsv1t8%F4yKeQUl=|CZus;z7Z(?f-2XFIHqb}`tjz*Bc}}Wx_dr8XB_IKZm+vJ z{3EuzE=vWz+pjA87>?M{+v{SaHaMqh3zW{CIb*$;izdowKb{#Z>jdRN_spD{9An!W zeAnMUN}%CV+0SQ5xs#5sz2}MtlGcI$hW}|D=uTU)7v2letHb`GxbNu7j9jY8uy}Or z*s*~hZ#Q%e2VNWE98Zi5A?GY+U=Ty=>HTdO zds(j^s}>USM9t@a0|vNW*j|2v^S~XzOxapgfk3akOtK4(VOo-_*#WC#TIA_aB^P%D z7(D!xYE*V5arLn^Q6C;3_aDG*m&ZMFhxYHS^Ao68Z?|7OH2*zQ7C$-EN4?n-f`fzA z4GQNo%^byPTRv!W2{O$C!%f~MIAScKC37NL@#KP^SDb;Pfb zsF7;ui_cWg`bP|;lJzU;NZ{kfHXVX$kXDo#Jfss4|9XbavWoC}vaAKwS$2CtM_Ngy z?c!XC+S#!x+753lt6yH8m1fy;Y`x%xm*1xZ<`|d{#OX{NyDa`vaW4!D$>i>gm_W`J z{IeRUL=JXWhwF@gTkX4|)KrLNnyMeM?cYqILh4}_e|s{QNP zy?ghziPVG1bRTM$Lq@Z1Y-FFsf}7TM>ftN=T8LXAT1F2}y=uI$gM%YX==CW~)_s=& zENuMo!9Mp5Tyn z=ZVYtJ9p^WaBYb}m`7s}I>df0wj^_)eY>@n)fjhEfsN9p4&%6^LhiAX@Iwz>MvdXI&9v_DU9?VciR~Irc);S|1AaOxxg$ju>MeJ-xV8R!ukq0EB~chNo#BVButvJ6 z?lyWgmQYX}a*(BIEsy@^y3^qo1lN5)8ewZ|8@>VfU#iePB9td_&)?q!amlZ6vg6St zXZr@FvuDo^e6cD~+mjM#wq12&n}^tV7^mCm^Eglv2H%Vpgb3M`HhGIBwiNo$?Rg@q zZQpo3Iq7j;djPQ|uj^1^L6k?Bg%qK+Gs88M!Am3*Xdo97G2Mb=z zds+{-mA`wZa&PXGOW`Pd202+aU6n=h%my2jkle67QM1ge$aQr^7fkYz>*VuBO;%#4 z?P+Tu&7DwN@cK30y~Nh7TO-dy*Pfe6>&o@`ZL)zYtfxpsv~cNXQ&Lov_We6NeC37> z8wL-wwzLH3*WAfj)RbBoB~7tKlkum$aHONIE{7|lUIAEq8L1J04qlequ$PDs(t7J^ zd(&CH$~sKT)huDI32dYXyW};3+wLy-QE+LAg?)W_*{=}qUSXo4t>n?-Oz+G?Vj_+M z$vriAwzB9qm%-{H3?4Zv!&o`tgp4>{^o9;`#|2YoLa0OCom9HhmHvI( zzIwAYPQnkBr>YI0sp$2T#ICNY%!c(dAm0P8R5+Tfn}4WARP{!LvETg~>(26+Vw*4K zc881m5%V$@4y_Y&c`?cwEb3fiIk>i?>q0RE**>v94 zT#?N9VtIU4(z%{1MIWt32aIOVysSJGwlX;%X(z3K&0>u)Rs^sLmR{| z1n@E}_|Blq;G!D^Ia?l2gq~AKn5>^Xd2$R~K?wUy256JyY>hvT1=2gmv7QW@j?%69Pa&{lg!ke&L8}Ui?;D-B>Z$JvsKVXB(ccOO zrVM|S()E;*pj^a#`tK{XW8>S?k;{G0>|#Shuu(E@>-QxjtJ-&>OjQBe)| zi^}n;IGgw*@wg1Vw~O)xckL=f9)P{n3TUnNW_Bgs_;_0q&)tc%zm`O%7?ilDYCAYM zWP4xiD19tc#mUL}q-7kvwGF(A%R1BMOGQ`xC~=*%3h5GDw`l8@EvkEP(w6p397QT! zetZ4Fdw1`Kp9~QS@Y{f_33Cdwa(1_Lj1(EtozpV_4DK<=AWE$Y&d3|_EK;V8!K*DKt!m( z&!r)q4-s?KCWrgLJ06RA;Ww{eziYhE{@7ppsb<^t)!J0BAT~Sc&S*Cf8w)`B#`qYv zuG*~{3J0nX!bb0Zr*7rT=cLb39^Z&v@a?dal!nWkdl49;9*lR+)ZF9z^ZoWqwoElU z$drlcl!kajBK5FY<85dk<$pF#Um;VIfVRfEk_)Yf}F@>6bjOdMFxD3=`9<4zHRGP zHBpgo$Wuw0)alGJLsbJ!yMJz?fDxzN6g4$Xf9t+0AHKHi^~J2Znk>5{WP;7MS&}tL zg)P2ST?w>$0ZKwpJ{j8?CW2Ird_ENMx$J3`wv#L}n$#9s6cxD>Jr-R4UhE?3i#&cj zYQW~s&LV*Z-h{EiS_=~_N%`yM&3#%CA&qO@6U)T>1U~HjfNtC5SYL*w9l|3VJT7fN*TD;4-CFqP&!6v?B)5Zv?m{aq zB;@D}H-=v3rZg577AhbkhX}9utxs!GI7p4lrG?2;kNFJo&6v}W@_|g6ZIC^q1whwm zr!#m$-*?kxnh#h-qE>oU8DAH^#dS#LD!%MfBAzaQ&U1!aX8ogY=^riRJ8&g!3pl zBGr?{i>K4<_UCO66&{H?mxoEtCG!)s+59AKj|y3}o=7MZn;N%TFM3H8sIB=zJhumI zh9AR4uym3y@oTefr*dH8)%9kz4lfs;0JD$=m81X*exzD7XqO20dlKhWHtH=h_<-GY z{9@6Aec_qP+S-G`c6dB)6q&;kiiuGV9Y0u3i4DJ6-07T%dQM{Ofq=zBzr8x_ylzYz zg|(AZL?3zUxG_!xB^om{lCTq6SLYXb0?`Rgs7cnnsw-^SES;qhn=y)V%9EAv81i{! z(tfo+tCxze?ET3^NflCV^pl_X2DT-@Y^U*|s%el5%!V)n>jBFM$AR{+rnlZE;5A7| z?#dm0v$q^aML}fSwr%Yvg#GtuEnuC4#)(40i~#xBwXVp28Rjp{Oi$V%BgFnkf5@laiWz7{b()&`9x(XGT4d^$UBr6Q>Bi zd{O2&9HdS0Ayit&3f+|8vppU<^&TiZ4_~1GH-9*h(=ExMY`W_+J<~j6`~9oL?_Qzu zJ2dBspKEV#57k|Vv=Ck0PO@Wy&r<;j+L1l}h4M`)o81D5-OGERql3K*+Bn0BRfELN zU@qES-LRg0xYYB~VZn$GM%Z#mnjb)Rle(kcsZRvwh3o8?wct9^P} zE2lUaIlkUNrG~}zmDhZ8awaXZX?{qFH(BOLdMv6y#lYo1ffms1cl#*PbfYXmL}YwE zziaF7fXfiLei02c_={&@Wd120lt$Md$IGGI1-*X9rv#+E(&?H0YDwRCGLNIqt!8@v-mqrpRmg8iYNknoC34mWXJmt`$${}fL9 z$WzjukWpXSm6V%^QvS&J8oUvtBV|r~*{gqkJV@*cRTU{g+%Gv}?2OcFH1zU=z}Q6B zh$uZ4{-6@rY$`Z_H`tjCj|0oFu!-KZo9<~+f*1t`VG7nk?Ht`kuFm%*5<9uL613k!v-~722aZ4a^iUn9hr9~tL2ADqQmcyJP974yvqPcRMOVoBZE=B0s?jg00)jmq0_06QBA55e#_{1f5}$o_48t%=fbr6qEFA46a9cg*g{P5G z;7j;O9&YerCu{;3%+s)%t73g9UlUZH99k?oyb-NUzu|;s*lc6$8K9PTz_#MYTX=;X zZ&A;sJ=9BHyaMIHyc#R5w(>Qbo;2V{AfdP>tZ~wrIA27nW75)wR1@_I69u!m8=pax ze{I3EGxeFiRI}-Ze{>y!KMN2xW!<8?_-vN69i@e6%@wbmRs@TA@j?@6OBKrLG@CBv zmX;PH^g7ba%hvEP%@8;Fc0mw(sm@0ysHc6v#ES9Zy@STFP4Aa#>l9_o3(&aw)NRgf z!m`Lr-HfgQPsvL~Q+>LH{MZ=L94$H$t*};sD}VZ=P%(EzOe!!C!Bp4|hr|?Lw2#pY zK|{Y~ArHx}ke*#JpG6r~M^g(SHtsE)&Ihutlz!eejo5fejOWRbBT1aO4^* zsaoCn#wdT_&2zaPbD{+4b{$m?8(FSlUfSfb?Dd@-mv=KI)=`R<5|0>o6MJxphr?v# zkZP^S&v%=N--c7}7M+c0_#772b8DQkwNdtvfiC^CMB>l>Ii1*`9k-f|Ee=Q55UZRb z&$mtX%@;Yyf69YDAw5f${1D%z(U^V0VrO2uZ98at>eMMXF_0oq`(r`S-HnUf^ly@` zWf4*aU*$y27&Krz??|!FXDwcfp0i7TP(wSSgCmy5k0j5aq~Ul*a^C-sH?ip&_sRY{ z0Rb^`q9cd#{8PK_#9U7#jSq7Ws(76lx%T9*bewZ-`&-VPJ4>C1>Y`xQm1|!IV1^rf zGOEsh4&}}uXV$F%!vfdOBVg&;OgkH zy||v`a>5)*RczKFcrZl)JRVCHxvjcNo(h*ZiscGKuB=+I_#lhMqX$Fz z04?7mBQzAXYqVcy@^2?Co!q^yUFjOYL@#gFfYTPkzdop73 z1!`q{zX|=M$fb%3xD?GT3(R?Yxz!x&iqOosf@8+Rfa0l(M3r$uU3s`^JXR_0!B$eu z;$wJuUQ_Bi@JV-aa(?n$%cF?Fu&S+nA?rjVb?TFXg}MBYwsW`nN^vGA)<^5C>OBRd zZCYX*fVfg~w{e208#4BVA^X4DaxrK=nHk69k2Y9k5bbbCKAh;fq&L{VRHRStq0?!| zPTvE^pu7vZI9Iu|l}k!W#dF0F zgB`tNRgeAJp4$IZ1$t^lsc5{TGy_I~WecK2sR`isG|g6A7lcnczj!3&B^8%sn?8@% zpGsT5S5ryHM7fG<*;v}Ln^hxFXxiUw;sYHP0JI4cOv29A;(|12n?#M>_<20|bzBW% zn?|q|p=(!(Mx}rEfoQd^^aq{5>JBuD3H0vXW|(Cx+MOCU?;FLC`PbGJ!v)9;_&pIW z3H*uv5r2T^D||9a0j^&im_?&%VggrUuBLUWOvQf?f7z=%qx>$YO;Ekx3|3~>!Y1j{ zz?TE|d~)#~EoG6rCnHuo6bc89flqX=&L=UA=Pol9|=Eb5vubtEU)GoW{hl%z+Cb>dS3hQgmPXJ zuN8(zEDexh)4*A&Y{_bVss-tzfbwMkr+TZ^Ori`N{mMrLKvz}fO6$#>5`R2BjL~XCzjI4mb8UpXNv0$w)%>3xocs86p z>j_vj>TNtAi+Y!TfvwlF#g zJ!f9~4NMm)sYe*-snq!p=os**rze)bV&M`-Vp%4u#pqi%9!*@eiJxB?Lu@KWE3$2H z5LgBNS_dpwjV=cQCFdxrIbDW*3#>g~z(a|#aWG5q<+ZC771ipp1fhktKsD{KP0^LI zdr4N{sI!jMuGGD5iJd2jl`SAdnX&w=u$kx_6`j=?-YR)Af9bI$+js4{g}|Wvo(@bv z^HdQ_Tg~hp^|iGh(&OH*1|S!*Kmt&8R9f@4@oS}r-!>i|h2`5$P+#9%E-TPIp)$!P z^wccmp`+E+%%M&_yEuw5(I-={OmSx^g`^juZG^E$$Cu;g4E>G>wo2^+iYG-a(kvc& zU1~s^%h~|X2ZbnTE)uAii;#6FA*dxK32(4&0N3Td_U3pZs`Qg5@@RJ^L_+-zDId%GtgCA$-pq!t^~8+g64GZNx;uP$sP zdXdwM8g8&>aW?lJhz8?ec8|!twbqAD)m9m5mG;_Ijb^&&r8_hR~dfNn&^x03x+&ad@M;!|i4zQ(nRlL`e6e@rs{n`ShuvcuelyE~PL;{)y zqC>f3cBx;&zM?g&x$(#8WOj{<>9 zL_~=ipXd>^gBh%s(4y#(!>g03u)LMv1Q?$y8$HV$QAu=63^f>}L#(d`OGOHUz6szF z0raju$Ar*c*6~X_8BEB7UEqEm2Z7hFtauJ>1zS%!2ndQl;-32zig8VW8Y%0$+n|d` zWOU&9TKp1wgnt)DdXF{;s z;(`V4DZpIQe-qoH=pKGvsZxY*Y(8(W>CT7@Ws&4OU*gH{7WKCbk(Zfo1dRU zpOb~BQMweX0sNArjxSjD@kFo1nyQ82U+gCZ+{>Zf-VROsZgVlZ>bzR?mF!|ZgRqcY$v~@GwV@D>iLVyU9@{%P7j^NiUU0e~?<2dk1 zN>aVAVB<*%iOVfQ#L0o3O3Wyg^M7L-3!W1^e{{@4Jf@NUw%R)OO`g@3ZhnuUfMql| zMCqOo*Aew%L_E zljqMpJ#E6Y2zt7IeYSbBK$s{N$oNTI$s9G?6I&d1ELly#_@Qf2wX*uH3)*e4Ikp~~ zlX&E^Jf$aS^0(EjwrvJk4icj~`D=_8Bvu5|VxG)qPM0bDp4D`s%fg%Lw2k}RJ0Erb z<*#SBePtk44Heu`>Hl}G{k;LJt=V5?9rsX|lw(!y zKULEfpoFUTd#2d`IGZPEc33N=<;A#n_&{auxO`R4Q`bmlimMf58j~|*K*K}Qhc;;=CfzsvDK^NotdwXRb`Ax+X_rQTYxFU~#5(nLjIU)YYX3Cx`;oj25 zH@+H}EC_Aic`GT2&43SH1JXAs^=}JZ8`vtR0*%u-;`)f&1yR)Z+=S57rKaMeU2j3F z0re=TbHn}Fu4}?8OziaRKkSq~&$mlnFJ{E~qF%D>{*psPWaL(W*@7a>3bRk}A=6zV zG@Wk*qoiglK!r@8*7Q%R14$kq1UkB=xXtn91Sw~G@GI%i`w z8hS9xBl{6F_ysEBukeNu@l=F~W!fLpG)14F!De@dRV}{X&(Y$b!RsXoH*`mT+OTvQ zK(<^qEmln=R(1CLc>O@5+s4JOeKV$Ji}syEz#eZEP#PNTE2TqX*f!ALWxFWd)f zDqTC{Ob%AR=~`Gpk?uqqByc@a6Yv-6ycZ#J2Kpw56H0wn_GFubrj_#I_d?Riu`_1G zM=DyiuErg^tI5=Ji5jWq>aU4Oi#|{*Yh@pjAD|nH1l(contTOp6RAqoE>c%0f_=xw zjHI38V?I^nvvf%!W@F26Xy-7Pg`q_kg+egDaIZm4Ll`IYkKx^UO`-e(IJEsknZ5(i zXLEEs{Nm(>gJ+O@DiBst!>ss{xIAXPAU*qlD z92Sq@x-j(jR?dmS-l)&n!%!gixmvMsAD4(U_me)mg-vV*x75_FBSv!*4(&{kkkoEl zZP#s-ngch%-%O<6w-~|qa#Y0Y&;2Vkuwn759KS%vRhv>>HeC8HksvYM2PU_)@t5SB z?E?{~bl{2pr^ypq^lw9!PoB``GD1_I^e7<+U^J4bWKmtWiK>61y{vG>l><-{dR*7H z$V3b$_Pyfqv}a3mIHofsz7am-+Ykg%nqY-eRch;8PS(ufp3WBJ00USghrD1%`~+Co zrfMaSYma>+WU(aR8$GLR93z{r5W~mL)>T$s^4r zzO1^v&4RD6XzNskk9Hf($ZJq&Dkp+Bf`)wE{p-B}o3H2JZaC?YI#D9fYg#_!TUyu> z`BI@FF@!5AslrrsI#2Sd+@PL;u~_-jUfV3%Xjf9d4-+MN7HOhDn}Q4%SoG+5Pt>1# zZq&&#QecKz*#kE_uP^xFm_7b*kp!Ti9T;Jh-<1g+-3H!*u5O5>E^&`NOQSh`FSClYDNIz{;XIP*b6W|jsCwB^yAO_s#VCVI)G@K? z0kYM3%^T063>CNkRfZ;C{eQf^c%0OlxhyNuCIUbeYa?aQr0KPF<2NAEBWVmlUy){K zNH}p>e21)P@7f)NeM)wPjq1EfYVFLs5=HB5enPZ3^j8<$iO~HaBAbT&b=T9pkqT5jx`J3wSh zw)oP6BANVK6^1)-P3>X^XbRdtO}l1>C?yu!RxKA(|?_< zacTdiv760Z-ErSZ$rtkw6-x&nc8*QbOhsAFt=!~!4Qsdf1ix=N(mJ-(tPEqclS@FT zdO2Nyo$>#tjP@2AokqlYy=?1$57lHW>FY$HiRI`1W!2;Q`yl~)L2#fmE@`Kzs5Su< zqW5c7fWFK9oJ%2Leh}<}GzU?%NHgz(5c5sGpcAxm7w4dsc>X(tMu7@+A;%(_(I}SM zRIGZkaX$dz_{f>J)Px`apHnYF6s=Y$M~~E~&sS7dcV7pIAA+&WE-a!-V{Z$Qd`kj7?V0haz|8ulJgg9r!!xxhPUnS8zogH0hC+C(l6xNYS;6h~ z$H^08>~wYG2S)!x(~mU|3!WQS?EF)^lN!q?&i{aw-N}E|!)&;QJ(Y-4U-%y=0ayVGI2?C-H*`ya!mC2YGrWw;4W*mr60d_JN5I%S0uNeu~Oz(XYbRj>>DYL zA<(<;q6w{LY(c1S;@`!neuD-zwxY%=T>C=1+WDp8n9g#ks%5%;h$toc@9ja4g8Sy7J?dBJ@_0~&!)5jcj5mpAJeR!P{n^qvPR z-9+`099Dds@=g8mnlofae?Xp`5*E9pzY>`^C+$k)fX=_;c+i&7v%lg{2uH!r@~0Dv zqpr?*9vi%)pU8sb?L%Utu;pdP^wvyeB=6C zBEKKw$7d=pO_We-t(~C+a2!1Q!x6}nmaw|S(TD7^WPSZ~24x?(iQz|<%m85{ltwB# zB&xd&?7nM18>SWYJ9XB3<2v6-+{V7o8cB&+M2DM+Wlu%w+0o`up3ZCec&rT~KYa?? zT0By7kP{cOC(K-^9Djug0%Gz8^7#-RAlo~-zKA;XzQ2K&7t)0FR)@RSG(Ob3FFn^1 zeT1MZdTCl$1NYDQqLNhg9A#n4#jB+B9(;7E;irxE`6rK{!vI9d=A9pYitO=Cp{f?u zFYg@F#kq#K13y#8bia>(`2{9Q=z(X`Y`T{x(U8QBQ+bUv;8gKwPtLHxFNaii%|4#ZXH#TTSD>uzpzClu7MnMr_lhp> z{=a8ny5zy8HW{ja!hbZUkH!gC}E|tvNYNVdR(>{t1f~W8))44q8Odw z6FLfbg6~usy;Z%Tv(^F0$55VnG%jC4j{ft(<_V4Kdhz-!vEJ@;kO^P7Q!%t1?Q7xb z>eS(Xs#C*H{||CboCzB2e?#AZOLD<0?5>UTHAv|U<_zbvI(&o^nv6sqp`+@EeLpBfl-%t%zQSME*f#^soZ+^iVcQZ4uU|(b!LZ zmGRZzwwSvjK1XZ@MZBr;B3;wA=EXmLUk*6G-vqH#LGpfjAiLaw=F%_Hn<$zzq_5|= z%@bXKId#=q5amvc=uPy=XC^xekd*})d%fwj#~;^6XBR-e+XD*lpx`cYwLR&YIl9_R z_+qT%2v&y5=dEAQ*PO_%d;LOZZ6RL!Xjw(>1HU&yAr(^-C~aj6??S1|Dh5(lE}izr zog{k-Ow4RVx|#}dB-I4&=`G4VSe&eV)<>M^l9xj!f;psOTzvt}{?3ZxAMx647xXRe zbh?B{X3x2)l@*;Ykn7Sgk2V&HiAKfP*vL^67oWc&?XaRWlTXKrb9hJ_JG$7$Cs(o6 z4d_T>c5{OaXDDv`C3H6rN%rK-bb#}a-2Fz|QvW8>MF6}ARIJ{s9Z#Pb&7$z`mnxqs zsjqu0rPSasSN$uj=Zk6oZWR9y@kC&gYdr@Ce zX}s3Zymw9c%C^LmF+;hkzZ0WNF4avL+K7ovq*=(xuGdQHXvnMRjRM}B=J>7aOGsO2 zT#iT0sLlmYb4 z6P!1{zQ4_L591#UoZ1#nzYaI~rR&%C2<+kFx{pa~wJColFXP)cMj(`cFzVTeTu0D0^Sg}i)ACpZ#2|(8UFV~I3L1J1n$Q*abpF@X zT<2co7?h#1x9qr#f^}o}+VJCyzj2<&C^MdYMCX2@pt`#HC|wc{-*jzZo?v@|vG@8< z3U*$VIeC|U{|wXg&z!KfPHlJwdZ2bvRr^!MRa5doimJ1%HJhQk&Lcl8Fpw2D&gr_^ z_U;^(wX3+N3i|ugCaXPYZ`w46NNA!^R6OX`PSGqd`G!2sthl6VWF0{N*3i|{ za^Whf5W{W|#Lkg+U;5urDEYkCFLrizj<*Ko32wXlzJ1vYg61?%oAWViXhlb4?IxQE zYNJbk8(U}Ex3HL+(Vsk{rENT!m*_uEWuAWNy8zz&Uz@rJ61axy?o}be4yQ5mnNfIq zeLQt)XllYW^Ao@5nQEs*KF2-!p2g)Z`U4#FFLcwMX6rHa0&vu?mdxg(-t$${&ySQj zavJ@$Cs;)9_mwm9Q3}%^Cd`+)oAH=KYnK*C=$v_?ikB9FvHP6eV}&>MgeE#pP3ZZU zA4q$9yFDiBsELkl&LkS69qGC}`lkx5Ftpuu}VUgV;U^g`}v}@C* zzstrJ4vJK*RjFJWuPhS?WMf`_{y|L@C+{+x?r?8Wyy6(Vl5#5bYlbjJVBnkk?wRn-w{+LZ2!VK;ku#a&JT~xp zxtt-kGk|3&FlueB4hezC69IqYy4Py`6=rN~JWzbF&w4}@Ahvq_HEqXOSXjt_8|G;J z#3WYedXpdBS8m248o7qReSl zW;^{d-6p*lo-Xy}e;stUuo_3PR1n74W5UD2B;d7`*d;1jPj}|(v_hT9(?Iw1Qu%>c zH>`3bl&FaUJSG~mSb2GQJ5;RkRxd;uH$1nBYIv5Dqm2v&q`%`-+E7nz6oT`Pty|wv zt$1|vn7FhudC%1M*dF&41=VboccYg;Jr!8=ikSY2<%#2gsZvrolX%eUM_JRG@x?f7 z+SKzHw)&KHlmjoxcWL;7*0T!|Ae)`SOqQB-TvGBX;Ygt2V;kZDH6+4E`^tCKS&kR& z;$fxrdnpvHS2(5^Ybk;h+3=#mxWdA9Oamb{{5VtQ^=ebU!j@71LkOoP_x$Q&N9)FT zDS^6GF_}$|+Vj-B8-5Bre3bOpdN#hP76CrPciiEN84+s@BR`|0Zu-aAc9F1Tg!jzF z3S1Frs?{xHBJeJ-rtwy7hnAHh;1aro1vg9%X24@~mYZm*4S{K}a_Q2r?GNn_5xUDu z6PZ#_+WCNx#`tNitvUGc>#TEStq3d>AfB-2R&8^F^Wp4#7BySH9m%BsYq0cA&V9b6 z18^Lcnze0!a}O|ALQ60E@4x?sp4DS1c84v7Dj^^tAPEwf`&9klVu3FgBwuPCZvcwx z+5$+@07{hW@kv_8<05c*yI0-%v~^}(afkKJZQG8XjN<+2e{1}G);i+F zYYR1nj9@9hq^fHjH;-;$2v~8y6Y$(aH}gXQc(VD8--CAT+QmZ;29h8_ah>S}I=2Fq z`a~O zejH@(y->BXoU3i3$b@!YjJbxMrBWNF^AP_3&O<__D$2{F!@^!!;2j*kYSC;kV(SYM6xNZunDa+-GA({NU%MTsKoX zq`b33=?VFG7CknH$~EnEB{ym40iB?+*_Kc{nGf{4+sQ7|)jornpwk{qbul%K*LLi9 zyvl#(4zQ#3$;F_#<9ysOKTv>_gl|i+tOhvHon2joi?njZ3Z|ZK z((tB#6t6!m?uy6oiPXHZ%#;9 zu>;Vd5Q+NSXK}^r#tnJAwT3S+MU90unnu)6iS;f;nVFfbOO!OG;CzArJ!N8PI;XF{ ztYC)FMkJenrvm`5@bW3Mfyik7+&+@np%~J)rf(>rgwDNagNy>KQzKH-%87q0E>^yD z=>;j?+_c+2e3v+Wh-2Xe{GFQ+yZzV8p#m25K$xn7qSf#P4`$`cl@%>ES`G>^<&IMK z9+E!xnAkS(QDXr=6J5+T`&Fh!{o>Ex8MWJ(c2i*;lvr9~KovP~rnLS=-p{jPgv zbn2Yv`~Q2L=XssiIj8!}XYTvGT<`0;-g}i^^;SHx>9h4ApBX>2U(UzVe-&q%sm94R zWxMRM9a!p68zkhnh)7JlzUuJy!0_ND*#FQB{;<&C2vIp=kOj|z75B=wCLjIyl$H){ zPKs@nM_am#CG=paXLZiuq<6i1y9(uo>9=JG8Yw6xfkc?SWINIo=KY7ROy1WDWLu1BQ?^ zGv6o)GY=SStj|tIN1oYHsA}i>N>an?WR5A6dmBVl2XeeKW1J>X)RSMK6vioD zsk9x+rhA5Q#r5QcQ8Xz*U#T+tpCX7~ao7B@`%=^*%aTfOqs0psvLA*(UwIot{gH28 zHDXrCQ>*d>F+Jj;x;XGcOoFx_(_ zMRBgd$_0K)!{0#UyJW?R683kRx-3ROG6pwxLaJJPNAO$+Rxo&f$JP@dApxE!82=0m zR3gXx3?tt_2a)`KJ*hNFPt=hX6{Vw~pkU{!e7FDs8x{Z(tE&IdCF7;y=(zQI z;T8$^i0xI#ii4Y8jy&&gx#BwK^^!#5E?D4%vQ^Qb31TWUBy)v0TG%9LRF1m4^!du4+|e%;P;Xvw%<8(HzA~UYCNx!}SP) z<@-00P41`~2MeMFr4_*ewl+4xYsK+R+nsfs_o~U6JL1?pLV|NJn%#(7;8n-vx)ZCA z_>uFWB5QIjb5Urt!o`}t%u+KtDkG{#5v>OkJO1*g=SrhfA#G@wF__hs!y zoK|6E|7c5{KViZI5iv2b)kb#L743|fPLJV`2!IBEqz#8%T)=9?i7#o4@J>MMOZ1yZ zR&pRcSO^X|cF*t8oc6*d*#>%UKxV^y&zxz3=M4j=ud|mmSj$GQiXTlSqE^7qtp5={ zV4jS^w>YL1_vZb~TgTCt54$QeZ(fqEM$iI1k!$#^Lm`x?nAv^!6+9}cZieLM=3@QB zJ$nr$^0}a%ERV6flNfQeA3`c#Bnf39*1?P;Jg%~14>-r9$gYyR_Q`c9O9bsHsa|;DOy>oCcv&hsRNWj?1(cu*}-`V4B$a; z(K=q|GATYf!1g`S(a;m{=+UD9IZ(Ih`NKsEv3Ct)ow3$k>Iq|)Z5Bz}oAe0C1`)g= zS)sMo4jdNBj~`j(053R2gMCGS#!%chzg-kCQ~9B` zf>}YA(K55l?6?>g#54DA&!?hRs-^eZ2D~~@f?jiebSbI2WAEMD0^Nh?ZU`cO=lBIGO*$2NMTp9ru$RX~0WQZ8SSahGgZ(}u`0CZy z7|sttwOoPeFFumB;6{fZc(5AyGnXXFsNOWfjr!+Y>A+0U`a#mGiv=szC1327n{7uLIG z)Ho#EyZ2@Yux4(?$>pZVt2ds~=l#~4DMJqfQO@rVlZi&f5RZQ1gfc4Vz}+gIVKy4d z`3=>iM5o*c+58B@w*4CAXC58$m*c|{&JzU}UKpMXUUg{r zBRBJbLd|oCSYQep#$AD##EZUujdjD{%!;p$jBcAA?|G(2Cm1;fKPo(jCyH`D1^c0F zM}V7fmKYi$%w7jP3TX@@7Eo1Cw@7zHc(0zrWVqRfXmr~*#6EYWs2baKP*ZC%NtiR2j zaxC@shLJY52iC8E6&UNN8k9jKuS3f>e)kW2Ww%ca+TLBNR6Xtfk^Jw`o!d($Gx#+f zYG$;kxW7~X*Zm&57VPp#L9gb)0|C7w&0aac+wM~1XU7`r3^mJ)%KaPIS9#x$=IEjC z=jFZ!FYQx%f1Bux-MVetgjN1Q*RR9Ph5h1FT^&9#x@WR{QY}A+HqJr+sj?}~M8!}&ewA&U^YHAB!T0-8!+!a?#RRdq zCP(mf&r)%I`7^wZBcFQ{`g>T~3Lathv!O=-l=(@i5X&D0^)KZ-C!%QTD*ye-#t8Lk zHmyxpol09zX&ee+)F1KekrW(MW`N+JD#7OT`;5khhSx9rB6u&1@;nXe8dFnKCNcC` z;6IGMIjKyx@!ctraf7|>BU-O^+^`L}aX~UNL~Kcb0FmO(3lbD+76*BO3MVqOF*Y_f zO_Fz@DSEK2NF5xOxCJl?;Nzq0{dlA`J)eVM7}jY*zRJt+O2qdqe&!t^&oz{|P?+Lp^kHW>O_HYb?|*-M(-BaYB@oFiJYrp3lJ&z@f9#5Q zQFqueBZX1l*Ao=D>Z|v-vaju4@|N!hZrQoFeq0?XHjDhOH1#w52{1DNZ#Buf_*R1> zU~l4?c702qG9RU7Rvlj>r5s*JTXr$`S!lF&722@uh~>q_#oVvC zVnh$p9beI1F;Zb5S!-_Yi{_mLFNE^RpG9d_2&~iDI=mERW}ljIA|W<*(_er6r30}m z`Js*fdVI~ps@B-JxQ^#!v$Bga#Bns^Tt?t7b}J%-r*0i zVSd=_?ot%Zn{!2z8(PEO(t5~v;vvjGDE7*%z0SOs$gJRH=Z=mJUR6u}z*#7LHH|iJ z*|Lf;Tz7lxgSpdJ9lzGLIiI)#<41GU(5bsh4t`-l0Zfkg05$$?(l8VwHFIE<#Gbgy zl9_rXtF>#Bdz*!GP3jv#Ue-($un8E~1@SyCHg?6(xHE~Mh6qAX9pgI2t$=q?-_7}$72w?6{Av=7w+yVco?Jk?|j1CD-~EDQs$87rxHeb z?;)Q=Q*7hVsVB|7U{ZeCTBOc2W)VKC*jd+D&CgnbE%fAu>+W5<4)Q=R{JS(^ z{26NXF#Blws{+@k1ZHpk>ZZJ3#hu9~(lNE+v(lFbf#(n_fR_I&GQh0Y37B3M-V>PV~bek)I@S-f_XKI-4+ ziq1kaJ5)T;^g*<)%=AbljyM6+makQ+Jsv3`Yy5WZ-Z1!+G=Z!wV27~h41LvTsd@sO zoSdLu7$T9-Sy-im7E*M29xRmbE0QiCYT?2QR+11q)6!@IBV1q*6)LL-;7^j>JRJkQ zvUl$mMwcGuH1VFAZn0bvs&f2rWPZDdf6iFL`^S{~R;=E9eQ}E3-kBGk?{hv+t-bc5 zZX5h$0I1)ApNyNeNk&ZqvMPo{wqM>oz?=hUU(?P}v*OBPVQlH>$uDk)_E1Y*y?hyF zH4+MeIJ`YMlX-s%-4C@9a~ljQFhM7tb=X@_Rn`-AzRU*3gtuS=f-%>eNAG9eF9eXT zI`gIO=FO$>8d?x^-h_veQ_DoWukg?IW}I!&&z~C_zCT&S`^`1Q5JyN8rj(^Rb3B?B zgvbz#qT|rU0sHD`IbJIT%$s;y&2p>039^)8Mc~DY_FY*Z1`z9tgnmB8*Y>=Izb+&f zA^eML2FouEN5%OSFP>0r9gvJ&Gt}sh;EG6U6tT@hTxA$e!{v`O3M)e3jX ztFSDAkI}pFgKH#JU5{jxS0x<|!l&#x!SDCaeUNu1&%R6(4V6W37$13c{3kfQzto_wL=n6^ig4poO6I zu038i4m=EC>}f~a{IfVc96hh`!W5$z4G8D?`b2fLo%7p!oVcYo7n`ki)7h}3YnoX| z#PXoF68e+ekQb~A*A{kk`pnmjv}dfZxISvr<&lFm3%%TgvDW@E=}ReU7+5p7m+3Wa z4e7D5iy6F96urpJKmQadxS^&xo79XkQ_YW`B0UUPMy*(Ffqn!DxRse=7`KZEp^7QWSKvc9x5+1SIa{w> zG?He<7zCa*Wk)F(l{2!fP2yTCG@R42G(PI<4$i7QEYc=k33uqh4@&CfS*MZCBXP67 zmzO%f=fdmU{vR?8&Hh724>1si8@MJM^13``Z}*rdBV*fEojx7ric2Ln2bSAEdOuIs zHNy+-^@an-oAfvV}t-Tg<7lZtA|I@tCSL@%1S0IMB)8JTbD z%U5_~KV+o%y&-25X_@LEY36?L5lez`E(wvpLT3=VpgUBx+Bi3=B{H4?DY zW|Z@IljN^n0fC|MC*=G8lmT)7G@8Feb>-a&8ap4y4SPNq)iJKqETtWXIQttW?JCI; zfS+e!KwjpuM#{jCwbI#eM@6{hsc;lZ2Q$A}jmzGfq_bhmyt}{<<)2$HXYI5O+jB|ptiYY7NXmr-p<=d3}C8M)@?bFEghtq_(6F)k@jYoxL8j!T( z@(PR4W%y_)#ng>6jQ^fdUUC~1zO9_5Jn{$cFbU$GN6t16a$VsG2Is7Y=%A(bH!i!r za&wg*VjVU~Hs$S$J(?s|(6o^yMRbUGiQO zlz08Hpt+&y)~YY6d!0*!sM)^01FF1|luf7#efAS^-pUZ|%a_P);n&l?mxsCxFg4w! z{|U0QvTl~yrvkp+1OFqknck6J^RB>is!bHvtBZZiRgoK(Z<9I`I8RBBI_`J^bB>1K;5x~PkyZ|U0;Z5LJBMza?V^VTzSLytXKQu2NE!58(%UcrtC zI~H(cw}RJ$srL-=i*|Y_abwk7pCw)>?oNCEkfp2_Id{Rts1cYb3Lt17a;Q9+*CQf| z0{-AXbt{1y!cmiB#1RGkw=;X>&Zd^oQ-=z}VqpJpbmh^hI74t;EhEgB6W>46G~fP0 zKU;H3nsR@DyMbn%p4xFWS=AT48c|}Z6uB%1Z4dG5t2#~%8w#{yY?Zl-LqpfDaZdPh zYo)q<%7Vg;w|oF%)cCt?AP;1DpCC<|`hreH&YFKflAbnpERvIx`=H|$SGE|N6Zy4^ zT-~?hWYY!!+a++z(XOg&b;T>)46pRS1^QtxlS}Oa;Fcshl*9%~tTe1JN^Rju|n&Ga)j_Jk`6Awggd4}| zEc&rgU%wpFCI$2b1A1Mv`4b=<6B}4^Yh~1@d3Rm(y8PgB=n0|Oz9i90e|dMeCfoQX zBB)&gsIN(&v7tuGKhn_Xu`qqb?YX+7svRYLl5UpaN=83o-JYIz@L#**ZY%$-U{zxF zXPO7&h?{=t{cS7-uDB4zE)%rPqc`g6ervfL1jhhA2m;C_0w~*tq3jG!O{hocE@!Nw{Zuc8#cJJET^@5@8cg_FmMWnD(`1 z^>iFVNpSxB`Au84>|D2QxRS8nY#dKuS6~s@fc)o`U#Rrfw(M!Ig1Wt8%`qy+(&A*> zu2&9*u}50CiWwi@wYuW=z|rx1SD_~WyloiC1FX6d6jVBs`995YzvlfPn}FYT{%Ahv z4w{5+iCDNssm8k6$WVt1pSn~jsx>E^SDT9qquwObi&5pj)SKP%{Ce}lS#Lh$MCynz zhr+oD)EL-}k6@f?9}YKWzZ3%;9NfjplP2Zrkc^3m8ME?D1q0P^UW3+@F~+w<-%7xuFY_5ErGhY^2@$%fFHF^@qf(=xb7T0Q)ACY zY|Nn<9$n8L*F1ygF8J`pYk0l zhXNBjaQHkrJQM9k$MZ(>Szg&&+aF(#2U28gRsp}LMvcLAa!cDkc*!9LlSO-adLX?M z3pldFqq@4x_8 zq2x_bkyv+Xg&^HyCJz>~3DNoOJXmt*`oOjroq?=>%nv)7W}3;u|B{laS{>>O9mBrIZN+wr&$ zxLIr=-uv3ovG>l8BRyAsOtrIhQDc0hKRTiqioY=OfQLsCeEZ@}QWI)EhDob+~lv6~hP5n|AWn_&eE`c<@BzK+~EfIcxtR-Sp0b`a^F!0?quFBR;8 zVgwI1~Tb&d$!oOP7x2dJG%` zN^bpJdq=0~n`y1a9S?al_`R^;i5!YdFgE?l1N8oZHZ%ic`OrwtNqiK%5<(Sr0T~L2 z61)q^ri;VfUq6PZdomOqYO51kOG`?yROK!no&4$8T~y5x)g7#nJ;Z~AaqU|IP$Kq5 zcQvD?)(IA&=Cp6dS{MxK@L2K2$Fq^3$M8-F#>a2py}OI0S8E@zCD37G?F#I~w(Z{^ z2^ssr)G8CxL6lgv@M!>c`#MBn5BXhi_#?$&J81vn3b+&i{N?~a#aN^LRU!hwpZ@-< zz(nYO1{tiq^BD3UaFr2#D^@#x@+Dn4vT{;fM?pPQ9Pn`CW#u6G%IE*^F?(APJJnn& ztkoW#%(TDI{~$~Ek|@=85?Nz@_MRBXJaC8*R3ZlZ8c*7vlRBgrig98h>L_$gc5pmI z+ZaI!`5E>KX4cO z3=((^!FfTnb%CKh^bug4Y$rrFZEg8RXB8kZccCVv9YiN)tsACCo?bq^Z5%yi5g_9MZeT|uEV-+y z%aMfPBn^0uEULhwR^%{%-2UoIIa*ea$leqLBCUv!L6XMgti~$U3A2sl#6EqIuas~m z7!X0Vz-R#*OlIK)^SO8h_#`$*j*xwsUtB#dw3H}dO=s{m0sB|n`pM}LUP?Ey&DN=7 z3{A@MLyiy+hM?9K@(;Y{bkxlUf5VD^=2W!V#T&)}L%}-0yWrN^v(L=IqwWHG6D|K7 zx<@G(edOyV9f!dR!)AuVH#HHuLu}9e?FpbZ{&@G7M8$)uR^l-UN5aKH=c`nTJ+2p6 z0ki<6-JBFlWGMuS^bwxYRc{pzk=d7aW6|W!I=@xT4JQ7C7O&#V$I!)31L5h*OHWS+ z_NT*7LKO2LVh;Q(K|DlhfuIJdj{_dv7K>iUW`mLAX9B}r2w&NhCH{4&*syY*Y$$?^ zfVika<1nT!XDePhULd{?t%$4H+18;Y3j&xBS9!j2IZAnMEbzNRq6lUOmHVc>uumn2 z1U|js1FqRSld}kbCBBFXG~dK+*DtsD69W--q3w?Y2Ma>(HD^A5ECVQTIv;tQJsec` zJ@JOw1Wt*}=ePUpiP(Szr&7@UJ45-nEomKtM{}m34Xep;L!Z?gVw-@{mROk&ouMVD zD=<<85oJ5DLs(kk9ODelrWII~f`f<+G_UFTtm9r;=9@xptxS%PL{npW<;QSGK$f9C ztA`gCS71Pe{2B{RHeMYZUXM7|pfpIb`Ph$h=g%8~pKMbgDl22b7;WuaWa!`E=c8qi zH)NydA{hapVFkK-5rhAbuQY zMt`o5QoS|bQPB<)CHtHa%$6jjb54ETj=&aAuVx3Mph1!$_QOzijBLFjqf3-vkUwn- zU9}dse2XRMg%5`=CmL0L!rEYKPhvU6yl^737CzV+HTu&=c~%0Ag0f>V6lT_7G=l97 z(aTvLs9O#fXeK5^r>_F zeX^PXsR4@tnqCapbqR_sfB&&SHx*M|U||k3USMkEd&jB6`zu`}(r>(b_qFlT>h}T46|{IM49)Qo&0=D@F1l*%O!haWunxhf-$azPzsk=%}ah zp=NRLSWR+91)}2k8NlRFjbf64?+Kx+<#?ufD|WJ>-$*0(Xokb%Z9sds;opW52m@Lk z;vEeo7^!%GpP(Ks_8Z|l6ICHlJpAhsW&?sW&O2){6!$z1Cg=vm^M`rrpk+ig=!|;> zBi~VGjo&Q`dqXQqwu${y?VbBi&04v0h#e3EWfi2OClIdi)S%#A!Rqg5_}r7`4VW*K z3gZ_jEAJsI-8?-r^&`}=Rct0A%&{Oj=6HUTC07eMD(5>1G>~!nOegcN$#l z|DRwf_^NSfIu4wqkAfB!_o4;mB4M~Nh*+}1Y5D%0yfR9dz5Zu0b9%L=HO5;7Ps~oE zUz;obQY#f>1O2Vm@QaW)6JXI)h*(WrDpqL+(}ju2z8)v;Bko8aQ6QzLIS)e z9`L+-!~d3Jac=~v5{#bo2~(G0nsN=V2%!^A1IRrNcPsw%i&~5zS>|=@8T*)!?=n0d ztN6O*c(YCj*FeN#{EeG^t?FUlVIo!9{v1!3v+F)e!KvebHw{e^HB0_8MCD2XMZnPn zh;X<%&2c)cKL7?NpuKjri@=&lXyqUPEkUSgARZO_jr_0zfs%+MFDTWNBSGGK zA4HVeF8Hz|^GArU&FZd(2Ms$?W#XF?NZ3E7G?4pYIt5?vcH9DTfv#x~5=(So*K)1So8XCKa@}OMuawOwhAn)JyJ2Tw>OC)J#8%Rmv`{ zFRIG2D{f;6!Xo+F%7Y_5?tQ4K(S>e{w34{9us-KFwvK5C!`sCvkU?fiTRi+2^A0{B z>7-pVhSlb*N*-PdMfYPzl3-Nk09bI?A^^dry#nfpeIc_6{{#sHcA7O{IND$_kbB>^ z7YevKHOK1$$d!ijkbiYh7vW@(dKh&PzU+gK-lHy}!rwY)Z$iF9`Qwv7C@24f6z=tW zpVi3ID6f$d&~xL9mo6=~94{6=igKQOYNoR&#tVqByuKTWpAGU3lw~LCI+faQlP?G4F&2!#n1rtRg~FFXdvp3krqsLq!a0G4+rV6AwN4oU&?&pGVh!d ziYwuIP3iJl%R<39#+{pFuP1>;_k6Sv?;Lop{{NcVNd z)X9^FY#Z2zP|-z$mGWEE{COhqh6E>q7d1cjPGUm-t1E+G%dVfRHYv!e3S`ph}1czf%1G7tR8{8)B^6{aPREjn zfglUeymIQkEnKr^&1}+`5ZM>J9sb7o!P6R$!+MhYT>K2k&h z69e$E0$D*?f+jvmQZfc?e;c4FTG9(QFB`?KL5zJ2O=Y;lU9sJS@sEE+%1JFO13)=| zyN70Z8fU<;sl`&i^7e$vAzGW9=K;X4jWJUsm3i%b^78G- zs&G~s7Sak}Eluf0eo44?BBuTbJkGPVGRQO-r3}En3R9;5w7;ay7w3rN*{A*=oePyo zdv)M4fH3q@B;q+=`t-ni~K6@^q(ec#4%>F4}kY__!cw(ln=6H8gV6Z^LKNHeo zW}|T=DzCdgg60a||2<#d;n|Mxk`{P@(A52xjZZj>`}unY#*3{W893P!>J>r7n(HS< zDpuUWc2z1|2d}mtWeQskF9)S0nnqL?Ul2hu{Np5@3t+)pN2x zRuhp8peW+V!aedm;j^PbLLo>GJS@@x894SVtYoVUccL-ZijUGX09c`Nvim+hkk45+ zTTHqTA8h&`iM8;sHc@IBObe#HZgP&z%2Kz!+)d_rLpab1gk;*KM}TCr9bwY@^Zi8) zF{^fusj7t z#T3amC%#DgSyzMv^WeJtk=$@xq4<%Gw-Z^>Fs)=>LGB67XddwhJvxT*tHkJ|XkPxB z@r;4(-@5i;KIKVL%2+ev)^c>(`mJF7F@2-BLrmeH$v`1fI*x@ zEz3qc8aNYv3F3%zdQ^gY{24cvdFH+JNGXc$J?E0oC8(DE4hg|LZ5HQr9%!MZvShxQ z=8ufx(Qf*CM)AaWbGQ?B=92#8*)ulO)mLiljnKRfm9tR%5}LJzXXlk)aV2UrbMl+R z?$fev6}nu`wE< zHSAuWC=avLtiG`|9EvVI8}D;J-e*vdHl~Z9-HgNB#fum<4zcbtcmD@()K&r!p9~35 z0IjmGUQKvjqyWOwS?FLwt3Yrf29^LwR6My!h(H8P*Lz60XD;zk204lCj@bwS(61=i zK8)>>=(xX!J`4>hlAnbEYZe*=&BO^98uty%zSd=GOFVH}m%LG8?BuDed@il$KdOsj zXo37Q^WVR4B5t?}*ig6u%iyL+N*;DT!Cvssz(CN$VqgG?N@{vM&H}Ii^ssVfjN91m zhh70VEqSGk^*fd|qPj$L>ahIU4v#PA3}qi%6g~GZMo&db_9=Q=VThS*pztjxwLO`g zh;gHL>9vdD#sbEedB2Zf>Hw1f4>Zi-P|OVzUTSTO+V6Pd)-j8%_MCU!2xG4{E{^jP& zf9&W%l{-m1?}2fAY+|B}f9ZxWjN?G-*bZTo04{{2@h(^c1$MK>`;i4TxDvlqzv*)% zsDaG~4jkwxNX3XnHlYi(Bp4lJ@)BEX@}^&wok}u93yLCV6lFL)i`d$1mQ#FjNmCt; z1&fGu%|3@D5~v~a0;O^bB_US8jdMTM&Apl1X)t&w;Fxr;V@EYO1R;a3L)FY10Y_$j z>+5UyHaEY*_9}}4A}t?3&;lHV8aK&?C&oF} zP%EeYB}5M;AXd9>3ifvMVJ%X&)p2BjWB`nDhgVCoXD{KojkhC3IGIQHb$>hAD)eD7 z@M<#YAUi~fy0^KfF+z;%)FT)%5pa(@Y@ozrItG9(p~L4Czd-Me0{vbYxkT&>{JMp3 z1wi9l`9bYT@oP1bpoSGCGQ+zxp7uf=b*H;2z#JITB9wL5;NF8+2kPCblo1xGPFILpE|70~dl5;`rqU%ah8-C&2^7>qW!1o3`V*f8%&H z)eQDBFrGf_NCpq{H(#~JqDmw)&SBI)s3+;5m16N)7_FX+O9(jvz68v>p$^M|;AxKX z%w-GH;)>tO9s;VW6HJ<=X?a$UUfCoo2DUUeBjef$n8&l9q2VCTBEx|y`eFO{#%VIX zI86To4m)f_B&-A>`K`^vV9Eo*K=edavY6z;9=>(KJ(4@yyPD{u{n>p6F-ET$PQQVzeNKB{^X^W=g+}Ly8XNpb3dXw0Jr6P?Ig`?^py3- z(vyyUTuWG-9~?L!DsWzSW|#WxNz7NAUk>mCR0&lRj-;Mlzqxn6i`utmlL+PB`U$S_ zs=+Ht0dm7AZWt-6HT6+f^CrNeypQOMg_M~-|DS~BlX^n4p9uyJcpqfD9Cn>{tJQo~ ztC&uzcR|?D<8FI+Hh`M|@fQ0+QKm|t*Ueh@&Jk#7930iqhGM@jPj!A-X!e11Iy!`p zZv4n90FB0H9?g#Axcv)i?)1oK$7pGvl{Axs!>H*Q?TCFVuCKy6R=|kN=NMjH?JqnW zUg`SsWwsJ#9qg${M)wSDJ4U(kEA%MAzR-SCbZ=x2`q3Z<^atdHo}MuhH;=H*a1XA4 zMDr=&-4mD#{>n8&hc=#~PvVa-faE3M3(Oqgw#c4cQG~fSskSlbo99ADe_M-+vaPXi27I=P#4z1zQYOk&7pq#t^}6=qYI| zRmy1cDQK%so%g;ytUP+)e#^y<&O1fXPTnzPAF^kZ#aoC{)&&FuWPF}Hc`{!eh*~P{ zpZWfRaF=o0Q5(bdjWF~PT@d;)C_yA()Lm4aEFodVxk*%8z^%|vC)?MkkQ9K4wrJY& zR9)2Ll|ak@;;#BQlXDF%Qu3kv#3UY}7cjK#`-cY~TL{ry(Vmo&xe1T8f`CFGfGMhq z%`%cpnl6u`=A6q|55Ck$1wYnmhS3LNbh$leO|6=RaoZ2;&Klk2O*$-phi`w_ZpJx*|1O|7{unS;cvX< z_uXpK_e#pQNNmX66t318acr<{7LmbVpu9>xBP6A*wSRi)3MowkGGpR}VMiyPQwY^R zd2bl=1D^~BcYz1bFOHJM@RLx};h!X>7q8gZd=EstMYm%77JZZ6ZGbS^g*dkMsE=H+fxpCo2~KqxEZ#3~omX_j)+rm3*vcQxqJ+>^ z)+skQJrFESLU{m5ce7IXE~-0(N5Zii*op#5AD>Xb<)=r?@!vo5HwvD|h4694mjQml zLLW;G%_S~8XXgr0(-7%iK^}=pU@I_la<9;Zed78$F?R{*f)$DyQ!ytctFiySc5UPF zX%vG7<{zxXz~<|GIX7_l@MEcPIr2#DHx8_~|0whX0hF&&;!HVYqp<^(8cZ4zoCTzA zU_GhnZK#A(Z3lVsU)#$(OU1*>*J1c;tm0`J8vjtf3%68YJZb9~4z>;G>vf9qD&{eO zvtVq)YyQ<0DDJ8Ml*4p%^3dqwrcBIhTJS&7R%;|R{_qiy0wn&4WGOrHGn5sS2Ch7S zV6^(h@BjdhW+BGR&5C*-!-!Zox_x{Fo;?oLalDt*p98H3m6b-<64AUt$b2j=;sRQj zE5X4nXlQvPB8q-NFdf7uKB0U8^%c|JtE?6tbOB6d1fbLd1L3;HvRN4I z?(l#VHZ{GrBK2XhJ?~%-7Kt|%iBX*0BQw`!WF(qoGvzv~ZMW>So?_w!U6c;1bK812_}_?lzw7AX@wYxx}JhC@jB7dj4-XL1pGJ$k>pP zF|GyZf&$=((dtG)kp_Kn%LDJ3x8mhBB7rCnLVU5GkN8%MBT};Z{0R;WY}%nS%MYfT zgdTw3fkz2)rr4!8j)ExxkpMej6!ptIonyPM6hTRN@J8tSebi^a!ic7nchW6q+pC;E zR!e%i-`(Vp$z99rWFc+UNc8UaT=}wezlc{)hlE9Tlfpn-#zz(J26^sh%XH$(g|nwR zWlK{*O4j!+%pO8^on-6wQia>gje2#PJh&eBDXoK`ACSRdtc?IA&C^m zeR1tN_lcPI=dSBA+!O@XyGy+@U#BEEblea|#s)LB@9f{z?Wnye0Ks zj7i~8P2BECEu%C}2vKR9a9?d7?CK~TV6%Z=tZ=h3hAC4E zZ+wG=JEM{$P5Ych3l|Es>YCCz)1q5|dlIQL;p{e_bW+BL{zu7GohCxDCj|RBky&|c zEEty3N9)?I;0A6(mMn;n4Fm`DkG30fRpF(=Qv1%;lDo_+drqv(D3e>Q2RM!FJ5=8< zOcN)_5q}ewZS&ih0x|ZHhaf~$X0%qgFLe{m&K9EkIo69&w74Xd)>HHj4d;P3^8QZg zD1@5MC#(LA(2(;G)x{3yPbaf;bn2@@ecBGa)k_^dd_%~a?)OeAKcuCT-j&^v=XO=U zp?Ab2W=~l~t%T*Gve(09A7*A|x?lg)pi$rT_S{vujpHG4kuC2C^1nhj?9# zDOi79X4WP9s5n|gQ#u$EdACXbsui7=BO`z>y%m}d3+)u1WYKwl69wb|mX?b&G%TQV z6jD-3&O^PweEBvUoj=c=yY&8zf+aw-qW5;=(8p)#+`q4T z=y8WtN_DnJ^p2_pYQ6Gm<>R96w5{HoT3!B0QC7A$x1H75JAJHVPnT9juO4?T4{1 z)vFw8*VH(^nB{Ki3sXNDrMpgRCrYZ7+^ulD`wPH25_5so4tIBVzlrZW)&{9z+U-+q2J#(#yi+rwm=T0u@nLaT zPse$Q;1QzMri)Y#dpWhV=X<(-*3Mq`GA1O@$9~X9df;wZf8aE=sLdB2#psFBj*dlZ*iK&7^>|2#WX~jZA(k2&Zz1ICOA@Ke zE8MUT+I&?b7o7NTf#Lu~)^d8>RuhaCtO`Qzpce_4rE0i<0l9+vA}Eruffj#76jfBK z9fG$^tH@nN7v|w(@R8;Y_^LMD+J7JYEiYuYr&y%+KI(@`g_vESZT_4&j^eD1cgZLB zoeaxh6jPMz3moAaa2Hw}v*gS@bp19JKXM(W&2k=ieQ~vHW}!>NsrWYUudIL--DA(3 zo$oj+TS#rU(XFe(JGXxby7`MNr*cZ6FfD;_R3QAy&&PMd7#a%2VX1XLKRR$AZUSB> z{~_jy9mJVC;Bl81C|v|SBS+WBwFuxU`i`xBUT!xq;_1G}d3~v{Gk4&k49D#ODc?Rz z$GP0^W;xTlS;A6F-%rFK7cN>$cr5Ii0E(V!8)q`)V(k7UIJC)T6jg&!3WOk*%nvux zue~$;5T&7Zf-UN>Ke(;^Y5Gbbr{eP+yHZ>{GvutyPZan3Ro~)*rn9%KVmR%A;Mk5x zh0_HF8KwM{zclkVU_pZ0Q*DVM7o^nBf(qvU!haWDyLkwgAk8i`3B=Hq#-qpLUW5Nl z_+kR-N=AEDRoi%#?O!ilf^=1^1nncwmO~&?Ofn{0+&iWm7pfZ1wzpbP=9Sd7M1^`P1Arj>4LOgp~_n z>FLwLe0mDS+3xFTtS;Z%)ouhkV=$pqlj8XlcV`$YT7PKeVTw5shxNkql9^0= zrCfMTk-SP4)_*w5N|_m7J&EJ=Zktlgt-n6*@60@Reuag`ZitFDUFLPh(K*YIAWNGJ z0-GPrm3CcCn_D~vA9?OiT?N^!{Vl`mCNF);XgRYlBgkkL|Vl=t9k?1`DYuqDp z!5I}ooaqq!D-L?wYGj}SteX4r;*hCq#8pEDv%l?P1`@vO1?lP*^&iRt(*4tUSlH!EjXfAa5 zSbg1am@?k`O72cy-?q_{0ZH3_S_yG^GD9WGher zg`J@V`U(){$^dMOL0O zAlfSB4wUWTr~?1PRGm!8`Wt{_5G2jW`2@xE@MkPKe)khV33#R?zrWscyc5+&`3#32 zGXeucXRTs?tKgQ#j~q8>>|M#;@peB@v+7A+kY7f@iA!Ox`>lrT?@>q;QO&GOqC_H; z_~p1!?q^>e*bdkQ_b9YcL_{R52jpNRI&+Lh*J5BJ)B!NwlOdY$k9wuAuaCBDyDh*k zkeE-XKz={cMx<$@P224{EH3*Dr3lCk&A>iB+^(=ZzXMLfMpxo?4dAKEK<`9uTOeIz@;zZS(2PgkQp zkoN9-z%L6J<5~0m29gc^;6lI#g5H|a0{U>hpn)E4O6JR=Q|0c0TcCUo8C9iPGEx4p zh@?T->TPb-j(%Ak;qBr%;EMhEd|xr46I&Hbmo-j$35P%gOfwbkQoikytVzliNl8hs zd{TB{zQJM3E2g;5Ri%>ZLl*?)ZDB02UeW|DR9SP;HI2T38j0YVubzX2BPJc1H^TFB z5q2;9IoPyPtI8@OrvKja90Q9f;j1hN6=&JtM-PY8X)k-wtqnLd{6XNWHA*V@?5f|< ze24dLj@S2g-+5@@ZXt3i%m}7X8Fvw39Z3P_mniGJSlJT$0@KtSZFm)ef-!ILTh*e0?$@lg;@;U=(xfFOyF=wJG$Yb>kfZimB)dwH?IOi zfH-*uza2qkZu1^p2c(;{2t?!mF9(C;5%YCP5SXXn=Q_{j zL~!^Gt7Kyr3@mKlTw4wsNSTwvD7FM26Z{h=7E&jV&USrjw8CSj4G~u85r8mMycg%dpQ&UHFO|=W^%nMbh zJ5&Vb5P1E_Ay|Wra!8u_q(yoGt8}02o<|EvL(wNJlN>R#SYA5(y_3wUbcf=Mrend41Qn&reZtD9s%O=9U1`yP8#vKqj~qLVc~C)E^R5## z+A9y~=k#F`fb9zaAxh!2MtoASaoz|R^(;nDbkmq`Sc-)AeD(bi`>qs>nB7)_zIZC! zvc^TOkrz}#UU+xsN^>f*2g)46p#x1$>xW^kN_9-40^pW{LDuB7ziz9xrDI^Hu9KC@k3XcY4cX#W9YZ{rPdKa#pDNdicHKCs$y05{T$-M5E% zQAtp6&pUjw`U^eWnR@}o1oq!-AApg|C1~H9XBnLAvw_TLzu27*_U~u5>A*g*_lz5g z4tu5(84@Wj?TPsUB`f>fd1C65(0g@{pb|-$#vp?2NP{UfuLaAO7egf&99(he23Gx2 z;<5VSDqslA)rCVVsb4cxTb*@K9>IBK3cRT?N=z`Zh*QPp07DpwmbN5j*Pg7KNO2AV zB>LT}&tWE_R2sfWhKX!r8A*9UH=z{(M z38$GsdG8J`q62~=?&EOJ*xdr|2!SEc^FszsGhf*7Wc}hQhB@BYu&d~s%G)lz!4Nj} zo!$I_=pwe}x9ky~0r?g!#|#Er9LRDSoroJ!W!r;i#&~MoBZG{RM}rfWNu%h}%Dy=a*>;dk*YyOTlUuw)N5P681E!mQN}z4^4l= ze3!H+g9Gh@`q48fPh$&SG!fWtc!`ZVEC#>9RrvlK(@#j zE_1rk^5XW@;rgUw3{iHn(hPl1NP9QCM9F4_^gYux{Y=riBcj^$JBf{IO)I%d@Tkb_ zcXv_AlzdV_zS;p81sa#P00v*-jeTwTUSdIhk$FwW;pi-h3b9!U09UlUNBN^Xy5rEB z>D9x2;~G9_>BV^`8uIN?qd#`ZbLU_V2|7UbjJ-(eM8QGxY7frtyQ32*Rddr0>lUgL zI#cXAypL}U`z?hzqr<_)KMpY zlwbXCRf>1WHd-nnT2qGx7~F?!fVok`V3(-ckE>K5w7EAY6mRnEmwg}gjCXPlFfZ*j z?u!d2U@QSp+giy7|1)%O=Y?>`gp&byIhEYkD`i~|rhBoE)(pN~p?B?uHD}TCEu|Ho z!0aUYmexwElhsG&151b>xq<)?vPsmV^q&C>5^?6r`;d+#0^v7OQ8b*~v%-4TN+S_@o~F?tjoor#4L z$IxZ65(?K~@eF_21uTs(--e}s6aFh+L@NA{YAhWx?IOLXh|pg7Mi|{8 z;Nm0bIy z^#!khcQ5|$mMa4{kFWA1~3hiJL1m(2bK>jlglxm#9&&JI5fb5 z<$z+xb-s84)xM7=mPh>4F0b)CGSmTCmbX9_?H==0dduapqqT9*ojQsF0GG9q@^#Uz3-tRw_{}yJ3Q?w8O$5={3Rg(8gqRs zPY8^I=Y7MW)9l7G6=g;R`6bDpB$xzwKz1*|0M+HLc9Wz}x*T znUl|%x4c+G{CaTkA)e!@r#?9TIn}U{lGSCv?Hf6xI{i_V4V=fvs95aURrW-!6Ig>c z?&VvsH|U#vAu8l4MTvgsC!Czr@I@Yw<;5YNb3cieDY$U+kp%-w?3#wZTj<0Oncl;ij-g^`~>d%`Cl5;(t%DV2G(|zP@blX++&aNKE%F z2DueNPN^)f3?NxKyE^`DGkE#c+m+HR_e^sbQEmjNi`C&`ccT@+&L)^{K)^2e@V~Kg z3cR5$hzz4)`Ab^@j}~NR$Y>82hAH#xA(lW`+0X1*S>R=JJ?p);^q*9Y3u}cfbL$1? zrAObR`iwzXg;U|~j%wIoC3Rjorr4O6H>t5Fln_PrwNmRK-VLQ6%s3rKTPJS=9n_zDRd7V%xq=U52K`x=Q!A-UepPe zDlsQJ6}Bo<97POTM^Dew)z@z;A<^K!WN!zzX)=7F6}W&zvv z8x&`=7MG~HO-8CN&42T@P+1k-nB{DSBY>)ARUq~lIm7?lzKQB6-XJ~V%{Zww@7@KK z>7(T^W;n;7Rk3tlL@^ITbP3U_$hD$3Z@yf7d>-WH_VkxftqtKeAQftv$+zpHT<7%U%7^##M)i^m8u53U{#e1=&@ zmg%_0SG?COMDeA7OCuMP44Bu_9?={7QtUEPgw6wI`np|KxP%zO+LyJCrMQG0&KI@2Bak@1K&te*;hW$cjnt>yx z-iEdZjyMM~>=OV7^O{PvcTJzQeB0$!-R(LPFpV6p00LvSqnR1;VQRVzSmo%H6uTM1={bvtYY1J*ruph42)Ahm$!#DyZ14O; zf4NDT8s7lAl17jG53?f0K@E6?YHk)}&j!@aJoy>Df{>A>PoJLPu7S5^W?Uif-IGu6 zh|~#aK3`fg2PzQJ>648g9ynioM@Z8z-86yDbIF9`2tXicc!o4!AgKY?X%(phWmLc_ zhkOq0({`Tjoia-R>;{>NPB1zx{8%n*dpkm<>ASr$oX&}o-DIHYqbEe{_;*PQfwqD% zS^z5co;F0b1A)K^h818mdY029WkjFl<&HLJxtde-3?$k`bLY;r%mP*edW2Rgex1Ze!q}ZFKEPhS-81`&&9`}WgU6wf_<@Q6{nq=`C&1_V$5&YC;ELe*yXU568KUc zK-74RkRvTtkC3;{NRJaddMt8g=JdE=Ut(I zfeRH)Y8z--oh=o;Y9EMd06$2Ya&^usQ=sHS?PD6MxQdhu9Z z{jZ}Vys$#zteCW1P*Y2O3(kvBepS_(1yJD4UXA5IWb;IFdMmPBUmp1H+_FTkJD$D9 zZPl45JYD^a8K0`1^T-lBaLpA~ty)!t|6^m>-CF{yHT4T?y4muM74Zinl&IW@xp0#t z?qJ~MPgiyQva3IA1nmd>9Js9x-f>u2l9l^Ji+l`znBRqw783YVQB{ z@q_!;ZIn$)!^ke7orV=jg(yx=Nur^lsnT$}DR;{qiguzsL_?<~Qb_YO)M-fC<7m%Q z|Lc8D-0si!_dk!%v}%r2Hr!IMDSo!hyu`m0 zqb{+RV+nseA}ho4pOjfk%nR&@n{J<( z3S~DdW@+Dm!u>m*!NCgym*#(x!}a7_=s87GU~0T308szRHJEaR=7Kt_3XEP@cUHJ& zAnjxh_~#Oxb-r5}a3ArVCZ->mnq5gH|F^XfdQJ(U>-7;_ODJ0i{SEN^zpk;LO2)mA z)tXyT!4GXnpqOpz2SXN`8DpZX6c$R1QNlzP*=#4rwCn&TM_l?dW+NawzB(;yYwOqY z#Hq2Fs6YOA&tXESr-Mq>`iL~$svVNCD7I+{F3 zaqItB>!LYjK1A^{cf8mfYAiiCW7aFXPF$DJVkxbeGjy0;DKX+#0E2`@$HeSiSNZOf zlf%yX)0IX7@Cne^R+}g0v!BXqI!)r#gy!O9yt&YxI%KCa?NwdQPnCCA(aoJL6 z(_Q_8yOn(a=goOAuqk?0v&wF<FY+I29GNC=_9|CV*=wR@ljB^9@FH@1xP z!q(X??<7x_{{>&tudwAG6ar@_ZXc-YYsmD%eD1l|_l1}-bY7x=3#{^5r)c)d-+6Kn z1#1{!t6B4ArJZi}vvWQvm>6{oM9p*(h5(t8lE7xx@3}U!TZNE7AMB9SL0h+2zPQU-TTqW8p zGl<#-LW;U%97!WYv1+q@?nz;5fhnNj{)S*_tIb_D8mCU77}hg8d61NkUh0ax+Z@M# zZxHR_sIOlPZTQ_5H=Edrjlok*8y8NuiZdB`PLU)v?OIA!xNGT+ zJKP|0h7rec)YoC8v!<u%zev|>e5b^`QB*T+ti;%6B4J7 z?+>|iKyIE{#u@+f)h{JdtcLB(Bk5daCH0uw8rgY4H2TC=sB>Ca<8F!1e{vcmg8cma z%}Kdfa?HAqLiaP`CAb+J9`c9ilM&y*;6vjX9^DD^`$M-XI|IfBZ+CTl75an3mv?20 zdbA~jpPA63nmsw8UN3ID#cn2u{?&4lO^>~GoGp&F#!aYY!c zI{c7!n8f!`kDH3gs2DIxF3K|VAFa^el@$}>G$BpdOcFfSzWAWq_QtB|U~`jh>X~A$ zFX-EU)VD8du%s>-gjtH^2O(kdDA7(9ITGET2!T{{fyDToFf!@1wckZi7@8b#n&BGG z7*Vq69^YkMx&QvurUod}U z_rxfmbQLL#iSZmMHzA<~W2xG<(WhOw!gQO074;BFaicNy%Wuv?w2xzk=lB-MR0CBh_|t;xAb=+*6;S5q{z*{&!O`?A@sBCZi{b<9)?c z%h!q&d#9|Nap81Y)ZE?PifTo>7av@6-r}ZFJQP3tK82kXq!cz;MX?1p*l^e4wQKJb#RNwqqt#xS`0 z`EOT&s@PPW(K?rQA7%l8g8PUvtBZvDIy`m9b$mn9=yX}?{l8g~ zHP4-U3|4_jl#_fYoN2JgBE=nS9;leNBA4q~t#jvM^xob%=PX#%J-nTJaW$PujgCHw zT@x2}8R|Hhv0*AJm^FKRsWhQO=|@byju^TNXuZyM(b)bjYt^ao17=(2yGjhTw(BvS z)MIoUZ1?A{C$dUJo|vyzG2mh8uC{e$Mbpf*9WhIoXl#q|dGv3loUd9nh3cII_^f5N z&6O4|5?xHO$ooAq8G)-q?kkQmXLhF~9&}w$+!;67A|pQKoQoP9FK10qlkgmTMwKNqZ3e>`eM(V9zqxXC2kCX3c$eug`L4;>?%E|? zu5qR_LF_?r1q9P}VyR4ug;YE-1-F>1b4t*LniFi1iDiv8clIcZkJxyRHc&FNLC2TS zlbKZSj4GO2_J-|Myl`0rHHlVSJ9ZT93Blfi$9AN$vp{q)0FtvvYEHULvnL1ryVG1d z&@B{Do{21@aBy|!6uFl>e<&D%{W>BmBy&!asMtQDbOv>q&$JTmlb6@hii1T(6-DpD&x)&EQQyjF;_h9(T>zx8}O`M(=Q@{C_&OrGniwj#}XB~7GP1A zn&1*ak=@B*V|)pt+o8md?Y(_qJ;k=x;H1l%1OIa(cMGmu_<=FHgL^T#jW9^XpBzuh zO%7R6kCP13jj7F9)EnU=jO&@@ttK+fp8RqFOVt*~bv_vcwAiKYGI76m($I<63rVlX zVR`zSJ7UKI|H!pniUS$~Bl_jfMaT7lH)3QSTw~bpziv8g0Wj%>zs>ctXWw?imt^*W z|3eH9;#+caouCrz3^G8q%;EY zPN(-;cN+eQjYh~R5p%2L-qX9(@sbZzGm9s%L;N!%e=DkWKSFbkPAv%xtN5|gvqP@f zJN?&z^@wzfu6#jD0rRZ>J1`jZ6n<6!ruYp{s91=ZFYdbmf)QigdsXuZw@7pDm!(uh zM9~c9Djt^|o94=9el<_ceAT?m2~~W8TL>mhS#S8fg|`o&cUC#WWyX=NyCA;QRom?0 z%$6A?sVF||s|?sPhzZZi_M#ntj!qv^3^JM{;1+~k!yf1VIgz6>;*h;?{w?ntHSAGh z|DL(s??o3wI{e7_?$iy`N=R{5cf#e8%qvZDwV$+bj7YqSZ(`JdG)|l~v~rQA$ke&& zao1&7MPd57s<3X(zOQ}K;>|(kOEM`G`^5ZxbEOmB{(Eta|MXPkV4ojD!lJ3gO}>?$ zy|#RlMCk^PIcrRAyzQJzI*ViqoezyM*%u-;zwx+sRhn;TwN%GY@fhbLz9gN>7hX*4oOvt z?v$GUP)8?m{JXevau1s4j%oLbn2jw8Mkqs=I<#<9^Y^sfRA-?UYGPi=x1_l0HNBuK zu*p)E;v98EFlrdrX_3gQ*J7H;i`dZ`ne^}JN=e1VLFe3UOaD0AhKUd0Vdr%&7h+%9 zC&X!!2{D(VkX+vrBtbY$VD`zM=(#PX%&BUOOjQXdPJf&7=wGYq;Y(_=TY$#Uia73Q4@XME(I-eNE;CYHKt zHnW0kGajTa!LW2#M7;P;ZC!b4W|}Fiw+^8W>Xz@0&m!hwFCz|em2s5|b0}W9RwA0x zKJ4YtYo}s8^}5l)AAJ@aFt%IVVaO22BdSP=<20NkP#+zrkJFuId6ln^vH>Usu4b}r9XcmvWz`cA0ExX>Uic{2v{P)CNtX8*8NUH*w!0=ZfYmTm z@mt5N84C+u==K(%2uNHertRuZ4iD%1JPu2&f5a8W7rWqV{7lsi1h_WYMPLYbG)?z5 zjR{Gm#1J*p)vK{gn6~J)NwqRTVLE2Qg|RxSsBh&T-%>>0idyIWzUEN@vReH<^eQW7 zk6dGU|1_k~Pm}cv$puW7SJfXl^5Iw8S}KXON0Ek^jhGs5`0M@DxPPQ0zyGZo__@nu zuW;9;K07h9+t^jJG*Hu=8)MZE6VrU4C)Qh+k^#m$u6Gw0o5Yx>8}ho@MSUWTVTU!8 zy36SkCN+KsoW|`xcMja#3~m!KI@}9Zxdbc7dbx6~-?F7p!V~k1`CjS~CJvcei8q(D zB1hTT^dNzfprofp3Q^izu8)Qhe>}`heOB%*Ra;xjZUkjZz4ZF&ix;O#6=t&hcIjf! zSnPa%X7i_UYOC9}uG*+jnt`2^^?M^?&@=kI#|LA3u$Xq}c2j)FjcrjebFGZL_-z>= zD)UK-IffL0$lq#xmy?~n*sFGwW$B>U&@aUET~+j-U2kiuGtd)QGPfwfKlG&$5_+H zrr6GSr>7czG?a7DcImA<6>W8=*?L?t0!A3b!sGf#tE_&EHw$Z6aQi@4Q*N%1C-|%| z+6}}UCe|yM@8-I&Tj-ujv@sFoeo)h$`l*SMwO`$nFPfsgFj!=j;+B=FOWTQu2l(cG z%XSBtav;!76e>)I<00R-57vr|b{19eF7M z9b1{RNgMfy)VgZG&R2Qj5)<1t&k5s0KSvLv2}C3|tRaSiZS~eP)wGry(w``!=z)P{0Gd~O6R_LN?$SBdObIhAXawdfZhr_Pk zMd5Sn#*J?)CeHg3LEmWm6uEz6ffQx4E7zAGJ;Ty&-qdNW!C`D$T2a|Djxj3jebKJm z^NdL=um4;EUws}v%R03N-TtAUw^H@jTe5#Q~2;vt5lvDg!3hPXeoyw&s)(UwEZEZ;)t1sMQYmXcppyWFaJ|| z82OzoeqAJI>>`a|@?oF>?b#f|U-Qw}6I-96RTNg^=Pf+~mnam-8GUw&$W zKa>kunS=uy&bo&puC`yXI53CI!st1~c{U}yqA|XgjKfegGzRD#(T=-y-A$gy^3~7gx?jT^hevJpL6^jodobsON*AdY;nyj5 zE^uAUgPvKH1`>$%k71mGm(gB7Mk`4N_I8igB-8o#Tl=Z1Y-q=tM9F#5w6?7C_FFxK z^hT(#rVUP>cZ`TcAbSPX0Z3_7i#ct!!8>8D)DfLY;mi;5&sv-{x!Wh+NF+bC#Vk21bB`*%(Lg+qvg1&j*La4YxXkL`o!+5{AyYe}zg zGpf>G(g`RbNMw!cTO#O^R^W+*gR5ahk=3YjcVOvUU%isT@0RW6*iB4+XN>+g?sSa+ zyX@*)!gnF`Z+2vjFu^QLqOtgKWNXOsdfoEHou#aDd?T6XDdW$%J5Cb;sH%N!9p(JY zu0R7MavL`uG)t#l5AfN4`M3$yX8eeYoV7a{3v~4oqQnJ~=PTDUq%b~mM zn4|?UN!adRE*E@1j-4$vUvodpDGFM`DY|;Q5syaLd<_kowohvjO4CIKyI$O$@Hsgz z1%Y14+B-C%d4>jCyalRb3;8Qrl=~6A8_N)#?a{x4d=9WY04d%ZN%GDYlK9pCt|9&w zFwS6+wS1pZb5x}vkPkoc{%15Gv1;LMtPyf9f~+U;Gdzd8yMPgnM2|{@9==VC^iMPJ zbO|Ans_Lrz;Y6H>h7$>2ut_|eUjfBIQ4k==^~H6tDGspfReu6Om3TB*`(XuwW3KnT zw%XGPub^8*hR?Qmb;q}^^u^8PZ29fhe&ifdlnv}3kQX2}<^(9Afj*wNm5>|@5Bdmv zj>7>k`Jm4Dx@PPq*VkDWJ29T1CCQyH4L^#L_pSeW5c=L$0xmm!;zXIDXnT9QPQp)< zut(+*NG<{YfwvU{fiGwz>13ZKK+(}jCebaiDV+T%))w&ohc>}i6Iv1Ubhz5iXzG{9 zeZ3^bN_>CNkhVj2mBAuYe`y8x6>HYCqoHNV3T4rD{Mkci74si{{6mMJlJ`2J0IvDB zkcR#=`6;oW9zot(z9$63e89RP;93>7;aA+zJ%FM1POa9K!ex@9Z4>#rifO}YU!L^Z>|jc*Vt(n_<;S{Jg-nypPASju_JNdm~Uc6 zc+$iWa}DKN&hv6Wpw|KodM5Sn97-`uDI6Hhe&}GQeb}@YZ!d0KU(@`lPcF05Pdhh` zH^S8vb7W{L znvv|&mOTC~->7%OV>q*9_X?$>z8~#+wW^ZaW{w7pF7%nt#aDKrH`LiOazxxAhxn+1 z{WZsYMLOHe23mA;OmZ6=G{Xn@Txfl!t0L_WD&?De5pJsHoR?Hyrp+O7eY!W1V&jt} zpKwhsma+TqZ}yi)rBVt^cnwY7X01@A-AKexBLbCG1t1wSKG<#ot3NDqP?W=+f`d7n z8<3=~keEifgEs2Z$fb+`sSeM#n)r`b2=r3)wjv^I%+IVd0=$S0xVWB6*Rw$v0vo=U z!_|TTi;tayA`AS4wTWs9HXKN&Ol*y$8ST%<8Ij4q7qwN+pF@6==j0~d9x_w$p)AJT zPhigmox#!O1Eb%P=Ygm6TAaD4-OPckVObZXc?!GVgd?sC6W(B<08!laE0QPI3h5G`&)_KUaEq zC@9)K=v36q^G4SL8l}Zu@48cs%Vmnj{?-cd-1_;XRb$_DW9iPq7h|`_dHMuamP_<> zaf^$bl-|Mbx5+@wMdE9_&B2nU$jYg9(_;M;ND~G$spTm}UtKps)gb+Yd)|ImFwAM3vUx-SQR41iA_mK(YSMsBv!np`Q^u;>3C;L zZKZt4T$KtOwg6G<$m%2!KfNFi!(eQp#(-U<(kimlTA62OJgPf$=zFY1-`rC)8OL2v z&Fx;0pp(`34^yDt?(7oJiP7jcQobozF5_qq6D-|0A3NE z5pCDG2<%Tz-2Tw5{Xs;og#_ouwco4-QI{fSO(5O_;$ccSrTq%I!^T7u-x#+{9OLYA zo6HH6gR&wW$vpAO_xt$q7?3BGEtW{!E{MMO37q5p84Ed)Cq6137gxB~C#iZ~#B_Ff z;1{i3I_8}ZmU-O|*O>TVx^|h{F)}UoaCeH_cb&k#!C^1@(0%%zCks`A7?(O8L^cio zg@re>$WT`cs?OKGEIuw>?(3tWa{HR`CUqX8wL5B0^PF^TiP%`>=X!LBx?O`*N8M$C zE85c7nsE)6XZW?WPdl1YbMtE2?=9=+5f@2r9M4QYv-OH*PC$fk(h=K`lC$OU69~yJ z-YxCH7}#%`duwyQHA>Nw9xT4qqwKHGnOW%9W9y3q%IQ%6&OTl2W*pB(!olZ;=vobaeB zyN6@-(0@3nH|u1}@`s`h&k46t-{-MO8IJ2jEwAI%Tb!l)a>&z}Zrv|J{K;)=3#1K- zuUDt-{U9z0x$R}2xCV8bD&_3VgahWLje5Mxjr&?EEA-YWcQvH@*%lACtv)3!n?mSNB*vB`E*`J`yq$-wq1iu{Avz=h&)xSy^pdcW0z;wb(55;;b?inndv*n zI*j&KJ!NhD@1cw57+%AnC(0WjK^zt3usEx~f1UEjk`hf;%h+^U>s-T)5CwsnM^LPg z$sJ$6urriVmBci=d$}z;npJb%Q+M;9jm5c^s|ibU`&<8^5uw=|ved*+gc4i(@``o= zjYcE#cHj?BEMYI`uM2Q69WM8-@C)0|$jKI~S3rjg&uXW1gR=*C7hP^M?ih9Q3oATw zBzE{yOPuf|;SkPfuO^#0C)Jj2RF{8xC1fmE=#kYP>sVz;y5!Rw-tSvfRV$vxD5o{w zYoKqx#NAF7<31hQs=ns)olC2C2I@QV|+N84fvj(qxT{edVREkza0UJ^QD0x7Mn2bER#8>KC;|&v3Re z3@C|-T`mU|40gGk{m}H-_}rC24|Ai6%BRsj&vLrUqp#ilcwQz`xh8_)vv#?SR7d%? z+{#7f!roLghITzWz!`<1(z^-p94#kdyMCDe#{q#(Gy0XJACY9OqT98vV|v2lofPD# z*4cVNJv~>!bx;C?gW17qgzgcj5_p)YPX9>(0Qm=ttl3@iZ<O>q4v;7{rkD3h2^gsQzj6C(` zM$vT~XwxNGS9tb%TBWUjq)hV_7uP>^tMu;8gB*sq>qTP%gOZ1xvJ%!7{PPqmcX4f8 z60e3AVF&*r6BkV=ez^%Ew;03@0litc*QPVhIdle37Fy>0WJ@LESFr__smUC5@O?C6~0Q%VYojk(&}P+u~u5#di~9wCooz-95+77F>hD@ zypvK2?k_X1-|F21zA(_wrTf>B(n?&C)PSW@F2kWa7!UlxH#6C|HQl@9;B!1?{nwSUV)G#%6%H`erZB;4Wk;HrX`C$JF{oORk~-}pU5=u1z!M~SbDzh1ScTJxr-r0E;MVH>#^pN0W= z)kg<7l1#@u3?3W#s3l6qpY$)PI|PgQ0kT|Bg=S5}#Ln~ROE~nQY+LqFJ#PXg zievaRDEY1u#ONrL+xA{5*^1JQPEw6+2HfoF3E^y3xsH`ebLPhz{wW1cDLn6! zrGF&p1PrEtyx(#&7Y%v;hJA|`u>zc_+n-T83wR|?xMP$wdqMOA&>IsQ`%z41MZsMk zA3IP`de6S=Te4GsK?KKEsp1 zFLKY+Kc;d1opM<4t!PQDEmx#eCp!E^s1-5*Cn@qHV#@5%+Mba-^PUPnvW{|V5XCtSo z*|J3~?pZ3lU5?X|bux+vl^)Gd2=~F;n+ODKY20}Nl(zyLNHkrH#C#QuRY5`atbDy+ z4p&fQYX*tv~abw2`@UMGg@^iXEa=@&7TdtBfIh4ovaJgW)c*1S=90_NE1 z6~m%;ybBJP&frhfHe)85jrEBq43G_Dy2~3!xx}u^#qKumUtDDKrPFg&u`~Frq@dSP zN~Cq@%>o{7EW{X5c^69hzH-(ftaeJwJ{hCF^W+Sp;j4A0>&6WWzK$`SAtCS;AI?e{Oi%R7Bs{)mx8nFv0x7G#>bKBJJL%&QhfmMuxil=Qhf*JD%=#mvV)}-J&v| z(bW*K4foND%BekX(0;nKTz0dw?px+9vOz9+u zITg@H#Z#jXsfDLCw5+@#_jtXn$pZ6X?@h1$B#m?grPz%`hFFgTkEr_)m0MfWZI#k* zEvVLE;F~(1jpKJb>a*n?Fi+q*aOr0tB}Bna7&B#NZ~O}#-ekdIc&eSS77^{w zVwAj8D(~>AbSKOpZ0@?9zS^XJ(V%d909{vt zo!m16#EnR)JCMhq2*DMIn`-3n!BKU{L2Cy&43+PFlS zwZpbOGGBs96M0LL51vsI$a(jtfz{QSeMt7{7v9Wf&0G={%1^~Bdo5RN%S_zTZnr*J zwj^z_9O0|iZ~Q9ULMte#L7tfKqBN7ScP5PPHp$g}e?^o@R1=^rKO3`Llk0)O_x&v4 z!iNgeSMPvhr@nX6izr$$x}*ZpV0Ea%jr}`x>`yK7D<(oluRD2r`m_yYT>ddG?mjsh z(y&2iX2a-h^7E_SMm=wx+~`hakl9$vPX#S{;{ovyQF85zvcLbKL`fiuoU~E$%O`*6ukVMXEcG!E${$lV$*d&ozcBFkg{95W0iP zGoh@R!=e+mzL~S}h_{$3$J{xnHDomG6JT4wFY&BeNL1|hLC!8f$&9Q)Z-TKXvP^Ny zNt>Nd5zZY-Es0qXN-kKgv?P%^Dbx@j`c^>RqQ3KBD)(N!L)n9y5#__qn|xT2tA(hL zmPJIL5mPQDM{M%6VF*gW*(XFb47{U zJOr?b%1B$R(b0j7o8R$^Kaq?GH$HAF5on48reMlLAFWv;j=aQo=&QMocyV`gC;za? zrMwr?obtJKZyH9&s}%nBmJ}DU)Qu*>&@W|kQVYN1Z5L)`HCQ$pPfgb_%*zvI=!XcY9(^F-U2|WbNQ!3d1tfL(;JP zLHWn=a#13zXVt1?mWs=K1rZa4CZO0W7tc4%Z0$jaxut-sGEw+8XaO7`4U99}bfxK3 zo46m0jIjG?*ZiEb1zp5Z4?+?~R-L-nEv#=~-gDPM*74&seNS8V8;!|PAe^d)x`WJH z5)J$mX#BxF9plva_Gba<*Qo|mSg7fo^cPhlj{i&^peJrRGTj~#ou)yAFt6HmcO@}1 z8?3%In65h6TPRv`c;&QolRy*sJg;j8-hupTqTSB`8Fraf25+G4a$XC(hl*B6z*29$ zAX=YHp+fV`-NHHtt+F&y^9HWG+o4YOb8PFcn&>!cBjC zEPANOd!h&de^ zB*oV+fiOkV=AY<>5wB~hF1c73KgADl)(;tVx(r*#(O!dbt9NH~q`i#^$(tB?c@3d*r9^RSn4yWe?2Avz{x%Tdo++aS#`ng(q z@WiWzHIE}XxcI`3gajC$Gpl-HQ^$Wj>G2g>SE0gWxRC>FxfAR%@5N&wVlUo}HdP?X z4MQFz+~T7)Zg8lZw;;nkaUKwa%B**LceZRf7&UZw?CNRprb{b&jJ&!}K1FET86d-V zI|P8-o910l8o}mwT|tCcq;j|yl!U9~M(ZP>1~SHYVK8sfil#Fz+@3!F`~VRG*zFV3 znrF@%XYO3Y*UEph+Uh%#Z~Jx2L~ns;1t|>MNnsnCi+;1^b#R>F+KLBqvcM`Mia91- zv^X&qqU0g5$MWg8im#$zn1$5N^gtJnImvw}G+I*GHl2JLsuL9BcD5kT2vBgzIWJ-s zOe&+At+P&WxK7nJyU;XN&l?c1d)pJ!&$kaC8mK|mqe=iW5kNZ6RXa5&yi*Ik6OHiN z&7g;-UXs!!Z=P%N3O;?07s8|GT)?2FZ4S({MaY}rtNP5ZKY5#isA?g_@ph~9wLR9y z%yV8)kRBOYE70dLs`{<~ib~Ku^A!HG?zw(Ut91DzL=-8#sT4mh@wKDEmB6(tGtHA$ z%YsJ6XSmjJCOEfqta;`NPbShdVQN#g*ysliVV`lrb0A~b>C*~-QaANgJN7g4=Ewu4 zU2lu-P*a=@QgPmTA6(A#Z0{iE*P>|t!L%&?!^Y)SK1Od&SVgVNynx;giaSB5akT~K z@PFARBU6deM0kNZ^s;v$u6{P)Y@xhJ)VyKy6D2+Gqat-d^yrawq+TULT6XMM3L1gY z2`-^ViO-$R{QmdO;fs0$A>3(VNP1zCdDV}SV%c5L-Pq-OFndsqM48=i1<>GUSJaa| zlgKG*ach#yEhs=35Uq6`4CLBs%eNOWep!W3JG|OM`E9aK)OZB=VygX8U2G>FbbiTs zn^t(W#xz=plL-H>%D8{ZXwT*85!>YC)EDU$Y`!)Z%jb9D4l=YwS~K-8dQg{eaB4}L zKRSrQQb<7Nr#;vyv>MyT1$!;{uUDl?DZc4 z{Ch^bM|qC8Rd6-xr)BH&-@V(0sf06Qg$l&-jo-*|@NYn@O!(D^^x+yJXp6j%@)$m# zJ0Uf<+{Ke8tZFb){AWEAo#q5ubx5!62@u<=%@bUporB&eg%;lbI$T}X)Dfg7McGcS zW$^`6{B~aZ!56Sz4r(Rh9fDeEpaU8On_)$gc?_higC~${N8IWJjftmM`ReUHue<@@ zY6M%A#G%B@?DgvO*(Y9=cI!QEi>Y~9Q+C3E73Pnh4NxYB%l|Q^WdVMvW&wCBa%sqB zgo97jv0OJWTH!~JugK;ysWWv7VO|ns%L*>Va z_tDv*di}J?m|asZk@SVwf3@>#{|la@n>XF)tMJRKqkC~70&jf6@kywAQ8w~Q04_bX zkiNW+I%-QBLgek&UO!s6*eIEqdb?jki9SD|Yrg6E#KeH_uepmXk;Yfbx7oHkAX`;w z-=XZcROM-agB*3zQNg3jVWpS7N0g+A6M~p1>v8m6&#|QbK{Dh7#h4g!i!PQtlr*Bw zpP$bmHq|#dRBWCREW*e4X_ifn&`R$X!5%bhesg;^A2-|$qEUd14R{){I|qq++bp0e zu2-a~Fle(^bx@bD9nllzM*q9WWVo;Qy=_g}iaRmc-Gc@RGU|G(LlsEL+dvQpBSPMP z$RXPSR&q=v^7mO)34+=b_u2AWaidy8O+K~*0rscM=IO{(W?}H^%uj$_KieipXYyrU zI6^^*vHNkVy4WiPSz((j@y)6#)c{;C@$j|OqTFU& z@bq$U#8LhzIGb2lG9EceUOQnEQK^$nR=LS7?$w{ijE4DX@LM0XL#sL@_CzVrFH%ke zmeruYi_R&0nmqan$3~o)xP_-aZOFzgV&cFVapONFW(M=h=j#>{u`VFl=-@QRsh*G~ zPQEKCJSeg`d3MD;Q_@e$EC|U}bUZOy_o;m$scgF3ajl};D)e5guRl+8!$Q@jD$9P5 zh?e~20DI`%m^ewW$#^V^5(Z|zHCZ=&IrK)_&<|XNGWxD*%}fv8=@Xk}Wwjx2uMb)r zb`T7VYP2{6<@WNp(8mBXX@!vVqfMb=pwvx`K8?7>N~Yl9QuXmsXE2zBAS%7q;s}@# z+KYWEJ86?)`vQArcnG2p=)(;cup+U!b6C{E-^%VH-&AqAVUpZdKd8IxcfFp)YnLVU zv@Vh4@&2G28WX>^RN`~*W?vq%jC=X%n))*J&P9=8E(ezd?78wM)h72WiV)Nf^y}YZ zVt5n`tZx+^8VEA%xfaViX6TJuY{+orNO3;%kI@YT0o1SC%{_YCZ@RghW@H3U7*0+} zj=m0`CkBX5Xww4}J8~2*yCbCC+Sx6lNuXGx4SpvX`n<|zhKPJ4Mb%QsUmb|Q-6lVN zp;9k<;`rme`-vFjem5dni#G&ul|+WQ*TgbF_hJKD9N00?xKw9O0UR~y%vC)_09A;y zVlL9VC!p)2x!wH1!1{mkk}qH0jC7VS0B?%=6r@JIj^91h6=Fe*Dg$LD@VQMsl||P; zpT@60)QBBnC|HB4Ppma7B6AZ3Q?+d7jse7k&X zfiWqVP;|q2ilaLg!D@y270VuHH_nu8tj2`u zhqwGe3OAyM1f#SEGWx^7S$=rGvYn?^Z+*v`b_jona$ zTc4q(-Gc&STqYxWop8??6FGV5ZJ@WdL;Yrn9`$RCan#A8duzmW z0E!JfB0M}%bqmkRSat1=Y~zmF5BIFi(ApvUIrR;NcDDo?ltwCVV4*nRoUAI#k@ z8re062%MYN3V$*2v2z3IwK9AyVT^o*R961++XnKL*TU&tndx3ts`}1PRUC$_CzgJ> z2T{nuRgA8x)BeIj}*PzOvj%c>+Bw_HOd{MR5Frh-9_-{ z-Nuz$X}K@cuGT*J_N~LEf01H$2aD<&H?KR0H5%9b93?UMYCzi{$7}mLdL7e~j2a*S z!h=MM7sO&wDm`Ug&WYdEs^+6gfu^S3r)#FOCtDlHy>~_le~#MggH4pN1oJdQH;0e`$OK1HOHM+gpe6R{!B5 z-RV<#7erx_z*OD1MV&V4{QaPPQjQK~+$Mo5DU!DYW#b`GRXIfhq3OZ}N6`xG^f`iU z+;{Zo!chg4QCdbFpB*l3_OEi)K{y4^N5yEQlKZzKnDc#O9*fT)Bbv^hsFUV zBS%KT6dMmukzjRoV*QY<{Zo`fy#R8QD2yOhT37PV)AhuZn_xaV>NG_wtcf?YJ_n998smRAF(H-x_OEcpKnLoH z&gmfC9nSfW=BMu$+>++SBmQxS5e-(!R%z*4#O3a6jSwf~&sPby8#)KFc+g>CuA*wZ zY?c2nk9BPJJflklA+wII?uGQ1N36?iuFUSfv8K^3rb2FPLbqlo%_olcd^I?=XgP$z zRP%NFFoJ69_U#quSpfutko#?(z7bBPo=@vu{{F5hn}dC|Hw8ohaQgsbutipaJXpJS zP&qpq9T};f*lIMODmqiz?qrCL`&=-}EiCA&Y)nK!(PInnLQFBdEy&(+BmP1fZO_PQS=8aP7)MI5;ddu~#l zqEoK*Hs#q+q^eTb4QGv4-1xyZ!&HN!)r5AMvbwC3tq$4%sYTOzj6;fk zrmrF5wSY?ke2upuA6PID(HUYe;@_wgO)Mpa(I7wo&YVW+m|t%h7s>o;m)d-0u5@w| zTj`n*JTl1y<@OqNXBdqwk0DN1*$8eTn60L{COd#;vx;`M#{av&Da*hxMBhK8lxOJa z=H`|h!90X?x7*EGx8}wuN`xEKvXl73R7ARCpdl?07Kez}Wrw2%BPVC~6A>wK%%l@H ztrG*HxpW>xCV#KmA?7%8l|KH`Ia>#;`K9lTIDXygj(r?bm?eyjCNy-4bTBlba%0h~ z9kX=+tFeN)o}pg}=+H=#EKXoS(V#F9NC>*Hlnodh&>+;e6w7n!<2f#Fn>xD~jXRZD z{9w0wwK#gl5>!8 zlDP^o(b-TZ%L4+;Qs-bP&6p)K)V*EbRXM>I6kv87;Zbnj7icf*kLg)_4t7H$K&s5crUFe?gR>)v>1TtNdTJr=si` zFfNJFu$Ze$h&T^~pE%SoS0)f&q`|&Dm>77=!oE33Wq+~{{Hi~GyV^de8~jsZiKyua3ZbM)`=?`p@6{SDQIS?^O8EgVavs! z@q-?y%DW%mb`_VKP5-+!fd%N{xRHdT&C>_v7^eN-~L^45+6$d zi1#ItPodZXmgqO@wTAgDUj9u*#b%4oKS8vQ5GddD1)MG!&QAE?T9-=r^(&=ENxa^6i>-29)FleM@dpr{b8p(qO^ zDmL$l^_z;B?5+Z&FWjI{ph(N}pPI{EcDD?mLpui%bFw+`?2lCRV4oq91!{}||0zJa zZvNx;0qVfV9lxwCs6;>4HiesIv9Y$QA^U2@_*iQTL(hKWlhM)9_t1A>c3oEp2r#Vg z{`kz&_b!SeeZw3*rnm{4Rld&DA3Czjuj;OIFY77e3J$pzr}pE-XXLztn?oc*}CA78N^V^>va|2?)Rmb?`eyHc_a;?Hw!t?lh- zgK&RV7DbW~oBf7`@TlO|(mo?9|M?9i{=2G(WOn}0LXgN%H4>k$c>L)9KI#cX+7%DA zbYc_*FYt`AN3yb#svqBS1qIW#~BuKg=HLo!07D$N>9+Dq%9|(x4Udwov`3udSaz zS?aays+J0ms%(0?*ruG^eg1YH*Bu`g#+nwkBz9DFP@79jw_6Wr(1&~4#NDEfNM_{P zgfDIh@;>8KzkwbC?@-N+hU3e3?{o#ARhmU2DD5KFJOGCpEM+sz$T!YBtDp zIW``epvGuxi4_`gx2L?Ea^(_GI5}DO@M-g{L@ghcUDe@jD8Lrl#&>l z`fcfBks37#)xk46WCC#%KrA@P5^TbID+&q{V)VO)`v1QIEq3rma$k!UYNtiH%>UDw2VvJtd|{>LlCr0uU~he!L#NQxVWL>1ew<8^SJ z>%RErI_t~-Rf(Pz@h|FJ-c0&6m>MLwn#3zi>DN21nExg;v?_X=y7ASzT^kn`oeq{t zh|e0PUjMSP^_!$KEdZc)g+fjwz8PQnh{?z`bEO<5Rr?no8e}T?XH4lK@3|#p+ea6= zIH!&C9=P8sH$(QY3z42lU8M{H&_t>@e%NKB60lHbo6STDP@=A%;B&qs$7x1c6{0GF0@tf!JUO0BT8_C zpx46^D*x!98@xKyljUzN-eFlK4ICam!i=)QS5A_v!);tNz55|2s_xSK>~p#&y9!J& z7d@TRZ&UA1g|IhI?5_G3f7DKzW@a7B-ci2CyySjvI^~G(slIiIv&Y?Ulc}?bPh(wj zjqSMO-U0FLd0R?b8=lA%+y zOHRidcYjKzedg-RPq+5{t)p6}esH)}&TP85e_-fUzU??Z#fN$)t?WTzSaRX;4e|BK zf91^!pqNxtMs;&mjJp+@G=k&3F-0v6Guaig*a}A)sL6CM!tQR@%heep8KNL6>sG&$kg9) zcF&eBW=cNwAFAWThKJ@lD~BCIiZaE|>UPziFB(5PUa20qa{AgS?S1o$7u>BnJ5gA3 zb)Ez^Z~OOp7qX7_OJ_?rn6<{O!Zn{$bjj6$k5lW0vJ5-+t;{+Vi!D;sPj9f)4-Aqe zkv2fD@(s;aSfV?2g#+DS+kdzj*Or`SYj(gAWIpiiLu4nb3fx1mKtIQheR1#QWNK^5 zUtZ7>vN;>^UwG=qIO z>4(j8*9<2&2L6-uz5R;z%+mrv7%nIymv~2CEp?76>Z&w8a8VWNBQab$zy6^~Mp=kz zZuDp>PSlx!gj)|n$HRK3$_hheoFI5NO#9(twM4l$%?c<7txfH|6s3Nu)1#IrSn9d$W65JhrkRYLpN`&*-h1@T zmpKEJrkXwS3@sC0N_u44V6krfm;+zAU1GP%g4W|f!=6}n+A|v-ob>d4{(&o&%PNf7 z<{Z^!Z3*s-$jtZarsu{V#oEQ69&bC$dFLA*e90`7&ReieL<9{Oul+cOf|uDrCMiD% z9j9nRz^7O+qX*XO;pmi!U%x@E#>Pe^SZf)L>7b92yH2(B+&#v<@Ab<0*PQ~z#qI^m zEBBdGxclXMx)wNFYRhoTiZsbS(ssyV8`j4Uth3cLUEu)>qd^|yiT}js~ z_f%KjdqmkPTvwx+=PSM4zHfn0lZ?9{rl=nT2e2I3k1$c{2eN>f&uHGlzBGc7KOYtY zbpn40^e!xy8IS3`j@3J98Wm1*rF_w!>Igq+nw#S=O-*`HZlfhXytn9VT}!Ul#Cem5wRl7-i;M+#-IMawJL}dF&Zc1};W>XHh}f!U z@wS_Y2?q8GRQ3cl)p-A&riHUO9ewMFsgRAC+3BKo`Th&}4XT>*!x|JD`c>NJsdpw1 zCqtLWIDOtNR9E<=sgHL>|DaK;h5`*wuBvb3GovX~D%U)uC)?HNd7~llABOc>lB@03 zl?%6F205YdPfAHS{c8xCzw^hDeSqoQwJ&Gv-*#i?iTvf_$18d!R~}ILPLtoRYAlM- zX@|9b-uuV$yPex2hALlpMOu{M#&vY05z$fFM88s;+1ZHQi}s%ScJk8Apa3r6cE(1= zc#YU-Uh?0WPyA_a{Cl4`E$+)QwTNhb&Lg~RIiUb(;gq|Bpdpo+oqHsOS(&*!&}st5 zbT3fSAAKfRima`yu!X(*ipPJ%D9XqUC4RS@%Zl?sq(ZTdB&aMiQNEyvIBN+xO=(|GIL=tmf3}MdL}I*DNbq--4qXX zJA&i@4Hdet%pSS9EG{SUA3bRO`)e@kN^a_NxovQjaZ?|k2pf?$o*dcua4Ys&Re!Cz z0%39jU%eZNi#%8~ZY+}$W=cHL1bQbX2i!ZmX`esc?ogL!=@pV96tox>i@o*vL}F-YxRD zQe@X2kxIjNHtFvwh672i@{#6oYy3OjL)mLbmcJzRcthBkgM8$rO#LsN%NhpH`NBAD zA|5&^1BTlseP%Pxr(Q(2qoa+`X(-Z8jQXSMTGQv z|NlWt+3V(Ndz*!{kpwRjSmo(ZP{V&bZEV6+REY#qgdjAaqwa}pEf@H>?v%w*tDPf zX!X5En~HU_G&5$#FGO@V+7@~Fh1K=X*u7}1b6Y}HL%5mdIWv&HD_wbM(r)IYJ-sIj z!^F3+Q)lE8tm_Ry&d=*z*2n6vMSPcN9Kru2ThYW*6@6yR4B6&`WF^g+HF&akJgF(} zOW^I#Xw9D&p%6!tzlIXHfceH9r)jC^%=MQ4wvLZC< zw$2DvQa##pW7cANB|5g7-V)Z|`%<$e^G5xu><94wtv(G_^H7Rf@r#82M=Wh!e z@{R4nss{QK?ObGx@mdG{k~{JZstYHBgXCV3E4ZckwEM;UMGxHXf5-hOR&idx?QQ$a ziQUutjcN&*$k{+H3>Mr%LIYT0SqxX4e1xM9M<(J}k#7NzifjV%8 z3QajTz4nY|$x_v4&$Bw;kIwwHWYt@q4aoH!5x};7qw@5G*^bLx%-nlQt77hKAWqI` zD14ln-yo?&la=?G1x+J2-!J~XO*q|gf_j3O#vONjy!Fjlmn1#6Ra@<5Vb3tDkAG^Y zQEh?I1`=&wd&Yk964az{G7-g+xH4$op5yL^jT-hS>9Pw=yS9n$wUDtUezRe|dc4ZN zIYoDp>b9BjAuz}W;Dv+M_<{Q7PrmEQZIR_6ikR(lr7O05)9QO?zFWhiReDb#pb7{a zEyl(3z!N~gR5+z&rErz_YO$royzf{LSOBA85bhKib#TOslu^Csr96f;jqRhGx^ja-~qcHPSV*u`2=Z7U7 zQ(9-mZ{_5gkLA-HtWS8JA8x!?KGKxvVmXsSNz}KV&paZed|{y1gcV+fIvO~EAgMjL ziu!F(HKw8*W*TqYi2IrI;akGV6jX#Ba(TZOEug~#b3%DvcVPyXSMw8k@m-$Iq^@R- zxq$P42D?)!77EsIxUEd(vSFUKKQeVCb60`YTkkxoEOI+C4n5l?>iOOCvWYEi=4L#L z12)$+9Ppl;>PgH0=I44>=iUD4k5A+8UhoSS7UI7f_n>(66n|WJ-_mD*k}8#yIU(&D z5pX6JkRV@RUewRLc^37PEaa;AzOouD;C{SH7aR)g%|O`^FFgQ=3B5T&={Lqk;)zq# z{PVG5H-xYL@Nx~d$;o|l<;zwC?D7@G{9FZ4lw2K$`E2KswLzgD;uQ#!e{5(obknJS zwYVx_*&`|Af_A3+nU9q|A(O2EG0zkp4%g_vg*Yh@jTV9&#q_D26u znPoV|yOy1%K&IH4wM*!J?&H_SXP#u>n<_4GRRs18>gH43T_{D2s_*lb2;wRE)sbxK17H8kojm z=Z>N%zRt@r8CJr@4W_P4vIC-uqY021sQzSfVeI3YQ|Ahm6dOqx*U$B@L)*J@-?p@^ z@|a8bgb8b;(&h~nHefw42Hk`MLtZMNt@ZXXQcVqKbYtkLd0i)oQoeTbI#V`j)vr2~ zhWP{O(cC+SMAyyFC4uN>_{xwj8-6dyFZpzSs*xCGXH1*UD0U>9`}MliwwFbnIGUOq zijit5u+kfH{lk6(yOy7*11W?g^+%7IL@goZ94c)Y0F0`+$Lw$Y|ME4a zGD}u&HJ!~M!l=Aw-+EKQp9zd@K|d*(@p!z&Sa->s_p;VMSX#p&e4Tkv&*r3Boq)i< zJ(!7wUg-3_;2amY966&TEUOa5!)99Ks<8qZPq3lol2kbPyP@h{HxRcXn?34-+20H} zIL%MDerIKby*>jKB$qrt2immqu#77tU7@zJkgiS7g~T1SYUzJMOf^s{SU^NIrR?-N zSL^ftdDu6=0}ap&N_h3o8XJX_lsbRrz%@}N89q@4FTqZj$%~0;J%CP1Xmy5OY&}cBU;6RfbH1xARUPZa?K%I<{Gx&q`ziT0tJk*#N zY&IgmoL(vu&81QEOAZtM%I;K00+6upPweB-?Y7^;lgFtfX{1n~nJlbR2%0$GF@G%XB>J>l{ zvztKnyuik@`uPI?n)FYU|6uG3ANoR62juzdM_SS>@Z8+oFlh5_VHnj9ri!4O5IVEc zD$1*?i9i?!1rP65)_I~+^1cM;84imKYCh$KFi+o*s4G_AtUN!q%2RHG z_YoGM7KZO4iNiNOWJ|tYFNw*BPJp5XP!dA2_6nLM{<_LPsw1AB^E*ppqdXlJ$+4(l zx_+OLRVg<~up(#Q51OgWp0wbSf;WmKy z{pl~+1+KH45`ro5KQ7{ou#k`{&}C&M|H4@7Q7oauo;R# z*ni1IqNXR9xn58o6@}xhsv}p;C<|eR2L}Vlhl(`%CGafJ?ec4mxiG|~F zU|ePwQCUCC1%#KJfA4nKJpVEdWG9@}tOftlBTyk( zXft55xv_gj+25@BonzJH@3B%E6XWC8H$LDPLodyks3;X!p9h))q)|WLlEV?qRkQtH z2!i#Ne?drw3Yq_vLE#c%HwbY>6|o8|zZ~npH!w@EG=OE6{+;gd(qT7San)Q^M@6fD z9W3ySom>DsFjEGTBMu8W8?NMgoQo{_@xS6a;Khy=7Wgfbe|7aWj~CA7*}p8l*UC6> zmj$j<=TMmp3^%ZA|BBzebT}2knpdp)<-ZV ztf>(PbMaM8kM%&_%*Ey4iXx7SQBhd8{5~r0a%x^^H1t<6d5g_0%%$=W>V#scM2lZt zsh49M)M!DHrG3|~Si-VpoFIW~E1P5*1PFF17fBjiyg0z=x{c!73!4is=o-Ybi~$rA z!59BpO!T_Da49ikJ!PSB{}foqE708j%h6s$gnFEEG#97&26BART@*|lmh$_>o2smk zU~yBR-Ak-(3y9;*HV5(dvpy;-IHqLaEpUBb4H4bHe}OV_^=o!TkyV{Qos(=92KM(> zA6e?07Z)EUj9>4Sb%MAL`*I@m~zt44F-I+y>~lP-o>6}n&8v1)a-E_?c# zErXQ{>Sp#oPkMjTgI6&zyMV>~5+VdQ2Z#2vQ6Os3g2+q4!e;A`s`B!{s~Z4wLG1Mr)c0?pLWDxU!&Y&;LW$1i${8+5ab8=Hgqx82G;yEE#luF8IMca7xhI z{9nBWUOEe6FvvPwMxB}%l=oQ&?|npes}6`QEX=TA(X+6)9ColT#cL(%&V^4@{U&8k zU?Z6s?^}TAJHYHVz`hcWS!hHPy$A~~*iZT27ET9Z4MW4!EXUk8Z%%`(E?mxLpfUJ= zAgX~~3kMYo{gyA{Fwo(;0LFNrtM(kU?Se~$h9%&Y{Rdolw4g4hp)XBFLkvS`=k#)@ zD1v(ee!q+5&TM93mi1gPWI{!Fh~z<&HTSZgn5yfL_~y;|vu9tyMjc?sM*k-lv+Tf% z4lI+wnkDN5jtwAx82bCIf%=s7*W{D`=c$=OTUQG2-TOt_SNoTQg4gSX_e^F9M6eiU zl;dbrI1KauH$)ZA?hk{N0x$;zu1#OT;KGGii1Rp*EJ_=Ob$=!EgaXX3p&RT;WIbzm zo9`wG^?I>_$pB7l3)@wJg%)W!bc8|A%L21KmrBLJPbN{otM80`{Fk z5C;0`;7=%=-A?w)|Bl{%ax))8PXwR0(+ffSxAcGCLXkunEAMgxl+T3LfQh|KvczPwwjY{XLe;W9hfu(iW7vy3(HDIDSDXX+F zpO9DL-{N*(6~@Y*>_TIP#`7>9n2;4TfrwM6jH#n!XL_o~y*#46eu9U@#=Q zL7S?KpEuD#ru2Av!K^3sVGV9H=U5KB7*_BP^2C{%Vx|nNqjNe>M86{1 zN$$vrXc6HxGw>^jlvb@Yy5gkRLht-7Tz)LUI6QY*ZETES3%|3LR;9s>+?KTM zKzd`|**)mjc~YrD;oYezG&Bce*Khw&LC3u8s7YqC4U7|^X+@&`Ficc$&i~Q~?J3qD z?eq%Y?CJIpy(o>Zy`tt()>2e&u*Gq|njk1i-D;j-fk~&F)o7t0PC;OucrI&xE6tF?W8gDC-E^(()-|<4RZ06QG!i?d zRMqC6>?51s{GvbqL^bc1DfQXeG5XOjOrj~nVc2|xkuGP>f4ARI3(dGUs0tD3N;lvt zJC*f^#S_Lrv6k!Pzk!kWfLzyhdx)dwf%a0_C}swY;k@=3ce*v@FFnuf29QqzBQc zkTVG4Fez+0YoN8qZZ=Kat#TITp*8Z2bGYy6L&*+CZfw7IL94T*=YY)b3yW8l_#rgI0&XY(iR7S@;wu>CvTAZfG9zNEF<>fFPrN7dS}_GpcmbsVQK4(l z;;hm4><4%J$cGJ^?8Iq=wt&;p&-a1uXa--sx{>rus;9*@btfWUhB>&DcY`Z0yxxu8 zTyn9p7z^`$V7`PnV6ZeQ*wFP9%i^$t`&1-E>GX+mX zdlT5w(_8TAwP7cJKnz$+G{bsR;rC}Z_ca1H>`^e<-lm1XPm%<5&U&R;NsKhhjlIB6 z0=%ZLK;IK#VlDkjOPFgGqijwiC^UPxEpnGj>W^a{onRXbC}Kop7&y~~$S4l=0AW@( z#oN-Lj=244a2L}LM#IlGzIDHP8#n;BI*|QZfLy^(G;Nq{%R^gbKIV~?2}0*Om?I+z zba(Z!l9zyt##j`1TP9-Psa%Lb6QsHZPWO%HYPxV{KpK!tlRscRX3t`rCKK~Dqt^^t zWDtgn@R4D)K(;l++iA5oFmCrS%!i(E*La~krMm{R);y)rveK%s zY~=;B9s$EtnUe=#S{l&~snl=dXQ6GBUay3QU-Yw&Pgp z!#L9ck7Ol6XbR4`CB*8cwuGQm0^tGw-S;xocu-8Uq`kPqyM~)+lnsN!XLP@6QO?6K zUr2hFseg`=gFxWdMP)ZrzoQdx7*v*2j6@Wdw}Z2WfpS?t2+?O~c#c5EbRQepa?M2| z#P+)!-~|$|D=R5y`T*#1dA3Nav#O??{tThn(2u`GgdL(S)^A2h9bKt7u7mlWH zk3WfcRwWgIwMTR~ zQ-*8N_ae$WOx-k`MV*X}$<$*!7|)X(`9t4#C2R_`2NPBBzF(tlb#a*6yc&2Yzkw+l zuuEJI$;{i3Y?!X;GF%xh6gtkohCDF_1X3c^ktJdkL~Xm^mz`T=3a51v7jx;)dQ?^c z)o@6DWNDR2hl-|M$}*m0bcSf86*ytEfNvzdVikdY*eAR!$XUb_)WCQIU`@+>-Ux?x z=fcSR2U{f>VB~}Q^^&G7r0Tcnu!2JdDb<_}?Pdz&(cA{?wt#IBafvCpU;jd(9-_*J zY`OpR4tE_ylu;_+V*V!XhPwHI^AfiPxC(wWT`PBn3$8>HzOkf3R}?7V6sp40nG>_6 zmYIcyg*L?ZCyK;DYFp;-%!9F8!>11e+4YmrWjbFERAn^31Bqz0eLaf=2L2=C*J$r) zx@ssw;uJ{1{hb;1h57n{O z8vrc4n#5i=Bb@Qg{vMk2H03>vgImDdIid>#+E`QILNad*y8k;@R5B-r@a+&QB~&|oXxMWjC(+Q%7`xo{4N$?0V8>_s740wng z8}VC~8dz-V0&6Ie2{bgomJ)R4G}u1XPb)RZUnOAJJ8n=0lZmZ>K8pmm&j4D|JVx~D z5%oqrL#h$D25w>=nkX|54q3vGbfED~7~qBh>pEa31@3~t)MfAX+YR1Sd!ZgM{yxIO zz9ac~-hLw>^>3oiY(4b>AKS>&ihv%P(!TzFCS&$Hp@_Wi;uP?11<|O$x&oBFU4wOc z#T8{RAc~-Cz46Q9RXM;xJCY#9^T1jR=(@l|3J?4`J}R5xnfdLE%za&W$>pU*FaUU0Q5bM6xs9IHsv_G@6xWi*KG*GFv~ zGVbZMs*xTl8Z(nOO(fx}!5>_mM8GzlYPv5% zNNR3~cPwb-{s8;$S%giEj({LD$o_%j^$I&Xf$XNS?$YtcBO$BH5#LFc^8-ZXb^b18 z?E{-CTMBh}*Y7k#YSbI%tg_11_x7l8Ij(zxoL4yCjon)-@&CAd|` zIfz<_u(+q1K`?|(bUXv)odN9!af0<*5s(saDd=n+g!mMuvL%gtkVW9KU&)S3S$S+o z^Rve0j&aq%mWNE=Musu7w1LRjK0Ug$GKGSOd}PWML}0eO&0m$SHc1FgD)h2;(#9I# zVFH3XC^i)FB_MoQsgw!KAtI#wenGpz;VLDEVarwF0*Fk3!+;k%PrgkH6BWDWP|c*b zF>hSz?~uPLp0_1$rvng!PXftIc_gB&sQ{K5Kl+#pdY_yzfdU2kBy&JrH3$p9@bwg!_ow9vF;9S8v}7tIR%Wtz!Ag zGi504AhW6fG#t&+a;jz5bfAz59fk)sO8UXfX~9_Ombt>sDiv?J^CV;G6~F|zujPDe zbOsQ6C6eqg?|p$e9XBgT%RG~1q_Jyg_|sVJ)1ec%cMz}cCwTxH2;%;_L@FD-F+5>? zFe}e2$CRO?QTd`_F0gqC@=c(E5{k6QkUVvr#kf@t6Fvls5vjG1P2`pF`y>x#F|4wAg0XHL}Q}qtw1Ghi_qVUVQ z<>lqBnps^^`39H8s%fA#z}9Dgb*5@%PC!gfyhh*LAHJ~>W<^eEc%>7DjS$7)l*MOn zXTNNtc!JDa8=_lxq)l^i$#38oj0SOthP1btKn8H`u^3C%r{a^68s1(lTIgE;JG;a) zK`eB75x zb20pk;XJ~Djs%>t<$)(@_>scra*P&l+FIq_S!pa`ReoPvkcy_qNbU>ukf58(H_5t) zm?s6km~=3G58=OA4v_J8cHa(bq7I1F?zqsi}a6PFF|NiQd)b^78+VHs_&_N)qr5L!Bo%9CinuJAV8+CWQ6pH7_AOKlmmw z<^4TQiXJ+URPt_plo2yc^Du7DI zV^L@+L^E$7HWak9Cbsnjy<@R6j{reqXq<(-aylrn?v5nF+@(EX>?*2cAq1* zW-k21;G|IG@%#}wWv=p|4lUP62Hn!q>f!!6z0DxWudTLO0WaL9(j1-CR2G97WH>@f zbwu>=5SKaTrr!Vx$`n8Z#n-drOM#ufHQVCS`S3ijb!HzcE-MC%U(a(R{Kh4}gd>>E zFHj+7Ha6JMmXou)224i((R1Q`asaFBnEQu5!uKXkN^8~@Qb!?ORNl;Vza=?DPA}%E z)b3Y5gAi9UATH@a!FFEk#2e!g6ro7oaJuWdUmy6ng5KdJbYu;5jv<2JxWNg5tYFNa zFZ!jjj4BaJ6tJpfVj=U8wI8`ML~{q5Si+Sw)}Z%7Ba%r=zO77~$5x+yI*0e$;Jn;AOjd^Ondn+qNiMd7c*X$&W%Eco$Vd$M z{J0d(=vJ$rI1u8$eCiaK_09>zLxVg4t!pL}(Zhyh!8#T{b^cb)@=1#ucMe4d==4EVK+9Nj(8pR^`orf%StFH_5Lb%P<9Z0en+O5&_i8KWen+ ze_T9teE3le>d!MW!dgFTy64RJrj?b@D9)0@<=GB)6r8fMo%+YJ z>9M0)4l}uj=we&kJZ_m=cU^QoUNymN^|5}mG(xZWJ7MO|AH$xr?uWkz7+BAV<1;DG zUi*jHopoq;WGK2dG3U#5cQEwSnxsv_6jM}FF~#)5Pi-27j2GW)T}L;p>lpXgJKv|F z-&2mSu6mQ4?{*=~wbK`gcuU4OZ{eWr=QoNof{)g}7EvJ*fB&mKW%}Bo@dBau^KRIl z3YdFLxG&zHK5oSm<)GvCTVd;$3ESCFg&u{D&Az9OhcvS6poD?gu4qelFM* z+#Ad&!uiNob%*j-B!tfgN0QGg6FUzoCT6A?>o#;vz1}$g4!6$Uw~XI(ziA$0aEP~Y zRr<%>OUtBW41zAS<(paDbf?B{v}>KmZfdj

!po}?ZRTHFmHaiKOExVfo>t(&BPu6L zar?BB#!QdzX{anv#_N1``}Abz$~FnP*H4s7?%m5Rk}(nclE5Rv7m$S;brzfbSio%9 zNviGK_3j#Dcy0C6NvxvTtuG(XDp0zUTKnOyAE`!EZAl_)jNfmJaLbizl9}Hz*tC57 zA+c-!0K;tD*rd-x!1byJ#Z7EU6n5=rz^Go1>LHSQEefuG3DD$6IpEpM{{DWLK5v#d zm8qe>vFzBTsMOCghmB3jQ=G>i-q+ak`Ff`x#e=j|ZmK*rJYX_mDq9TWV5Mo<*O7S5 zX;@(~xp1r{*jDiIv17iG&ZqISn>@ZQlRJDj&)R@rA3Kq^lGZUlh@Daua68|L4Q`jTut}Y>ZYnjwJZ`hfmN6Hq_R%h{!aWi))cHZz zCSXIkKHL^$bsmyo`8W>xQ51*m<@qWrzu=-Esp7TB4AzRe6v;yO*gMO%%snvqnS0#p z4oe<_&s&GNxE&h%Tu<+ytx7H2L{v9;GO>94RqWAgt?M!Bvkk7731Wh($HJH`!U`*l zkzD4#!Yw1|l#ID}=6Srjaal$Lp9wF-H=1SM7ZTPABD|qGMxc`t>47glX5YozwCdUU zp{Q)n+KQyssvrYC&5tKzSAL~Wi{*YNFfc6qOE{DL1MZz zx$3TZV07@MQ_oY1;!|zqgmG;}$4CS5?A~wLHe-!)Q*9}QLYalRuj|Vyc}E}bnN$4! zIL-QVf)>I3+Tr?A*Vi4%l;X9xLws-{he(-EUq2|@WP&SCNWC<5QwmzUTHnT*u34yBob}N9D3QZdvDp_BUBS=>nm>*fs5wJuungQh3je z5&aUqdW=?LSs(qHZ8075Biy;$c&vD*mWhXpbfAk2oW;)3OW6lgWa$s`568I&DmV=1;T|sN$Q}y3c3tSs8O%&;u&p^#0L;rDS!3gI$(KHcfiv+& zQf4iC)pYQQY#-KNbatL5qcK>I_k3TL+BVwcbj_njY<$VYjvXw+lmD_kGR#wL+j`1Ss}vrmjppv#5K*pI)76ersL@p-ZHlgIOd=+n)^WGzk0r z>{%^7?8({#L)@)y9x0yomPjgN+RDd+dBY;IJZE@Qe5U*gKX2)81vgp|J{F!&=cCuY z4rEF<{uo~ z@SA)xS?x`lSgsr?^SM*%WBAX$>5IrQgA3g4ae8URfVQSLs6}7+k@`;F+BR*lkygtN zYUFkMxEvKAyMS1Q9Vme}>G`p9)$pyZBHW?<;07bsSF|E?k9AOeh&iWBdG?ZzPpSN# zUE;NQ1I+OFlv%Z(j|Fn6qUMpOc(J1VZc*YSwb@V^%MQCFP}1=S7pRW($IS4xntO@? z6$EA}A)m-yvYsxRu}M#U7v+y<9v!k1)M$+G8j6;N+(dr8yoX^z!(p3Y2BuoBurf5W z6ttvc&wwY= zk7Xg^xGoi4l~?L455oATl`FumQv^R^ufxGr)*DTL8|{;py7sWW^c3N9|rrVGh9 zwFS4ieXT9ye@>f4XP>`z&vJ)y8lf|mb`KfW?~8~IZEi=bZF{;C+jyE3XNsoeGTbR* zi7*r(-$w&A1$c{_)dyxK5&ueq{ge(rG&YG=M-2PmCPs*iRX70>{`^T4t z2!`peXXek`wqkZOz_JR*2dnpYqMB>gD@ zqTXME!y98GCNy5j^;HYIQjd_!`iPp3u$^-%cp>YprpQoGER08eDfY83?y9XEt#kWS zq@0)5rLlR7ot{gQ>sW+i*RW$2p}vn6U6}uEGOU$Gn7C^-i0w%eb8ro~7+RhZ9-v*$ z>pdcT!`(=)PQ?BzdJ1B@{UuEgfDNt}RNj!gHiYx2VLZvtvisaHl&Oy=xn}X_>(ELV z6>D8>?Z7Gh+dvEZmtB9w;HNcU8lq^IV2NWGc7EomCO&Nlx~^)Qn>Ru}@>JHJHDn}0 z6O>s!_rbdAvOwj_0$NI$gpOqbfWRJqtq$J`n7N8qsfV#l7+3J41Axu2QHNHfH7%=Y z6_`eoFbUBrS+~o+pltkuyWyOz2qf!n@Mp~w$HKW1Yg==;SFwE;8>MHy=ir%c0ka?! z?*R9>j?B5TUM?7;uzQr?;BuCZ(2F=TCAGwNIIbihbhsz0uyaG`g!?_Epy+YWaxR>eNKDl?BzYy9MG3- z(@(xMMT)Q*9k+9)(C$U`%7Os zjD67N4zv<0e@P180B0!bIpfn+ylRwzclPY?An>0{ z7SIB$5Fd|gn!4ax?6?p%;DUv@`D17#jnThrC1Y9>rHI8$w2sEK{l?{Gcy&(v?nD#4 z9;c_9thXLB{fXCh2*A?VbPrB-U!RlW?>W@@t=o5Wznm;OmDK=crCg)xu7_L4UMbOH z=`kZ#9zu=zFpjB6(CJ^&CB$a+N(&xDK3| zX@FT!F8L>?W1}-Or7cct9zDwDL}PrKo0%{I5-PbVmx=xo2cm(H@1hka36h<1pavU? zH-0!YKT>Y@`@lZRPOeEufs?r?={vy9(r+MVKb5;1{qF#H=lVE}a@fSs0C@4r4*i7v{$wG?`u&AztVjXg$JIurM_}nJWVS$3P97V-GiG`iQ1SXTHlj*QueCvFAWf z(YI>wta^jWntwi} z$!jpCRr140rZ;a^~ zLcY+fwn4#h#2Y$z7rs@`{?zwp#lPMWu4&Sk2n&%C681vN z-NG$Zv%{+$etZhj7U7!s*Eu1BFZpc5`_60oVhbh< z9P#trgjk_h_%pUT|E(R9+Aq}w`>P`bbPXIXR9Q~+SNhD(&awoN)%$c{8(R>08yXs} zCxfpEuvbHGd=$QhB&iNVw=h*qVY#3p%wE~V&!*)+P`RVD{04vtG z#+46j9k4nB{yW1k*WRp71T0HU+4X}jWrvf}*usgd8>}k}#3=T`L z2V#+MLHmT<6kMk>*u|aCPw!@*Jr|jk2=%fG;7H})hZy|j`av5Q_AFx=x`jjJc!+}b zC>Y_Q6vTwW1$xb2gT%q5!j5dPRMJ-MFYNWtyght(33-MdLUa`BSh-6Ol)kAee_f%+ zV2QW)bcb;0&K=!5cb+(LtoTu+oRmMmbpE# z-1^Zh{hqD}7x~WFo|I{v0oI!rJD(;?oyL`^p78Zut88a}mK0P?NDbEzJH#s>?{136 zD3BGG+~2yhYXVQ68&?!Hct^8)Ul8GwL79jth~W09ppp~G(~2{mzcao}8!aIvJN2t3 zbJX^yBS0o;a`5+cH!0O?cI=2ar5uK?y2c_LL4j7??B8aU*phbwzB%8kHNPt{6J!w5 zGb27_Pb6<|U6PC5Ety#5DCbMPkj?os-x0#~1AK%o{AZQ)M}?+@L!1 z!iL^qg(b~sG3Q+$z6xsOSygf;QS^&j1YZlo(ZYScN439ztWT3EPB3vNMD_Fze}AAb z>a#3RbdM=>v~I3_Hrx999cNFnHlLN-mys>q6%%6n_MLw>>=zUtIw7-Ix!O9rhak)F zAvsOAno+oaI7uIBNg!#p*~xYF)U7uO(_U*gPhrr>ITz_lTCKw~YQa^$tux#ThdK!! z8GMC8U0PAfgDEPSppY>9aZEnE>P_(T=Oq%=ht>ig?aA5mzddG+JKqPjAHXU-u}9+s zHIXqKhbDp{90E<*^I_=0!)tnd_{Ynu+UE;XAAK?M$Mut%oZOTZ^gRV=B9~VOYi?Xi zynI01v7k0!a`#sYT3hm28576L@>kUlUk%H&%d@T9B+g6~8=Z6!VKjyDE2fqOGmY}H zk}vBKU(-6m5581ysC>z%t116F>VW34Vh?WK;>Gxg)m{4b+F*ND;tF%WzmheXe0RLS zal>VYQ|ijY4b7Lo+1pm;4gR)%;db~_kdivG`FNO5>aV=@Oj)LkRL&p1lyZk znNo2VJokCtqI7GLnY~8B-C?z5j{9mF>bKt$XzXemwX9C{ctcd&Ff#($D2S6I)Z?nr zOLQ69JLY{1Mu4(fKBGSH1`?la`%f>XJEKG8Gbq~1<88#Yc2}}*j6RqATRemkol}v1$%d*yUQ|Ekm#T5q8ouJDBeVB zNOv3IZA$&JjMu}y;gL;ynR1IIQ3Q_3AvmJT%6rh)&b+(0+FjaTiJI*^jLYkOSz0zh zUB5=0gw^nwbtv@x`0&}f#v;%-?>2tDMoUt|4SHrWpC;YKeMY&|C3v!P^)Wz+mw44H z+zK8BdCDp&TDgbmdFUL=XXbkjsVaZYBTraFK3+PSy_;}wcs8tULp9SikC;Htn0WL! zuqRo0w3T8(BHZWhPOpw!yHhp%VuPG$1k>rPLqD&NXXmkF&-m;{JTXUS3?GsUZtUCv z2GfM*hNb7FGNvxZqg$=4PR5Ok_dF&39}7=~Zh(8ahe`M0+?1MBo+GLs$>Hd|A-s;pZe<QF4FXxHN*~2t(&gjTe8WyiFTj3A==#$$LyGsb#4Cf zVggICrZrD6XS>5xKBm7+qfK`4MaATHh0*LH=1%sZ+bjfSILNtR%cp@pIM8aD*P)^QISozn z>(`pTAZP6qUW%4t&%{`bgjipLQ|6o!e|Arh=oCUc?iJY=8>)JPv$|$+_fqwW!Ff$Q z*eCkZ<7hjz=8fZ`-w+TciTNfn=QG3Fkb)quS6fgqgQ?#U_{DyBMv;nZVRk5yQsc@n zVIH13&TE(d%88J=pnN9Gr+;mI!my>3tK?z}2%# z8eNlcq+3qM-h*o>OYKGt7?(8}@H4)6&&gZ0B-gG_iwmeWvNevGa|CVT>VZwk$;mcN zmrVM2dH{6lu!&uw1lSmapN;ypwY33Gp$NmR1^MZkPf{{n5mL)y6jfJk;Sa7LY{z@T z@4lmxn?z|c@MzNm@oVjd5VX+Tn(U@Xm=mxwN8-S1Fg10Ds!R#k1c_C(LMeQa2d$vL zgYSr`r`Wq(h=To0kq;x+|#3{@7goT8Wdci#o z2GInfED+X)^Z;%FYu{{nzx(fy*~^L@nksI4e)_njrKOG~bOW5qTKvJ#aBfCI4z{Dn zi+?Ti6`tPc_HZ4QIxZ&TsD>-^mK0B*K{n(45%Vors(qCb{0N~d;T%2WP=T~{!D}-aflcBEO+M8H;*!0=FVflXBM5V(NEIP5?#Qu!u$3`L)dx3cjHqIe70Z91~J=N(Q2%k8!6cFw*Jzb!@H`Z)=X%VU;yaJjLO8R8ADr z120mai{1N?rz_3ds0}7ppgm7EkqWmmy6pX2NejuOBQvd!R&emo_mq2)-rg(>SGLji zm|0#{!SqCAUqe;MxbAU2=D^jW2~DrBywBQ*EtrT1iK~N|!zupif`J#@6d8G%mkS6z zU&?ffYl|L`@|QBcb(SnH@b`j37CYOptmnM0ZrMb&vH+y{!VeupUW}{Y?DxWy)$k8{ zzW(aZ)Gl%{DxU20uSig=pLSLU8yf6SZ@}?a=?jdFoqZHx?G|Wy8aIFPY*~w2_i1q? z2jW-j7Tc7{NnN@fi{4LAD^cN>Iza=hTdJxWzOk@lvOA&`UHVz= z=dOBghka8n2YXn~&@g=sO0J(C9Ty;FU5U}!vg8@;WMTycLnu@oPRUD&-)qZy zF0!-S%j*m|sD~{K~4F-9FWkhHrBvM}>SbbyQdTG;% z+OtBA7H^{^AV=of>1skS4YNM-RhUUgi1J7+aIn!NH(8imPgXv7?fO|p(q+A1?JZM- z+V7h%VUA`^vrB2BRWB>*y85_lx7rRY^--#NSB5==&m6o@THfmPDTdoNLfMDJG|;&+ z^7(!Dk!y;BaG&d@@Gti2-(9u$5wem=t>BY~p4W#)DWmui(ldZjmI}BZfFN)un-!); zi=E$D^(GkAWsH{)!%^Pi#t^E=;}`>;+VHBurik_W{C8+PuHH@LwzDJ<`blifj$xS zP-&4tD(glD4zm&AT~@Nf3c_`Wtu^_1yhbMnZF=m?Q}I~cNt<~iqJV2Y@l~vVKSTk0 zPxQasQy;82)BHHBRaI-x+}1padq1YfvCyJGow2huKkC1y@?}MU^?S{8O9X6BYFdI@ z473tXW$AMG^vmiE`OME}g-SP(u0M>V=x1-sv~CNTOyIi(E@__H=49N6UIlsjQ&~$G zhDJt?d(vP*07U<5fLjax3_ZHaDhUC^dE5 zOTo#nh1QP&1X0G{Da-c$h|Ww6XOhFN5D_wwW76iW*ax=!u$>xmd$ksec!5qXl7j!&&#$ zSFzjg9kB#mvtUZk2u`Py#`>5WNd^aB$9KAYB@lP+xHfE^YyIgQq0EJzsQ*~0ihG#Y z+wj$2E_1+;1t+p@eT-;DHvbf6#(e7~>eFcd#=zX2N$0({42}Az3Hg3`gMyv-@WKOw zYFrFM2M32kkn8Q(;K@pEM$zah>~aZKhLzx9J9ztXQ!e{pbestYhK+dUWqdP2Cc)$! z7zxM5N^Q}9bqlC5SE6uX(lTP zfnl>x#eHQ<;H1JTt^Uv&#N0F(&3kO%O!hnzM4x}Nwpb+g=1G1T~1 zjT&qP*r<^!H9!Otu{KdRbwepDLA~(`J9+bPA!W*mFqh<03Dbphd0l&_H_jXpfM0h1 z5O;N#@h2B;cs!-d=x<^Sl-00^J{9VIIN(S+?-A#?v9UW;?1(_H<4njLnCSw+$(qKu zEfA`9gXJhe4*}uiPC*aXWA4_q8g5ns6C{(~ku-37$fAp8m$jwj$>W<^RGPjWJ$m3!mI_T4s9U0p%ig%H-~Yaga&K-B6y{%*b8)C}`XBszPR&TP*(obZ`m2U{MPk@M5J)Km7J9 z24#zATHztZ2N1*IJ^NxLx}Va6dKpK^V7OmfRgHwz965&K$zx!Kz%1j`V$aze?eLJB z;sn-aGBxQQZA~vcn-W~^{)VNK^L9Kx?U0Li-+s^I?yi|bQ?CVj8^E9Yfi3BxM5e{f zU8wTx5I4i;sWbCtWUZK8#NKvwR=_iG>5d#SG+c|+NZ4GIVP3bN zjXoP18_5YsR5=!QW6imQ^K;}Z%HzU3=W|t-SxkyWHUYt)PBM33S!)e^|_;Y678iTT<JB56AJ_jEZp9nFU;sTXekd9|Lqx}R*)_2ZPA zCbK)(_IE`s#=ug(CAQ05G|7n)!~wWb{#3#og}mfEI@*W0+D0r^A7M%b64n#K?wtHr zaYw4{P*mGCr7Suf;&gqstCI5kc32g);W@DsX~__;*#6m^CF@MuGA-X-D|!4v(G5~L zVft1%yElj#Ze@97R9oXxk(p&1@((KFzZh~9sjac;+%3@!+WQzHmUjcNH*T;Z9z!c% zd2}PA_Q}rEJ(&-6N_d>wdBNmeo}8Cyx4|GZX~lCee7(2cAHprYEhrGBi zzoI;BfC;uQXWb+xQjt}Gi*IOZy3q*9kO(N^%>{#&;$m^iYtel!zR#_hBCm)-UKAx; z)+^4wK^X=J+xN@Xz`?M^k8esA0ZZ^sKS>ed8xuy-~+r7>-H6qoR9 zPA5?;4BNu(6`sU%vg0^Nc~?Z}wTPFMJ~bA21kmZ+`6pK(V*=rJR+@PU|Lcu2qt>6- zsBT!db;#c^%erl|b>Wo2SX;1RyEc7bnNaEWhn@%6`Fzl5QyB5o3@@Wv>Q6MajVgD& zTMBrww%)ZhLWn%|fVGNC11>3ON-!pwBzxT75=CJbgBI<91YqRcm&ogl2#LnVmUztY zy0SQ@diO~`sgHYdMH8S6u$D@quy)ey_jMWwsV#@PUJcWI+aQ9iSGN5t^npC{B3OBb zDsyA7>8=F!QBGLZLqkXzpVMUo)D3}rDn_3Ld81TVVEvX?BdgH&v*r^O5DGK{cL8cq zO!F0gOgBsJYv}Bf;qS1a{rTTjv0Gm2H&@6(J^9-F+)Qn2YY0b8IQ6iTAAq+dtqCll z=l`MXOTc1Y+qlPZa0tm(w2>uBn`qU7k}M@5v?@fa_H`=GK`CWRXlf!y(xy^rx3r;} z(x#oZDV1rNwwY$;yPyAmY=`&zuFrM7uXCB1|2*?N_j5nL=e~dU@9va?WFDbzdHFF! zRzGy8s}(iMa_YT*d0r3p4fMs>54;wDeo%69P5rcpb!;6Q6 z{wcq=Rnj&eL~R}#10;#z|N4tqv%$K3XzN-L#GJ2%!b!kUC@)!s+$FX zCI1zcO%k7qt%uO4E6iq}T> zq5J3`F!KJ|w9)kX{7;}{n$t5AkJ-;`O+1#KRu8(yDCGbYjX-ePz%2=eBxUj6yy|y6 zB7td)7&Dgi-fEO#=h}QPKX*i+7e)Cwt9tp_ESEO#{-isQziqQe6c7-x;@XEGZmO?- zT^3$w%7G{ae{oGt77shk`q`+fE-w=-KkMM2$6Y0LJVon$9?Ej8f8I!)^A_}gF&Dwf z(EI*LK@hH*WuryJ8wIkh;I&c(;G6c?)W4g#&OQ%NAU z%FXM_!*&g1Fub|uAdS`n=GqcUg^%@vp`JN=Ru|)}FjjNlf@{DBm^}b}wkHV*={6vR zppU;L)$)R}kvY-rS!U+?RxBE&nSuuz^UHo-r`ZAde=8$$L{@x4Lhw#rwJnet*u=uB_##HNwmr3EQ8u zw7nz&Eitdz#I5eIoZW|~;3WBRm&Z`Um;KZ?0|9OiMh*yo%E^$gR;hUBrd}Lkuq^%g zV;3=NDWDZFG1dw}Obj*?!VZ6OeQlE}-W9 z40rxjabG`zO^lV56}H-lmos4W3m~8y<7>GWI#+AKDR#uZ;vQfJSWCjJjT<-eaeuxP zK5NDJW_fnT0OLhy8eEy%v+s`V>L1Rk-fI|nhHi=PTLpz(wzjr7HM)9yVb8uOm8~=d zAc-N<|6$_HxD%K9)jX1prLV^(E~W-8RY8F|C}h5OkhO+wa{;@jXr|yikNg(Qbc2^m z?ZN45X?$yTad$Zpr}EI9e;rNM28BB{@893xo?7@cH2*pctu{1^$^wpP3$|fv4Rnej zWyVUws`XYG;2)cVA4g}ZA7IB(fC$+{MiNwJQ4^W z12%5C;{uEh9U_qkc#G>)8FKhKt^r*}a)yJQ9o^U;zE|KJI3`~8NeifN?QV(2EkESM z>q}hUq=u`~G6H{DoK<{WU(S6eF7p?leiHN`kM(N@FJFh3MXe9t^WS%14h)W(GxiWE zL8QCghf-gXzMFnkDQ?H22M-=x3}nh<8L$U%ei?n959E}dcZTLQKdk8kTHh7u9La}Jn(IA$66Joik`!oFfhfG(jgS+?eJ6BR zH83@eI&7SNdOvR6DTnQ3;}B);Hq0n^Xgz%GVlYp}FLnGD#?z_l2hbG%_;KLbnn%S@ z8Nf9^O&Of(M6AL<6YLKRP$7wsUr1mBn=_;H~At25~} z1Te}s2AM8=vsB}er!WvlJW}sn{XiS%auaV@V36^^CfqV&zgk8}eSJOd7u6H)SX3gL z_s7KE=Ud=5uk-6xA(|P>#zA)HHx_S!fe1o?y4}x~K^FvS6S{dE&2>-0hjI-+av4v* z`cUE&!hSi&pqur7-tmh)#+Xa2zvDuU^4RqaXvJLS?lr}qT___1PB;LUURl||2D5}V zzXs;!+!Kz)bz)Ax8bqPdv12RXq8N-40FJWaQF^5p`OP-qhV<=oznI6103g3w0<^=I zL-xgw_plgS2(F>irX@WY_*-=eM&hl@iGb|>zTp7I#F(#PvyS{f@Eqt9-geEcTF3MY z9r_OvQ#T^?GVzKBKZ{rY%stls<8~7`cjfLKd2f2}QDCLNm%^f5Q8*Z2X##x=sioGz zV%@}OsD!&UW@@WYietl#s(BtS*wXF!#s^{mI?mEb9^LU5%7 zKt*%wBcPOh4?{3BXw<*dcLua!wN#O8dza|}bGDRwKD!ixjFH1oih?RI0s~RqWRW$L zIE4AzXw8>aejd{)h04er1<5P3Tq9z-}qu=Eu4cvJz0M&xRB5R<2%518_Hr1;pA@xffMMta1(LY}bN}L!$ zC;0db)O(f;_c@V6_(WCVI`Oy4w$nb;DXMj7-%t}(H6n9#Q_;AQhgJ>6Kgv*s?)|J` zr&bX#){j)Nqqb77&;};PGpWX_DH)a-6L+FUsLPxtuU}r4p>I7%n?eO#)FiUg`11?V zH(#8GiQvH&9^dbxE7ZJQA)GOIjDsk(K}*EfgK1%6>RpS=?u##7T#hJ%poCcx$hYQ( z13!3_^KJG}7Ma==IhH!|f}LwvA`-Y`e?Gu3;NP zRWGw`6Rs|LL{uuGJK8!+I5xkuAg-O$%$)G_xpaGBkbLC|e#F~HrD7_=`x>DjYr42B zc%)4M4FYNaM%(IJDO8@LlTyXr*m zA3R`}!FMZ_e54n3&m$7Hi4q(yXuA%-l}uR+Sd3W4C^0XZBIl<)M*OU+@{VD_dSqcJ zM5|syM77GkQ}jkBs8!pwe$?T8#6}(XtHu3wzEj&qd7g!xT3U8cNNUCWIzvS z9%He#J_Gl3@-*r;mu=%2S&1 zQPJvR|AiV*@iW*qUghC`>x?5HUc6};eU0IxeQ%B{{s6iNgB#%51GHgOqXW^qQ}1rW zL44pTs?a-a?&<~R=hvzHSy$*f4A^l)BDq?Y?7q!pV7Ohrhw|kwoVy1 zbf6O|z57yKY<=xmeva+tPh(S1p|Ob*#91k?!nJA529LWV3McnlQp4Zm<+f8lyXY%= zNC-jOG$$^D%asgtX(-hVgo}$BI?s!Wf|lHLKs7TT!T_~t0a|JC9j$O`b!#Ywlg>F4 zf*QvVNd^{XrDD6UONfA+SYPIh-czV>vsI=K6>Iu_ei)_=MI(H3L`UL=cfl;?_0>(X zg~NKHX1lbWY7Z?nn{;@1aE_a2;ToL(LAOd+U?D;UX;Db~5QhI@5+K6 z6s^MitzdweSo^JTs4DB|@nNsuMRwkCgas9b!JRNw@|7a|6YcEXdb=&aIG=OIb>_{J z^`$R)4NOhFcRtOJ;yU5I* zKa)?J`JX5+=b%laJJr!OtlTgG;|SKsW1rrlo~iDT%Z?@yQ4N!ji^;L= zlqg@ri^=9&VA1_&A%m)SQBnTw?#o2Pnps9t+-DgHW8PYn}(Z4o4B6CPQmIvY zfR4EvoCw8T?<%k4`aNGo8*DG^tiBMMPCIn}O@w|3_P#;(SP*sLr1BRiHxmET{iRW% zK&N>04NNT{!n_c!hh&%qs2`9AvI>y-uYU)(gech`E;LlmM00tRs*&uRfV}O0!}D7l zV@^8Wgr(HeTq|Z-_x8(_FH(@`u05iWGag_T>&qTuL@Ix}6c}jhc-O4z;XPE&J$}1S zd2Gk4MdC)$0#H3hd4dWZKY)ABA{m48#Dp4bZpx4apwv`3OsDx-1ltk`GJs=$7w-t( z)U0xeNwYER4l;}EzZ|`5`=cE*)m!^YdDX&!17f|V6Lz$V+3VIZ#w@}r4cz!IKhmeL zKed0mN%vlWI50#kVgXuu4Z_sWTuJeqeiaRa)yHn~T^HPhTKR$&sKm6nKyQR3~tSlruvKYIETgVY44@WK~$ zjfG^^yTa~TJo@$TL z@t5sb^4i%3i66LUQ4|lKDL>ndYR3c1L{EBmvB@G&M;pPQNex}f{D)~%%*mj*lfb7s zWrWJ8XV!}nUTEU=I;;Z#Br7AFzrjdN#b55FF>?lY)5_z!Bj>+AmTL?OdTc>5aU(kX z;?x0?^-TrWoHtFOUb-8qoSfL(*~MeC)TTu7QFm>DvX5>=r7b`edDC6i;H-vNUIRt(edat5$RscAFC*#`iLIT)((}*o%bT8-#t+S z=o7I%G7nC6XB+11b2p;LomU<5?OCFnQ5TkM@4@u_6s}KZyh$7@XmCPu7TJ;Ob=^+3 z4^i+k@O1NjDzUpfaSVK?_WIE;Gx~T57BP5LmY>b9K5P!kD`eU?LvCAB49RUEKvGX` zOXAdjhDC}64=D!Ydf?l5dt47f>u z7SZ-7KYj<$??-3dMF)UdTRR};w|Cp*M;SaWnSvU_w@8@NE@jRn2n6ExOooOqSE03Y zViR0PZ(tRAy{qF7K@is>;H_fyzjtJJM}m$(nX#yI(dC>ZARz7R;MV=ReUv_MePqAT z=Q96kxe0#DW%B=SKxM_?=P_}Vwje2C1QHRfJTf?98;6{6Wu~s4kMbh4RY^^EYTk}I z7?&NoA2_}J(M-9EkFNEp@$rG$N%WF0Z~3c0?XsT>SLmc4PtOOas*FDr)d14n;WU82 z^U+jG?eb_fRVYWSSAc(lV0-CnUJX1qx)(-!M-W=PKAP>4i7mry@^E^Gvb@JCBFn%N}=}hhQo>gJsZah z6{2P`4tw+p&WgcsY3B-2Q99_HASoMlIR_J8w{0T>9iHnUnGFIr`4S;Z_=e+es^2#B z`CO_2J7#J;9wc#?t=3(0!Ovv1amGkZHyrEv-C6-2rS5I$(%k2xd_aM{$>a0>sXO{z zwdepbU6|TxXz?*#KAIJA)l_{~K@40EO}UQd=r1?c7c+nj-2w1D1l=urDk391A|aRa zQV3!YVBlVV%ZIgaY953dDo(ITO@01TjSDAKvqy^V_%}2-vrW=JzE?t}?BFK%FcxPd zbjW9a3PD;2?|k;h_m#nI?d1N9#`e^S+l~j{AvlAiaxm2*(!&&`^F5`I}ns zwA5OJZC=hef0a_O6wa6KT5%AF0anf9*f2E-$2)zZj*c591@<~5Q{q7ZA+Kip!(%+s zZ{mqA<2Nphn`Wf>_-!=OM;ieteIR2YF~E^n6_tj*iT5x%+HFquOm3!}0*`7M-L_=d zuPU+10wnP+ay^Q}FCJyp>f+fh_abxIvknlv+a3mMKpy{cS1pAz5IzhNJ}tBY?8kea z0Qc-O<{J401Phu-O`mldO6(O|1n-vT4t|MvlP3rQH!DC!#JEb12g5Pl4X6Dx&CjgD zm0Ot93f?05zRMF#>-GZ&;Rp0#b1Kbw<>uv>>*Qn)*#1$mVtcP5I>xcGNbDUGQd(6rt5InhP$K^ZPDd9B%_)qUxha?!oe>o9t_1o9!_l{VSRs`snqIt6QPs3*-t*9j}MC&rN6 zhwUTjKv=e0%WN30`yERF)s}2EG))r95?uvBY0wNp& z09h|Hdm^9KE~zogd3W32UkWW;KShnlsD%HmOqgV4!D`yGxN(t)tqIXmz1{=;036@V z-3)?48LtT@sU!PoE5QFQoM6)9$#BH-*a4*QipQj@V7=&lHJzYl*p!Y!Hv33Qa#*}Q zz$`}U&Zv?9n9=|97FK3uGNy|pv`A+qz#fa?RxI=RXvV0t^oFu(^YY{8bO97o)LKo3 zT<<9$wXg^#Qov~P;^!ZGu7-(L@DwnLFN`OtTbs1F6UU1pXzjyiNM3C~^zpH2oYIbJ zz8ah&-TRy=&wLOG++#Sk^TPH5JZ!tRXH2D*WwN*o!7nY##Pz*{dy|>u}!fcf#h@WghxWk24a*$%h3&wl3 zx#81PxG~>RakIfeKP2Z#LBP8FAl1(F-dB(}T6pa&jOpm)3pdk+`{-LQi+UYJu8$NX zUM!$&1C8SbIvjX5N5Lcf_5|3+@DSxoIrI562$sE3 zJ=_Z`DB1f#P3_|8y0=@EGg5p9_s*jEmo#R-eBk>M!sIG=>x$U{_)WU(39Gq~WsHTO z3HjG77HYODAc3RPFaI58#Vs!ZgVY{uUTz{_L_GLD?s_2d6a`rfP7$aq9ibP1Xo-r5 zr_Biz)@p64$;jF#b0*iCbIMB39x$Kw#lI`FAFITTm$5Vk>qnKZHN858G%9i%?n@e8 z1a*=8tIA2+uwyqbk8&uo^7C7IbYP%<6g`#HgkutiEczURY!%KIB*+oH$9V^jBp$tv z1;aIEj_i}Q2KKLH0Cc?^%w(x(MYrchm? zgN{x80S;oRV-?beFnhlrY=@wh0_C)!DiTb^Q5MyoZ4I@94~=8_S8O>0S94770di?U zDj5UQd2Sjur(z(t92zX0vjNaUQ|oC+;UTdgh|%cgm@W(;YUm8D_=D~TI5-HGTJ}ktceNl|CN^D|u7BuJK@(^AS42~aP3HjP%86b3zXGyAbKrFHp zvPo7TDxgf3dv+J{9kt}cLbw6kMDFZs8x0?wOzQvq85pwMFPyV%NqKxq%EStc-^;?=vdv!w z4p!OxTI16bc(pizcocxOxW<$&N;kFkK-~L#uBplqi`Ks&$P*d##IY;OB=Ll2?;I#Z z7x*dejwgJ+f8f;HTZwLvlaYCW)}{5Q#BA>b20SeSHhu-3K=I@PPaPFJnG#TdH^ZDA zgPp@h->g*Q5LoF??1X>|+)-}feyJ(Su^~MI0wX@tqVkP~E zOIMb!z}Y{{7L;WPkjB1|fP_n^?|4qf>B!rlE*o*`O2HyF3N)9~x z&Z*&A8s3}}y%jszfKnAnuf?#xmX_96-28nVOb^=0T0`AFPPEVz#dAUf zaz9%@jPZbQIg*DNaR90Wl495uCL)DR=?g#1G6_EW?rvUZYieY~5B2F7wjku}2L=KX zMEi-qgOevd$Y2o~3#fTOG2*f(_8*?E=6=i+y0n(xmDj=7ncLAYZ69`F2Wl#obtm-7 z0UG^IMF3J5;>r-xejSE)Eignt^8jOdVhR+nTW~KR@DZQoOgJQUf<# zB+SQ`Vy{~o=|34ZS*t4w*{kcZp;x(CChzH{XC6@~Qf?fo+!e@M43US4=Fw1k&P6xp3<;`S{DL*Hk@v! zwy-Ph+o_iKnSThYau33b=RW~wDhP_iq+v4azbHU<`)Z_Pfij{{y5`QQEg-L%oDa*_ zl=V|WD?kXiMozT`jpkKs3TK*vsiExRT~c;FdIAz*rB~&1mphZ-ghv<8!L|aFs_PlZ zwS-&?k4hVomPB#N>Q7f;p3gkz#5pyOO)PDhP+V?S?VuT>n!MkqAwFN`Ab&sTJC;HQ zu3kzl42j7>{H(-_-#?YP8S!=#=r*? ztpICB6r6%s+IRqO%)sCmn%Jw@h-0! z$-R;w4T}=OUSVUPhKbUrgVCkB;p8g_6Jy(>rlcZu*t+no>fMv0kCL1?gJDSSkQN^< zY`79a2sRUWqq8JYZenMsQsT${XIZRPBcHdS8wbxtvecprS!0>*qH#!?7Gn2{0WzDp zn$LL!DBsDsJ4l=6F|^wM_)sQNECoF?QRFKkyemNeHeo%*QIVtoIdDt@y6V&A5?kZ< zPzdB-|D|&?T&_CFiW-Xm6jnABs!fS>TAFtfG}{lDnTZH0I)osd9&o=%Fv=R6NJzn3 z{fJ%d(Ol*PIaZ;+y}~0RbbxLO+#EFoN43CkK`(%=iZ%cWs;K01M9uu=v3w0k2PC+> z1mqMz(AC<0XtmRXzC-0)#b>PzDB(H~>`>WA%XwEIjBW-<_VtPNW?IQr`Tmjdr$gua zE#U%sEF*IaykSwd`EzM;p;X67lqPs4_~(@}(XuHB%SSDAM9v_ppodvg-8qF^TuEs? zl~`_el#^g4Sf&g?ACcVCKE}>25SyXii{}}cZHKf8fpI87%_cSuyg>=u$A$emDEQ{$ z!#GVu;UTqt))?34I8vUtjcr{>pgTZS1kUi7Sv7}iVfoPa`Fe&KgbIN)cdfK#Mf)2R zp4lei^C_3PARz=)Getn>QV&D~l95YN9vgo&gft3CvXyP1H1XJT(%Pe>Y}@vmj+JIm zxv2UP<yTbyB*}eUBasJkXoDtGXh4o#AQ*Ax_@@AwNu-4^*q;*ucLSc8esv^LBOV%iphn{3BV9rRWgY7kz_)F28z(chrKlv z7x5<{umC96`Lgs5gyu)<&4IfCJZhLc?ZHS21PIDhm-Sp6D^#&)NKn*;)6!7T>(x^< z?0aOUTWuv&lV^gLMLLYCZh5dDqYV?HJe*#hN4YIbSsV!{BB5LqOg9%J#Dv|0@UdNU zrj&^9(967+Bv+MI7_LjDvN&jUZMu)x-T@*wd2OHpny{qR1ATip(rHNpJ_)t9z_%`2 zTi6O7hnOrR1^@(<-1ta=`MI)Cva3I7D%+gjEri{{^w@UKP;fZtJbzhGAlh%vuZ|-X zchECl>PAA@(l!Q}th~O@@K3_*$2fLWwRlMamH^W2Dk<5;9nI}Mm_A^PN92~X2O;C< zLCHx@mR>GxA`pKoY#G5jf!FWxL*vlVrrUHsM^Wdl%Q@YCe$0M*HNX8McU_7-t~9YB zpI-3z&CWrTFFHETs}Twz)Sec1kg8ssz5F85VAM8GhbUzb$igPZ*gKk-2%4)C$D_v} z%x2)<5Y~Zm;01*UlX&$Lz1~)!6GPCtE!tI(pve!MB;d2dp|T9!qIbuobMjl4m*0g# zB}8=xYSfWh#D2V^#w_Hh)h&OEwEu{q2!vP7^zA^`wE-j0)rWiNDWdMl*OwMxDsZ+0 z)~=ckxxkB+3 zi@Pmz$9dI&Pr*f#*d2c;qZp*SXiYoY+c=l13Wtir+-dqPbmkSDmQR+P-un>6Jy7sC zCk(lDNaVGT`qDxm5jkKB!76P?G%XA>bl-P2#m|=zP;ei}cv4v{i=l!=dQ$#Q?w1@% z>d_q-$4{*^4gy33&CAH)Y+Yv!(*paq^$cL2;CU)7&nG{t&A15Pyl{6|t{KN_al|8i zBpqD6Iu>MyixX!>tZnt~KyuvFjHg{a4CFpkh~;5^mLL2aIiTczsH)!)0#@)$&8)LL zJ;-)b&wYB8NHJ~V7EA$W%vGTLEbz1d`byC3x#Q~O76Z`zZm!?|_Bh;)v=Kc|iXdhQ z0UYIw`ubMT9{@%MtIbS(y+>MwC}-$M^+fub{855gRGT9NZJzgE=0S?a!AL&1&wa56 zump-h^!3dXCeK zKO^P0n4LgF#5yj}{r!*mfF*#eCH;?`p=Ar4?AquyNSz;q7huV|Yk?jz5?$!uJG9fq z!O`zagWObF`js3qaKw}|U?PQ|_Ge@P9m%A29Pu-V zg$(GdeW2_-Ts9G1c8?9&9;_XKnzMg?E0VGex>@S+y!A=AZ3n;|@m6UQ%gs}CeK(Aq;3`9YivFGjZ7+A#M{I}q+G;30+7D+~_%gq~-Nn{ypW_wR*0}5x z6W*Wuh6-}7w+)$q)koMwf2QkNbH}O5Pnt@M53$}|&j)91NXBSel!p=8iY@#o-X}&i zPh71dRBzQ_=ZNA)5Wq#qt=$kMuaWfoF$&8USZ$WDP>HhoT8PW`teF`h&GKz)Xm6V! zKLlc(WN(fr&@$d9{XuAr0V9Tjr_P>8$1mqpD%<|SWFm4XUyn?|uLn*rC~W}$khPQ54I&LRwrGo1J5 zHY4@wycBstz*_J%sQl|Y3zw^u_#m`(`P<=a#GKG-xe8wa$SSO}*6BTjAn~q6)w!wR(43 z%Ij}sC{%EW3zfWu**a>W2!nk2BOaOWi7=nw$m2*K|Lrxl(Vks>h|5P}R7`;0kbQfA zRuOZF*^#zU&&v~|0BH?Swv2%`-aIzl10^g#nH@MJFqYfC=>;xx21?Pf8iL}&?pUf) z#@utD-#q)@j|VE>d9B|ygRu5&Q6Qi4OVaCws+-z??4>7axB(G-V{Gx8_l5nazXOB~ zsCxxL-@s=zmghi2IJr5yaH|ijxEydI5eJWwZ4VAfj&W;=^Ji}CH{7srkl|xlxvc$- zpgZdP0L?o@;7D<=bS0rQaz_GXFJnab01_|*64V0K(2 zx>{wo$21{eNs=sBf}Zd`ZXw~C5yUPd3a3v&oL5>@A0PN=4!aK(rN~-9#Pv7Gcm5gs z)nSljC%H7+H?%LwGWFL-_yebg4o~OsyS$R~VV1m-1b!bRh)48lkHH$CbwKz?FL|3B z7zdoIsa_hZZ${FlA^`~R8OW)CJ6cqeBYYxfOPw`072^px-b6;-R)0~1m>qK0x0CYM zUJ^3!BzIe>^WBtDO=xWhn+Wp>By6Kp<`4p+hlXDBmI^xNRZErH@N}c1Hi#j;kDCn>P-Ts80Bz)nFu>rA zA42&r019;g=^+?3U7!sj`FTgB^&PAr&~O@6MC20pEamC4*I1grUz+LOc7Os9z-y)| zGY2RMBYK~dug&X|C$DJO-#hWE+I$6^+E*wEbbXKT zp&=F!ot1jp$1+f!3N7oxEKx{KAw=-lSXmKWCmzZerkn%ZT@3hJc?=pctb;51O@|aC zWz?D>4736&6BiMkd&*r9Zh)PQoL?9Q+xKEC0Xyx~Qt(@k@wOncXz z9hCd3ag_r^vMF)f=%9JISH@W%Qf|k`oorYAqY^;YKF%&w%o)6gw9W^|b0%0Kfwmcd z{;QjER23Y(-Sm@y@7Y_MI&Uz&imFvmnp#EVrTg&A-x7B_ik$UDC+Wp`1>nvDUj97H z#9RG50^!Y8nmZ8jKgvS_szO25$yS;lzj2nsRk-07rTC2{fNtr)FjER^dxYkq z;t7qZe1QVuH8Dh{U%^M0Spfxj5n%7L*f4dK{;YHbsIv$B4TgZ-G!YS^cMx*0NE5q0 z#avX?<4ed`e?zIEzFJ+9q1(vCALd100xPTzD6lOFS%E6M)%q`8nvI2!17;>daUBAJ zaMJMcIdz8{eaW_`#j9wrhr{66{~8*);oGY#3OgEQ7v06;;jYI)JQH@^z%f+)J3iQa z(;-+|7KxWT!yD*25PO8WoR$3zDPa-je#&Oveg9r~^m#ySW9dI|f7?Ks3+ojI{|BdR z93BK|jFz#1ILH|X2ZeRmAr~PD87PQws~NgA!PUfbFr%R<>Ph;0eji2!>AkT+ z{xlL2pGXv3xe_}T2@G^oL&cyd@CFeT;Bwhh>|sPceyiVM!okbndsi9!-6l^%hf(hZ zZd;^uX;_lM<}4_uh}!bICH1SFH{W$iT?aliuM@2j?JHil$5C;nb?dnCK@>|ze_{%; zq9a0%9ZdW&nDk3{_#M3#AcFI8TpJ(i-h{H>uaffUHHgo1sRxV#xRP&xQ9?H26xBh0>IpKKBBI*A!Y>bUV z$Z7rEu<=5@UiaS)8$X*{lmGXydGKND{BE|T@L`MmZnksq`h%)9|9jYYAuB=c^lyxY zn-KFZ|DDln#>Z5OL#xK)u=sWz}Cb(y35TjOYjGPR;rl@Y*wG2x)It1ug~r(+2(e(HFmH!($(f*?`U0iy1d@LD=s~_1Mb? zz787K!25a6Y~XJF*|#wna(u_2o`w(@hEE^(2ZB$a01Q#dI|}yDaE0(<2A<3nI)DrJ z$0jE`=qKI3k5|{ngk=L6_a;=w1(@wvH7DSFFV|(w-}ZCU1N=ARURCMiFUtJ*z^~zL zTL?4tAv+0aLFnDoYzWOD05#RzxwLh&^Hv(>&qw4^4q3hun&iX2J2{ z`-vGGo!J(vxKcWVnKi#OvA_naf@f~m0=)qf28Ai%CP~rDF*|XSw*1ZNax1g1-T7e! ze(+fX5t!?_b_b4`jEsz98OX6?lm_(R02c1_7azuy@zYM~Vin-%CFq5PJH()ss_u66 zzdG7n+;eDs6+QdeAfMsv$K587srj#|4|2a0*N6L|?F06`0eSd?i+=y$F1)0cKJPac z@$EHCv;UVpRCXPYd_n>ppA7>eqpY}0sD6jeQTjOrol@oH8Rx^;;R^oH9^G!Q@K_ z`9DgK-oPq7u`hm#Hw~yt`07Ny39?`>TkvuR9sh&eC3nQ%UR!~zAqP@MAa#oqL9lm- z!G4AR=)L$K6(s^k!-pekeu~&0#?hyGW!0UDtBt+S-k~iSPP%>npsvu^&c}ZYF16im z+3wbqwtS9L?O1DNj@IV>CbwIBq&&sM(4kKMj zTxrVO@5o|*DpMk@UAy|z+b|}}z4NpgrNPaV>SanL?S3_;g!j)T1pbBnG>dEZAc|b= z+5I*g=bd0<=fl@>{y}j05-X|B*&CM5`{A^}-Q!2R??mR~%&*cC_{`wX9s1pKxgSJ` z;DM8LMDULLTuu!7WC*$jz%Gjk$K39JpKI|Cj$>wL0Q;Q(s!_7Z-R~Ib+h-Mi|7g6H z`$Y^c@%_uA|KvV8|96l6`dP)_Kl%XI<9~?2w-7YD`e0xTnh2YYK3#<)J%K;{d};S4 z8RkTZJ_Q0{SK{9bRW|f3SAocZBJksGILr*-dod6D@z-P_!2e+QZMY<;VdL{j2;kd; znUcU_2p1mOQrYcn0A1>_Y6A3*0KVPamvKv5UI&e6qQ2XBh$eT0z41sS2ZS<8(Q>C+ zyu5(>J&S8oUF~g)ttq4^$Tkp!2!-CmjguYpD-W9q7JsJgK$l$Elh6k+{tW3|_k#l) za2gyox-~dmNcJSYfux=!ndPhCWES+gzzN)Stv)qA(5T%RE@_1AQ0$|Gub+MRKOT)$ z?-FDm{*Om*=RVr{caP?Nw(Ea98b{5tUH{|JtGSOp`@2VTKf4yfcfXquJR2vw7I)Ua zOvnuGqx+{vDKaZlOtLqBN8AEmUqp3z+MOoS{1w_{Me3AO>n%^wia zC1rf;?KTFU^MAJV3t1dy%s&gXt&P8I=@+tiMsEID0C6DrV0s8y@b^oEYzThz?QDaB zvMVK$M#=lWqqs}B+J;40B#^MGl_K70-#>tJ%dmC6v=JXx8MONj2KB86O`q!$aP~2J zJ>Ue_x|c7PI(}%2S+?`3??3J7-{4lKf+?RI%76A!r{{NW% zd>kgcc$x~;PXFCiC%D^2;nBXk2Y=mFC)S5!Z88X~fAQ&F6ItP2i+ViqEZ?*{XDh+` zld-vDXuXt1dy3T|(pdE52k|(M+=@th&ayB&?`?kc*W?m^JKs&wO$@v0>1V-DI*JU# zuF6xA0_EAgwGi6;Vkk2>SQuQ&?&Kf7xgcpcT~tlPSmD|73bLDnc8C{cc=$YTfyR2$ zXq(c*0bx^-erpXsu>xZM1at7XKm9FF2ho}=!pG-Ep1SO(aoXG`dAh1dX*%kZ9CUpv zx~{`IQ5eN__H?{;kHyhrU>OFhtYQ0uuid$?|2DZahn6`o;gmD4>sb~@T4b_qrxWY& zyPQ^k58fkdnlya6*?-W5 zso2bx{H#98JA79oJGmK11n$e-i^q(lf}6smLL;J^uhY9%E_FA*$s_KYqthUQf8#>D zcbdgf6wLa;8Uwa}{FmxpRBUa&Oy8cENhzJImzpTfdYseV@902y;}wMq3@+~=#rpCh<^zm-&bJ8&IHRVJDo0!d|%0hSv=sk16V`x zl!~AgqLJ@A2iu)`oA=s~bMr=JOOH{H6#YwP?jdOy@?;zIfZr!+6Vddtt8tXL!oG*u&4)C4x#EZZrz7&1q%_dgP}!_kU$t-cw3(!E@&_AUtxO|MMa5YKsdGlsHz2#ir8UU)cJW8c^7c_84Q3wPG%hIJX8dJ?ocIY_BmHEADC{gNxCJvPmAG4w@+ZJ~o?Bdu#fsJ-9Mx{}4& zbEL^@oL@#{sQ>mewC@Yt-VxW0z2)7%56$9-xH(2F!);?V5Yw1JkQJNmx!TcQp$HXOA~rOLy+x13I;QXK0J|YiaeAPtcj2c6ppn zH<}LchLH^YbW<`5**yuJ)4`O%(awBE0w=_1UalpJs1a&hkdsqmU&-#=_dWuOAIAIQ z$qu1X70~p0HsM7eeIT7ul1@>NbQN6ZkT<*wzT^ZRwi3wv;l!DKAZYOyqoKQfaLds9 zaBW2sJ{^&<%dDoI#m8tH=T4xZu;#_M_S|w3zg1gL%vy)cir(M3whwI;@OJ%Ueq`2r zaVEY& zyeV={m(!&i27PNfN>Vtd@@NA!Hy!Y261bNcfgW^5FD*Nk;~KBI@Wo&95=527 zGBU>0=_3O&W}D&<`<<~YZO{=KtGqE~Uf9H3q|d&%L)RYGcTUH_1$)p~t?Ii)uN?aP z@u4xqHPnrV!rHG?i}L_sF=Q<#$MeLt;aJi2Fo!d_2Xs9?U8mEl>+6>TJ^_LJ7c(}e zI{PLv9rhXw#wSsVKRAhxe^$yho1!>SDU-2=oyIqdDUeYc5T~t1Q9@|k`>lkT18!NZd%noHV`(~(Rx^ORut1@BgmHHfzH<$Ci_Dfv?rI9 z%@D8*RgDMA*Ev`6xY}sTmFL|EDE1dLFZR#ql6TtOsOs$+;?H?`Td&_rPc6B#zot^a zaC2e$Q%MnKn0XQS$b6l!(f2N@9WbxAYsL5ZITo6fzI0%l*A>n5&{(R=89sIuaOobU17OaHC*y3zdzI-V%o6 zty+!o_!Zg-nL-E0L+`j797fAu?QYJRek+nPl-5!5=tk4VPQ6X@*z>KyiLJ5WHy3Fz z&q#dC&XDm_d-7=ifXfCR3)#le*MI!ufkr~OB+Wg#z9Lh>cF4v!hJ!m03s-qwbFCC1 z8!<7Pl)}7sPsd)6gql$%(+y3!@l%XJ> z<3(4v<1OXm6OZ<^6!IBJg{%$;2x?1olP_b~f)U*Q9Pzo!o=3VIHQ+_$Rj0ihQp+^-8q^ z7{eyyts^8C%n&*Q86~GT>mrt3v=<@Lt0z173kBeM0l?*rdUKXzSYRN(xvyL_aXd-3 z2a2j=4{XrVqFvZZtcw_`?299dK$PohU?9}?D@_l*h8_}-N|@P(!R;3oc<3_ZFYaCH z*`7+qSbx%unX(r`&~#UZIfhb9+2MLdv3)`ML(0hnoP1JXpPtCYx>9j>xvj zBG%|s?7bwZVtUc~;T?|A*5Mbeyy1UFex`DEZt+V{2ip-7dytm4**V(;l57tS=&6&# zP|guTQSi@cXUMHHZiDA{EeAzq*K*XeQ%lRo#|JG4pWK#Ls*v|`YV)FM|AX&M+0D31 zSyO_n>zhkCf!xXTyPU${jP~am1>?mj6P|?#516w`s8#^2CeAPcaq%tSOvrRx*vnQz zfak^mT_bcUkde-pSZ~BJcpo9*gtleDbNZ9#J)Mfn>$2?5T)mve`LuGz-@72@NhSNM zgR+_#eRP+W7Kn9kGRso}fbLCfIV6$M{wd0zndJ|4A3N?JcyH|gjylQ|Zo9k9%ULUL z?EJi_Bx$S`Pxj8&AAsE#8|@Sz%>4lFZxD$2fE}sv;i3)xirtS6&M7TbS5|u>LlSDt z6}_^co#O=8nd51#XlswXdLuWczrGYXBq+L`V5Fj7&@vVq*@0l$1dTBvZB%|2wu8yJ zH=@AYZOb)J=-8M0SmHV{*U---r|W|n5!y#_415O+9yJGFUW!4ijreN>Dn02%)YnjZmj!8%|Oaa;@4fZ0`8B zJ=|55FsB9Xry1tw??l@GAl1?$?@%YKrIJSWUMY!U8TJz=T9GpJrg-bkcj4BuQ({|M z%21_MnA&pK8+{6@1Cn`A1w~vnlb){T9|kd3{cQ7z;oVGH=|pl+3a2hkrGz})Y}_$$ zH+@Mz^a?d!VSJB6^^i2Takv^AswS7}2WfzzaklCw=uKHOe10WiKJO8%dTG}!1b=5a z-kdT0MkzP2~r&cb1-Da$+|aH9IkK$Jan8CtLg zt_cmCD459STb+l3*MxycEIAY{h| zoj#gedsi<`^$%&EOCHNN9X*gXXz>?qU@Gq~+^-SwFlxvKt|M?Atpi7JxwYaPJTMJh z4P;YGw>Fq80Z|HV7o^Bt9L!&u(O@PRUt7i$Z-!GL)3F_{_$m40$3fi|3Z+2Rq$5Zi zijM<9m+|SDbq~&0MGhhPVbw~f_YZa9xPX^SQcFbyLFz-;qb1O{5}g|sN+Xf#=-F6jZg6vyWeAhK&N_QNEfri*_< zJ&6bcP1ED(G#7CC6`O$``A!It1VWE+w2ckaA}dOUOGQPqO_HsO1sKUPpQ5;~wJ|X8 zL!OrlT=)ZfO`kqWVJvP_X3=Y`pzFFY)pPRG

gH;O8v3yIPNXH%q#a?#J>@=z(m zJIduLA$R4imscbm&yau^8|ta5G~WsgS6K9~RMny-)mdNtfP*Hp0^dxFCTy|H0k248T?vk_8IVb%<7j0$=aUnIlua za6UeV!bRdag4*rh?Ad$#ZP-S6j3?Q5C30_1IWx|%7-J6Z@XUpefyV*Fq2VBX-v?`N z$%0(&(L7Hl=o+)Gb^Ysoj;~a4;Lu>^BL(Ch00mIT{T=rlefG9SvvuNz$D8R%{*6&UE1Xt2b>P5IL!-O)V9LTub~Zm;~e8L zL>ug#RKCxo7E zuzzMhgNRaA{fCxwaIKn8 zAFcFV(J=g*=80tp}v(G~*6W_T;`Mh*Uo!!9p7=?F3-(F8^{X(x;`xk$2 zn%Vc?^%;J-m$RuvZ&DLaTKs(r!xx%)W4c)=x}bO;El-6VN_*Y_J@QYNMML#{JC>|d ztfY{X<&p3&l=tgq&7~jsf4i3p^_m}yJmbb^#;O*mgKYp9n7po2th>w_cJ5;j@**7} zR=t9bh>M*NN#$u)(lZvLPQ)?y->Kve?|qGgd_J#mdKn&U*)hd0I^+jCIpAf5BNl34 zH)EI0O?cP`Jbf?Z%}@zdjlPB&c1j^8J`|Hw1NC#GV(9CqX+Jt5FcP)4am|Y!(Tu(x zGumi}j7TuTiXHv_WAT$^hZ{<6HIlEq))C$s(}01BmD97AgY|qY-2WYgNx??CffQ6@ zv>-i&`2U-=A22@=d`w9B;WUSM$&7K2*MZysVzCMt-DDLA$XK?Y4s#e8mD;n=!7 zL%;WDo#?~jU_hyH(SWKL2{UM;(UFcJXiHVz^V9Vmdan_jxShEUZ++`} z*}%btG~jJL>SSsI1}XIgC$6RSEGw%NYWf!Oah9}Y52U?Y=vk6^*TAXyME5-;!=%=*jm8#DiXZFU4eo*H$g?k}mBpBj2wxxShMliZ7SwXRlu=C>Y!$ojZ_*kl-dDQyn~Vj<7J`Z^L+{ z&mWeo5o=@6{^5ndL8H8NgdHj|6-Sb#g zuWw*+Qf};!;Byxmwp2@S19A{%3IPv`o%A>MN(cF>w;NDSr=t#H$nAkmtnX8p;hmH_ zR2fDyh1rUq>Uggd`$R=dESlZd<2hJoUVN{R`D#Z0n4)8z*%pW9jP%FW@S7}pnSbLF zR|0lup0`wD(|j8{3Dk~&IpQ}Z12ro8=c-!Q$p;+ropQLG9b|Lp2Y$AK<&K|Sr0Qu#-5)4)mH&HyO9kW@eHE% zX$QDd86c5x7WP*-1N@DG7p3(q^n--2~9mQx=EES_cn~m{O+Qw z2lNZjh6u%b`#Ew*fFL;GflRFo`5+e8Sdq)p4r%CXS0amt6SRW*eAo&+T9~H{&d=C97l%bm zQ68Ct84K}7lt=et6>5Yt&xJwB5@Aa&hZO@_?XZ`#3w9-H_oms9M=f=Wy&&WCX=->D z&rKQ^2i{4OG&6Z$x)=X#*ERTw*6DoeNzX@^)o>POcZpL#qkJxkZ1y*WDcB$g=~Q`O z)d8jO!I#w0Nt6?YxK0Ob2rSW9R=pMbB$1&B*52teYfIs9T&`S@s?!xQt-hDBMu>kT zI+KUSgfiuA%Oaxp7%GJarlzI>DY=;t16Oqu;u1Uw52rk_OS%Jk;L#ahWV3dF*;olE z76j)ws;fmk47qtClp5_PVq2mV)*mE#dy2UZ+~L!O<{!h|0&ENs2qdC(?Ex^G zl)%f~&IU>h)E@kS*PhaI^Un<%1b$)|IQ8`+ONH`_%JEY)sT3O{&5do4+H*Qq_}FO# z-s66;*jkZ!8<8#1{GNvJolq<9OzlF21#Ur$$AV#=6lAs_?O@OiHzgYJ7|~e+ z$FhPK3Ho6Fbth;5)KC{Ecz=tX2;c)~f{-=JB^-T9=;mL%bDyULN(FI(9EA+;hs{Q= z5up_+I_k0j&2dOuZff0tKDDQle-6gi|N9YVe+3)V`Peq<4$wfOJ^7vs28!wN*ZtmB z{!>c)^C{(@0>OD1SQtb^>6`RM-SjQ)?bzv0iB5lNm#yJ+l*_|F^94~9kO0khDnkqf zw!?3H9CTxN89WufE{%}qy8n4j3qh&^ng9&C(MbZqsC|ev1$dWF$FJBAYE;r|1aqlT z=m}>8wHE13Cm|4_hQIm?r=Rf2zn@S{9u5GIM}vR_DfbF4oeHxj#)eW9D0nx)NG74N zqp-K&?oS{C!FxBcqd@YH!gD5}EFfj<{Vs>d(S- zxFx8k==yMHaE4&2CLn_?tw>@=D9?uj*4 zygHe17M6oY3`32+Le~7bQ9@(;1P|E^{+0KTn-Deq=lL$Z03nzTnq4G=Zky1UI)CG= zp70DcZ%Ptu%)OV8>V})e;|2ke0B72n!x}Qk&eedjC}y* zvAad!=(V~aII~{#uYCsN${P-8&O3=c$}+t zy zXQ7lcQW%x5RxO|2i_K%IOU#6H8%7GbwS zvP2i|3yhYzb0l1;5WG;Z1OH4BU-LM=ZzN$AOm|zMnLskI$MY}q!zL8f*N#qlN4Qk! zIkjUMQhd{sr^UC91HQZyjHZQ=$v_*2_nVypZex-5)Ybp%c(n3a(G81wT{6EPk5qPS zi{PP;n^s^5AT@xJFC`X`K8F^eQV~AC=O%v;Rvzx3$5;LwM7{^64Z%MR;~f5hXYb}Y z0+>-{@sEbFJs2Jy$pYw#{KLSbqrqZ; zEFQ|(2tMb+FV6Rf@C+3R4gY!s>u?**aYK+8@_NBM7o3#Cyeu8eyOIKfzehU^S*mnU zM?Z2FK7&P*cF=G=zylAyDk>=zfzT<`Qe^;wz(4ye;rI4Dm4pX^lYJniG@ReNlXnlu z?aKfk8CzLdotVa=Ian{jGj0sy8!Bxcj=er@0_0;sDDI$asPv@J--5^(239KYt=lw5 z^(m3(m0DoH=iuO=U`611plogoMAhe#KtXK{>M!BlOwk*Buf)_-ij7(ig>h9C#y51o9{k9eCbEU<;dQsmWP{dzXb!xPRBZ1P+0$&ug;v<{$&2h~x? zIHIMH7Z4!Fl6mm!#miKW(QY1!5lw}H#NvN#GLOd1LsRp22}r@BL9pnn!0HT$qSMGT z5c9w`HoSr{fk5U0Ovkea5JObzN{Yib#K{U<7hGbMI#Pyoi9Ieu1se%|pF@lqs0vMZ zM%Jm(>S9mtfET{b>?+pxkRXCWOi;j}IwsIJ$|I*eNza{19`2=fHk!J-HX0Z$w{rAP z$gR#UcV6q-vtD5`mJ%~y@8~~v<~;kFL#w7~ay$ND>_V;d*tS-?SkmRq7t|70!AD%R z@vmZW1?y>kt7)2v?Gm}k_1|t$j0o@2aubb;`^)RUwbj2!U>KOzU&UbQV;aryU=33P z{%^bE2nRaosn@SIkAKP?_k19UI~Aeqb~pm})!3`fbY@RE`EA zk{rsQ0>!980M=#*a5T`UdC}^&+#tAZqFV)yA8pUMYM zM+*IVlIV8rbvXBqgm~^?T5jM;f=1Jzl2^IyLCIyKWs@-zKUT-lRVJz*v+$e~q4lp4 z)>Y+LZNyfd4EuhY#noJ+?2_|jsvnb++Wv8AWMug+Jl2&+XeW|BkV!|}i(5^%#wF*B zyGGgPPTE^|(S$9v7ERV$$$4PwqSUvo#QqSD%u!;o=*d);Gd`*qe`V6C#ABe`4m^=G z{vMwun{SMO#)}EbFOYU=0pc92+3|lzcW_4_oofVcN_4pyesV-B&EwlDqb;r7Gu}=% zu2NqB1BA{Iz15m; ziTlD;#jkWF0`qgO+K}=Wp@lm~98lA!O+VwDE280PGny>trrvHDc5%~ZIB6|mt&mg7 zx|H7?mhmqBOT}rU6EeAq4t?jtL?8y7cGp)G!3Uj!3~@17d4)$iPA`prfI&HCu|IcR z`08gH<*~U;l^XGhIx0@*-_4<1E(b&;k>fw~H4UGt2u=0Yp`(ep-X6IQ-`WR*2$+_B z>}H~icQHPznfuNW7Z;m4@z~8mu6X;YPaJ1mwlyp-kVn=$nX1A^E&HwD_?zF6`(YdL;FPEDi zhViJwVL}3pv{%i`t#0XU8KS;BJEq_?q29JqZpc%nZffi;GAe3Vd)X;NXP7hb_f{`f z^M+_g60t2udxp^LUG9*j0~B)aw-6NEtMO^oH{3%I1Ltu?f7WjA2r^*^zrtPzLTPFf zn9@xf7*$_dZ?CJ*A>rS}`zhR)0>3Q*g*-AHTwBTEfUho(0f z)$7WGCADgneDD9#AD2cmf?VzZWMa@VS*wKq1KG=W^82>=&;x3K2lwyaF9<)x7N>a4 zk}%O8xft3!BtG&|Yw&w)v*cofdhW}M5v#F>Ph_*gJ_TsKc!^^@BGOqlrttr#+=j!bTXgwDrirIWPuyY#rm+e6gvz zQ~apLYw3?(M~sKHO{K~%W*9n|XH5#zqdkY7hsMxDLOR<#6o;9iq(IkEZR2;JmkQ+_;x}Z-`=BrzXga^a zH@_69KfpZXZShdaQvrsEAj+TrlJO|c-vrltx%L6_kQcMv@7bpJT)FjBl}q0KDS~Vw zQUa#viZ?T3M(;+~A0H~ma<$ElmBZg<{~+cLxwVsT?yHT96g%&TH%DIg+<{nO^4!w4 z4@rL9Icg_ibAx}BG}R@FywCk`VZb=%Y{@Oj?XSA5JXMFl;95!A!Jcj8HuX!M4pP!;O_DY^7 zEI@GasHMnlWi7UaUeDRe2a8<1HCb%i<)!FhQ)%9=d1YrDrIj4E#SgA_d4loiwrb{{ zBTE0Vz#SoulK!C(cO0p{zAJ^heL>ShNnFFdg>@QHas}g&q@huxXM=&U=Cgzj{$x43 z!&KV40M3G(e!+q#-~bQl3$j1JYm$H(#YKdEAIV`Pcr7I01Q(;5e`+GKJ7{=O)b;Hb zvKjA#oSGeRYa6!^heT;sPR~gT&0Mg+L8eev>6R9J^}+apk!k7blko~-WSa!8RVogi zr@cJ><|5#-cN>aICXAb<*w%-0YrDoHXZ<}s-{^?ia6|B9bK2Ana1Gyl#&XlhrnJUg zKb9|1NH}XzgXj8+QvhKUciUnD0LH1TW7BKFvB`KujduZ5p;fxUHxCMzd%XJO7e|>) zSuW!lvEVPViAJAczZGfF&u_FSNbkERJ@m13I2z;m)Nch;dSRbW9XN2HBu*AEx+*W$ zJBYG@|B>ey%R(@5B!5xAsIo(Y#d~H^n{@X;M(5|N?^0du^z6%P&DGZG>RiE%7tZMK z9kU7A9zW|n}-?Zx}AdG7zeVRm~+3DR$ z*BJ^2NQHhkEN@+W`BG!@Z)D7(n*d(u!g@CsZyf)6WUZ2J>7e`IXn9c_(SW%Wtjk=! zbrH#fxv>5}Lw2Fg#G{7OsW#^Xlf=8f!tWk;WX#q$Vkb?-G2Cx=;_h~rsdY3+h&#V* ztnZTn8HoJP8^BUTH_^2x&Qv8rEsBlM>^SwLZD`GfTy8+L6DOs9(V?fEk0jZ;dV?Q_ z<3el3aWYB{4I1A*ggmqEiXc3w=}zRd;l!P3^^7(N&+f^fT6`vJbRZ!sI7j1GhH!>F zBpIUNOvV_UhLBcN2XU!@hI8&#H6+FX*#k=<1OhQ;?d!Wv2Pmxxg#+Y1ecSa9JFF?y z35lx2r5AoFj&a}d0rj8#2=pKRdKU~oeVdcMa~ZkN9O{&CTPzHc{KY_Sc|-(8mvo80StdZ3~LR8Ip(Y}buIcAKebLg-E zXlzNwlP~eT#s?@6HpvW4OcWkSL5Lem&|XcGF>?2MP2v2Alu7~P;GUIP{mwtG7=;a$ zL@#RXChTdr;rwaoKEh}U_5n%}%(zC_UIs4zcrf(v0#gobofR*I4Z7o_qhdn&9|ssj zUA7M1ev8h;U!hNsaE;jD#K!}lhmYF`LmEE(9 zeJz6Xr}zR9WKBsJ=9U9!VT)~PoDW(Ia_0aev<9K!86H4GIv6d;JO*q+BGbsRpgboc z`tmGk&px;uK$6>tl9Hp7(^mqj&7s78A<=iA+c>Fc4P;GsJwB;0^@l0|%VW5SGsDO7 zF+i@#~Y4sqyz?D<8*E_I=H%g^UKD33m zMTh$`V{X!SjF+YvK7;n~@?3jIBiAAtiXByt9CEKGmIc*(5DBd@ zy)vbzXJ7XBFl!(t*`DdO;>pK;d#MgWRL&zcFPS9!)+Q2r!FhLlUzOG%qm+QLIQB(r z(&Ze-;lOUq2!F73GEY!og>Y;PIv;4&Ar_4oix(D}Z^kFRs`wyDHmEc89DfMu&4KC% z02*dz2y%hI0epM0tJ=>GY63-axMBZOAV&cuxz!Rr9}?-$J6Gtwh}ij}deC#}vEKNb zD=pjO*tp#Hjx5)>WFdVc(?6a7>el}_2L&W&8w;zmTIE>ZtDUOacl4hmI&TRefVbZo zpT-&KLbk+M2AmQ$$z|!1`&zGG1M)+r09L6e%vg7b`Vjl24=_PIeJo|{86)D@G4r>> zTb%6a8IiL?JY&-8xe#ZH!5&+dkvzXwo~kX$82ZvOhr)|kdc{z50`{`Kzc>-RaO6-Z zevmGkZ`}&uZSxD)5sMi@0Xl*O$?GmBVt!?oD}?_$(cSY9ZSBq&15>%jo_!^HX>~L} zb}8ivX*xQG5$G`kW6nzm*b&{mr#fZ#uT7hJx%=oCa%vPC_td$iaLVf($@ojHw}YFe zJ|}y=cLd|A4hR8)_i=Ol{?^Z`_^LRzL+%xml9WOJjAcc|`Snw$34d5a2L`|VoHs*B z;ACxhK@%dbXr6%lDwM`RCCDE!RN$KOgi=<{cxu`mIjSp_QucAneoNaAA##T5X~it} zm7;h2aPtCAK#pf%BD~x+W2XU&hBJM)&q${}LCp0O1yTwZYVPY1)qsHGT!A?7?OR$=wI%ORbw$Nus2qt?AfH;?= zu7$L$zw7@)a;No);(<>Pq&8j7qjf;P(R3p+zX9c*57g{60gP-h^R+^-~=VqF=2iX z_&1QtA9!CLfgCj7(zAgswGxT*q3W=EACmLt&6%>Yn z?{2)a9!d|OeoTNIyP%}ZZ(A=>)&qha*I!i|6Ky< znI%{XsIw$o%$}a|Vl4z5yHjZnme2;1el7{*>Y=M+C9kG}n88A0e4U9s9|a!7CZe1g zgQg0hvYO~RU=3DQRUJEqsyj}d1-}yi12_5XfpzjcyxEju#}gXynK4}osTA3*qXVT* zC>d$J1b2iSk^qh@Rsd56ELNH*nwW*SKMwO2hAFpS^tU8UZQrm#f?e`L6yL21#KTX| ze2bRQ^tZH1U~7PFe=F1~z|U{c$RRMlZL&<|)l`F@pD1?f4x4#OPLq2DdU%+3{I+Tj z{f+)frPETkaJ}y%dh;-&rkyH()Q!POnDQTHlJMd-hHGIuSZ`(V!R@tp?%_B65BFtH z7OfpF>L(Re8)>ZBLe$b~a~)G?%g=8ZqP6CJ&J~?p6$Nb4VcRH*m+~ID>B7w_OBdtc z!U@ERtEs6$xj^_;A*c!NeF}^ou;Lqu<+=st%9s@B!GNf?DpBEM_SjUiQ?F_qqhFyu z_s4}eI$DN@m?VD@+N9gSj_j5$PE1vCfrsSe#jmV=`_n$$-Bv7qZ4c9-b>!%dFvXN= zt=yFJ<^_`YEhxzZq3>pyn73l_1OWle+yx)B!2u>?i~ivc}Oc?HPSK$yZWKl z=FNwrmQqg`5nhdIR}I>@j7x-jb_~2^6+4dgu5odGhpS7x=5ZsF+=k?&1YU#jE^?F}5Z#$%jHU5DfRa~cwy z`)CY3JPy;!^3pLS)yv^+>?3y3|FUA#hKzcT2BuE*zF07v_%M*=O+X)V8k)7LXd9)- zJ)?9Fmexk3I<+>bQ*Pff9lM0_$w8Wlpz4_3{lqT%?c28(d9^Y`;)dRsX3}0Wv`5z5 zY3Ya`k9fBY`XItx4CJlr63M$RZ!hr8%ulO!1*@L1v2k=2p)* zWj1ZdxqqWJ%#_d}!%iwVGF6@OKT3~##Nc$G&y{q?pj=gRO@GoI*w+4GO7`SI5Kfrv za_V}Y+?gt+JTR&_Hdd!gnHO4qcb=j8_8jNtyEttc`bjsk>I)XkBl)4f=A7>hjV6=H zT)C@@NJng1718HCm+$aQ@A{B_relRu;rnDY%ULg_;a(3%Hk%aOqN%Tc#HQP3K_w)c zZXp+@sF)PyX7b*_<0Ym%Lpxt3zp$O%Wjp)3VtVfln(mRV)x)>G*wTA6am~SNJ3n8t zEnHDucGr@$JUC~E&`q=dgD?ZlQKPwXyn8LoS1Gl=7ly3&^}@TH+)HA?sc@yd$E#NJ zmSc5vLvz{3FXA6=#5xS%U>hAA_u+p;Ic}+{2f~H>5ev_|C(Zhv*$=zvPO#3|arXbCcoKerxBGW6=$0DjJm`Auv z#O5K-=Zi;Q5SM-7Fq5&K5wntCjGJ%O6^(ONpC?z$;JBX86(<30U=3$#Wa492vL~CY z-kC^c-|b88jMwtU3yEI(KMS5S#n8K2zu}j1MK^)Dyz7wpnK}vrr%znWD@f=YIta@w zODOY?UsVB_d<#T2?Q&T zlZWAAK;vmOw8p_x{&o6z)&lcEcoA2;Nwxht>%LpbS=b1_G!whtBi7yDoha%-SybYX zRa}}(Mu*SsMQ-F{JxV(iwqp;YZ{$7pIiVB2C~Gh)>)}f`kzJYdtmxlWPs!W$;V_bC zB8FsG{pI7$jF|AXTJ83KIxxIqvls=iJ?K;&{LVP!)5b5_nlAlZLLGG)tJFnf5e2>) zB;NQ+D?hZs!%Om@QHotX&z{*UKxZna;1u^`xMi6#oO3DV^B926X+v&0NRyx^XFcO^{W(_0iiG$U#-BDm{ zg8PT7kF4#Ch+j1}^l{M{$DB%&YcTbDM7gf$^zm04AL>6G7_BY0tnFRE33SD;cI8fh z)&>xWER3-xaW_KHLL(b0cJ1P6WA<|@F;Ex(ZkvO{#J&<7EadubZ>F67c< zW*%@#FHU~YjI_ELN4&n&{J=Uug@Y5fa0K@!>2BtuqCM#ns3SB|b4SxH&h zJExj4T<~hN(;y9(@P+y`xj5t5_}z5-)<;^Aw5xwoTo{@Ox!!bVQ&%RgFOb%f8eN?S zjiL&bKgy37JJo%UW`;YyFH9}Qt+eBI5!J8+5+@G7)iY>e0mH%P8Bj@!(YQN=O0FoG z?=}S}1bo*ZfMR@v;FcT{xQwyc?V(Zl)N1RZ74;UYFj=EH4#bUD?mwUu30 zziX;4H2q2&yUotjPS_UDwxgKpUpBRHwj`Kre5GP{O?QF{mzvU-J89O!PNsHtN_R7w zQ(k8DJqd1_V`J2xMsbfze$!kt&z1oL33fVoTbyKc1v8P6{O&q+Gb`sbYD!knocjSe z)aV~gL)4CY`xgz?X1J{=%3wO?Yz|mf*~{upAojERv%5qXE@reVim_xQKn7mkWoC|+ zXV*!m>L!uS8-t|5QI?ia;s2JpKYp6X+JaVWz7ikP-~2i?S*To=WSNyp(EaN}OmcW8 zDZM)r+g3qt25?pgdue(MgKu|V)S!wGNHn@YXc%ew9K_V;=D63-=$F;y}R2CD+Qo%p_lx z@z*>ST|q4wyi#2Kg|<~|fLt=Fbxp5iyXum{a!0TdaeKqcK5AHIyc&|`y(G(s^UA1v zKb&Z_!HyRxi^lBIFK<=vwCqiHn6&5Z;($wSeALBE4?Vp^<@PMXbyKw`N?I!yUBSy- zUi%20v0F|;%!^QX8-}*XC`t z*_-mw6RSWu1^5&%{sEFre}d$_L>JeMc*j44d4S!`teyUu#B{ldcWlY(OAT&fm8+&v z`bV`!DksJhmE8k~05vsI=#|1tKOL%yN@y5x>ziQEnJUXR8*Q8OKAY1Ik}J8X{k6eZ zLs74$8S3RP!b4rwXz{M%o6|Zplb}XPFRQp#=28u{DZh>Cnz>lj#n8I*Q3b;(YFaN| zbUJOk%*q}EM&EOcj2bE7`mV4{MxgD3*U(+CbA!VaSNfDYoqT<3Paiou%{Izgqt|3c z<^!1g2~B)S8Qo7n{T=x7s?HaXJV6rlWuG~O&V-84)JoL5WWD%_HXaAVBHRN2k4oRm z_Y18#atn^efivNN1J*T;7EL%UE0b+>%xBOf#&1NFecY=1kiogY-q_}PLT|8e(8$%B z-D8A1u@gTKc05zF+n`*#Ii}cussq4a7NCvwe8K1yAr;h8#c6gQvwb9TgezWyfPpkm4KCCfyEKOw<2Nuxiam% z&xajw&)=Q@(phudC=jxfD@$wZ>x*Gd1GGAx=8cySnjO}jhmhC&eJJXl`~CILh>=-} zOx>kiIKpRbJG!Eq){=6So{}A~-;1SEq?0m|E=IsHjxn`5xx6KVHU4mT_!iN9_Z-MzoJ4F%Cuq9X1$*Ga%oAD11DCNAf<~ODCm=cmE}?q> z5I><|e{<`~vG635?1s=Xhq89ld)D~Y)bF<;+a=u@!MIO&G}d%F%=ehNWaF`8>SqQn ziDTST|0=VlL!kF|fTX>4RfOA#E1ut2!kp`Hu-R`(*5AW(`WAKx1x}n>~0& z5cCY#3b1S?K)F*V)IC}zP|&HLinXs#kV-)j?TH9Q)!u4Rc@$)VS$qF-93pbEmD7cK z>0HqMQty5cL{c~3ONH|*{$?~AEUkil-E+$^D?b@7XU}d@4*7e^2!twyiT$Ys+bFYT zzrLfoHrXFf&G>@L8W~$gcvQ35B(i?k)%HW0$dHwDo{8%88Q+j|y{lM1Y4jw1rxe&} zw>>7C`i$yjauZFP8JogTxKcUcj3H-i5}K?nFC~y#J=(PhwY98tG8y|RX-W}HsFNXm z4C|1#t+|EzZR?(;zqRs%$i)F?*jU7`Lz6-GJ zLm^_YgY1Kw8NPSY)@wye&*hW8H!L)tgWG*jgM#H2bcqJJgXWGw9+-=w*Y@6sQtN%s zcr^3ch@3`!|Kh4$Mp{6%skVD_?~W<$u3c6>l|4ar0n?6PpA3pJ)h{FHo@g%7=klCH4%C-C{%*Tr z$MdB9?p?)^6}?`LZ~it%mW9%HTl|@==xZX!<6JWz~goH$f|+0!K(xvXF_ zcgT={SMBv$65Et`+JV^un^N*=bBgcyZ zX)d)7NS0o%1cwIv<^CAZLbB8E<5Fo{N4>%FoFB?kDe&MG344m|vaG}_`YT6ndZ{JG z=}M|UC1klboMohVyb`(b4kdJUpV)=cXb{NyUk0BMeJ%?e>l%n*Gs=q_>H6y(PF+8s zM<82nvrE&q94j=$c!9~S^O^nh_A5XMGLvoZ8KaetlV6;0Uvs;E$D#U%QM2V{^k9B9 zkN*aj>Vd(8RwIjFd7gh3zUlz+GNfy$o#ZfxG9X!Lt!1G<1*=Mo@d5(1Z zk_A%EVCrOGtAW>)1Kv<7rRjDq0b!N#a^1^GI~r+RtYf>h@%)oS*5yk_fAl624nhj1 zb@oQI%}Eh%CVJSjwH{7oHS_fQsE5f_5=wgvZmDEP==;*l_`Ydcm5#@=>cT0>p=CYeYH85@ z4EeaC;<7d2P!$b01srNZ=LlD@`%hJm3E?1KP9W>+shq(hxS)TGJ+ZV6_h z^dNxOg1?jh!;B#d_rKbM+N7Td9n9#Fp5h8#&SW zP|`n_)y-ha%&%=`=oz(zld)y)&7{Oj4FL|5fw6&t~4wLKO!u#4hoM#J{_MNYtaRnopCz( z0w_SiSjQzi(SuoJ6wlDYIWnAg-T9Giv23{Uo-Q#HD4$_Tu^0BN{zgvFHw{Lyw8Wqn zxHbhy7ev3wM`;C=9glN)cef-;&3w*Fb$DKK`;CYwD~9Hv_@7~;y}5pL1~#qw=x4Op zYyBFl;THN=784t~yiGfZerZWgrq9C4Ai1{;Eec_u2{z*(kb2X#a(;}mT87}B6~R9C z)V@VH+BmEWMPpq}hanPceq-+5L2R=~Kw!4~L$GEbIwG(eQuQ?ON)*V1F2!f|__Nre zWK~kD=SbVYpmR<$$07}`Hm%w&BejE;HNRnOya}0OO;3zaMJ~#1F57(A4|R~D_7)03 zTMK$^W2;#lFJ39`hxW9BZI9Leg3uC3wd+K*q! z_DlO?dnElDq=(O<9AjGX>V-iNjVD2#0Z0{zGR-IqsHGNbfkjHcG$@Qp4Hs9;_W4^N zBXbv*6L7uX_ABtg_|5I;9El#|_Y~ds`K>HaVT+D8$}CPm+HqV(MIL1{GaU?Sly+=0 zTXAmP6<+uO*!7@z>p)*Z0Ea*_cApsw^+unOe z*<;VW1coZ4CaiJ^D7ljET)m9Frl@fca$JNE^*Mdmj*-*DC*({F*UUj#PUi9^)b$Cu zx%y`$RIxc?C+XK*XWvL$K^AJjgZ=!yCkbG!Q}q#LK^NZq)go*TBbSS<;`IntkYRQ` z)cC|HK)So$`#hA%c0#hFK_pbps){RJSCiTG>{IR!3uF7K$JBv({f($nr@;(4GTfA8 zN~t7HT_jL&>p&XtSb!I|fW1&@02dH7)*5b~l?=6s;{M#WlP40?xEsay-gC91wUC_e z@+y2*t>4-jN9$KPjl zqwPO6ubgKflT2}-|Dc=Q|58-B<27g~^%LU8Cltr3MnSJMRXZc)dq#kt3IObau<9aL z4J)I4S}3%U(N4F`EmMu2NU3=URka4@Cqh=X5&+%BHs#kbxqpYa-3249OVUFnbc0t)iJES~l6P}EAe+FM>ZyCs+>1Cdr7Cz2ftN;W35X}+|7yvNBW{RSG)=~X+ zz8x^qXd4pa2=)XEjD{VnVZWJ9*<4)msTYk~%KHQ}ndEm)7v3s%0X+?d7=R<@buo&5 zgt;(?stJi!B>RRsNO=R?>5n0A6lnVT)P3HVk@*|{v!p)VkiYmCC>sG)Uz`FBfxHI#SrAIfAA=r45Qf>x zoZPhmEe@C$Eyu>`UOa-SV14*DVq31sX+mw+M&~JSs5a z+VK!S%I*B4w;1Rcg3wqnuS&iE_zA&ATdW|1z-ud)4*-o()N9Y{rGy@AG+Ybh3#0hy z!mE1(_;+uzZMp0kL?GZlX*cjoHu9Eb8?V>CH#X$#uU>l*{(OsX(Qp>q}_pSCq*6M<4N1%=9PYPrUkG?aM2( zL+`jyfHhKignnBd#RytaL}3upPd_A3XAJo%3USsj@tI1>RJkyDOymbNTf7EnBja)_45;{@Jnx-_wOZ zY0Nqd2^&@ZEO^0_?78!QZ1pW(yM}1ZF+FAYu+SbZEE>?S)`CoeIC@Y%;{|;>?mtmz zZ*245Gu$hp`}xDddZhSL2lF@nYh}&xLhPM;HA7|Ay37I}7~{c$%93&{SExWQm|1`Y9fc zt2&5MsiV8^2bUgtOziCg(K>+A0BCWG>!m@WKkrX;*kIGjG)BLMK{=ojNpQeyBpvAW zhuw)azlEc-p4TJ6IU!aD^bepv%@L6O<#$b=o_)_M6xx7x2DYo9r!odb5lWLC38nX;>ED1itfJ>Fv{*vu zhQ`naX9x}+Uo}8}pWrfWgylj<7$wAXbdY4~@D~h+Ph5XmjAAV91mxbrb@z`xoF)}b(3o-W)xi(0fGx3w&miH%rXr#Q3 zJ$xKA6#6DG+owH)pqV2I z4G0sum;T?X9rHH|_OQiPYxFVb;Gmfs&9k^0x-f)Nm%E`?Rg!cfB zDsK)D>MMfoJn1sDycM?mx5)oAnAUDJQj!aQCm#&P>i|gE)YR2I`f3T&sGaDwrn*1f ziF|>;QEmi)sbt^ufVWoiFAE{A9AX^j^GFJWR{_nKm8Ht4-p8#qnAMpZs?#4MY~i4{ zL?ekMa1$Md+FwEY^_MJ%&t3r%4bumwODEw0M8KC@ABEuaAMO-{SAN^Xz5GF5)}rAG zHV0vm&0ePL8Kmhx-10LgRb#f=_&hDvra?J2~G|fRloWF#=eO zt}#z;>E1fQM`GbwhBEt;_>Z>!8GkGFwr&yVZv0AbEET-XN63L31Y66y#z27q_0UNw zk$_mazZCgk{ikIs<^Mpd46O*$ANXI+%$=V$>6?%ss<;6Ly9J8={9CqP|G$xuKvp%6 zOC;wmBE8BU)#Vy}(b{QKKszy~=$LM-{w0O@`6nxS4k=z1HZzNDm}y6d79U=(DRJdw z#QKi`Pi@3SUVaK_z!l#=8ee;6X8d0(a{ibld3&i&{INOvk8j)?A@?S7e|h&9qj>(* zd@t&mkiTb@^t)PGgtBsIuC#J9JL-piGn#j1JdVvQ1l^TjS#-sNwxdzvGUll|AUhi} z46(r0ZQJgEzWFB04QO$E_ipb+S3&5IAsCPyX79HGIgY64-eJp8=^cT;|K?Xi!sNb$ zC>>5E+~dX@!;qFn-vnv3!;tb|S)K(mjNOkM#2#e^$i-b?pTxlbJUxHw)~&baN`ha$ zdiD9qTGeakkZvj3g7Z8hqsKqYnHhim_1E=|TdM?*tOlnH6;+^c{`pA)j8hpIx%Lb( zPkdyzKXgHWQfLz?JzWLHZmz&jcWN_*r@LpUDg_Z> z%$qmzz0fQVg#A5g-_hp*@PJ64K_*JBt6YBg=ur_2fscU}%#!QS{SWwmi%Fu{Lzg?~ zYhCzf6#da>;n7p4MA1cqVB>WLklM6y@Bo|nYbnmJ_nyi0c7_xWz=Pm%E43R~i%}0sPL`7ey5}3I*!JJmP&o(4VaCQwA*i4GGwB@>V! zh>nfjgsct!wFvyiBJ`Cie=J|MN*7Gig9i_G^PaWZP+caoo7Vzs^$^}e8^GhZX#-NO4IS0%yGIDFi0NWWht8q?%fPF=Ps%Xn+M;O+^6gnbkB+nSM#DBo8ORNuFr zUs5BSL%oNRjjvnm%^Fup=RB(GPPleRPv<*FELGRJK09V;bP>j?xl?hgM$)?5QBBI6 z?*#7FoTDd1iIbAWX&$t?-R^92rdsbOua!$=aN1w#ii<19%Qpz|K`&a=yd|&UE+N@v z46CUgiIa&G!7%wwh}#JuOmzTEESSE?fehvLpkeLF`il+ZEp|f`o6&mmk|j%4+$;~! zB{452blhe4k6IpAdWyI3Dm#7pgHMr*T9M|ki*KZR;kR9jW}dt@nAJtfY3@|LFh1yi zXCB7yu^h>s-tr2cqp@K_x20xgZQg!o>gk&P^agMDlL__gj)$b(9y9{c*3N2z?)ZdU zG&$32VbU|1`ed?+V`JTw!Q(kQH8x|EvTHRr%Xq$REFb(pPaDHcHc5W!{Z#Ir^n_ua zm6Ef6XtYnJ*^H}(&*=Sf_k(RwEoS?jdD_h+OQNr_Gk$3bjZ9Ruf}eWbv2U=nuK9lb z^|sm==h@fLDWaA$jmvCa$%?MobuiTZ)B>SB_4JZor}63eS+i#S@)gPs7T%1kc8K{x zTpsV)^0ImIjK#<&io!_clZZ1B-Cr?`gJSN~_X}y=Z`;C6F_CC@}sqQ|)ML*&@pO+^8jA z+Hj0}3{gfwwK3eBNE4euFIsU1o_i{m!O^;7Q1d#zT&`Tbz~zTWE6GZ7;S_iO&P8j?2%UQ&PD!pL!(o!j$)5U8Mj-#&bBNw(w)>U z`@(XjpOgL&Z>#iPBCqeT%&g`OndP<%gf0?KR3ZP#`&Hov7~iQcKCTE<4?(K6BM(`+ z&arAE0){gZS_x19y+Gc}4DAwCt7{{APQ``Ko%-T9o0uFu5=<>4pErNqYDP?6`xTy` zpp;5n_MFXqe}tyqC7n+hYuj=zkUQ}Z(=~t2?t!*t${Ld-W9QPatvj~*d6*I$+-NL} z#lu>~m<)!$p9=jrKA7Q}02_LO>+MmNfeu{P&DjMWagj@B6^kufkTqU#o6%N&+Ek}t zf;eS&B_T)Vn5O6Vllid+bsY(Tz)Q4#@pt2-6>-d<;l#iCQ~nl?$Ml+>?&Fq z1V3({-$@Q(^lXWHQOcxUN859`v{QI`d1!I$yLs=%)oJgKn`IYKqOPmiy?#!oM!McLBwE~&T8IAm7ZpQ0+f0bNw;{sFe(#EBCNx7|UuPJ!#Z*Tgmq{U`jURDA@yu*dyl z+`A%a-v>JAq}|>^EcFb-ax#^pyPqf+SB!?O6kg%4loRP{RZ4Dumf2Ba>l{SNrf0rB zdaT1~mhn_?NHiG6!&sb$$-THUjHs~mAGg93GBrqRMi%T%sg22toeZfzxoUpk zvU!~J5^Az0L$zKJc4BGuK`Sw7-0@Wu{0QlEy<1~5tw?b+>eBIwEBUmxay8CdN1P(u z#_Drd2J4uwN%wJEyk8a4L^A$RowRDVXa!5=2%k9!HzMXKRl(OK5LUM%L=O#p;r-gf znRc>2VcQGC$R{p$^49Cq*w`}PGTT5|LP_WXM}-r|o47_X?rej>=)YdK(DIPjLCO5 zh=HV4_?(gkWY&r_dlzD9e?=7+_{q)pYoZx_VruA4nKrzXAtu^_wf^o3M--iGokkzd zks?W}N&PshFWukgrdl#H$}c2-sg3_r=o{r1jtYW!W;@K7S_6|%N_gPfKsJLH6`lXX ze;;u46_n(9laZT%WE{m(DHFJbwihHPNqA>AQ$^F}E zc&7;J_(Vd#x#l=*Q%XrcSR==(%ID;3wO38rqeVS0w~N)EvGgT7+GdUTAGx_1(8Q;V zp>Ow!Kg@WT)Or6T=|Z68L?cc$cj%g{XYZf4idTG}KYrjnr}6VZb10pB0e^8Ib)GL^ zKD)fQjrJ3*ELCBncnQsgE)h+%Z-xhighcr1Bec^hpO3&`7?UK6$BH>1ekHo|+#!^@ zvfOab_rThApxKN(>_F#ViKg&uY#@RX*;x3ZUNVK8Q+5x~gROhSrBF;uwWg&gjf~DEro%rA&)xd$u&yp0qf#3upufa&%0^sY3#S>L<#hLYzc zXUn=%Ijwu<5~M$=<^alZt5552r|ISoY2L92e>63Yr+rp!s~b{bMY=a_J8dK4Ui0a> zLS)dEB#G8@uiakg%Fr=FWl=LBsg5iJ-8L+V9D z?u{r6-HD?tOKT0O&&@wS^kiUYmt=0ah;QhIYwq5?TQOU=f3?G+AI#ay===7grdask z65q6WrA>)+nb}d-T*K8=@`QzcIQ_P5hXhdwP*&b`LwPo{5?}Z*Ehr-db{F zg{Z5#w#E-fFm!|`^>#LWqhBcL>T%~auSza)Vo0spguQdkjl24R?bCJ9lSBLL6>Z=k zHN?8P%qSUXUbPGG>bcjjDk7PlI%4F#f1P*Cg~4a5h?Vc{K7Uvra?5clv!wr7+_>gt zHKCM~Nvx@1XrdDrv+{&9d|LdLn!fY;tnMWC-S(REf#PCO%jwFA6f#xy=8x9*+z6`NRU5+&e_#1Y*RB%~ zj?*_xi7=rceM?jjz?SlioGW{{>BHG_U5w3^r_j9sSP&ji3TW;%K>0|9@43jN)W`}oajV_>cUzCQ zjM^(@6;x0PG<)*MYO5M=Pp0gtq9_sZ{8pJYmO;Xe$F^;1>nkGIjgWwsqbS?H5^r3$ zJeU_8P&Mz1}1wo0CM_li65-MVJLV5wRiO0HFv; z=d<@RrA^8(W4Er_+!%B{93^SS26@krUA7t>&_!1Gp{HddFYjFiu=uZv<>WdTUPK4I znZ*I2?g3T(>1L@=%EO=|APn`c< zyQzikp@@$PfxToFFcnGTRCeT%DqjG=&{*0 z`@&^q+But0CjsvalbmKthzdqYTz(o_jnpffrOJCJs*J}OsY+B=VnuamU-P>SFXpF4 zER?K-`~O;LcaWB#*IPNQCaM2&VAeqG>L?8zz0=GhOeJ$x1bW)ofL*(H2SSH5+L~I^ z2tc4HNZT=zW~k52yxEM4PDl-$d;OR!Q6y*O@H(>;ii5}aMm7OB2k#P%w_6wd88t#_kL8A&+!ZW1~*;>M}jYOc*w>htCy0JWJPy! z`&vuBLKpEXc)LFm#0?ATvS>DBiOTz5$A*S0Oa2tor$5{>p-@MfroSqNUshyKAXekJzmgjNj%1*GH zo#QXvar=Wa@NRBn&vzl`&VYevy~bxcesji`BhR%$qTc@H_b{cziU0~YT-Lc$K+Wgy z_y-Z0S0!hG2WhP%pQVT6n{67}Kbp)N=A`Hl%f+-FgTJwFsRxR{&knx`I#z?CQ0=h# z9gbdIBcDIgvmNxXmhUV+{Y5C<@MSoMj{6lenhu7|FQ&jdn?&!h0%xp?|LI~3i37W` zH6Q6HLDQ_0Xi$$rxw}yVI@@WhoR?s$5@on7f@J%fL-}sqwrNAXnYO}!tE*GsN$ za&Wj(Q}-dPHe+i4^y#5_k5Bn@@0!WlfKi?fO-+O(Bn4}IRx=TaY;O73n+m%0gMm(d z@Kdq3Z?8o{V0?LG`d)>XT+eEy^WcR0HBpg6^=t=Nq5HD~n-w+W^qthXQo59LM;>t3 zQ!fbfS_k-_oh*K;`-}&p;Qr(uvwj^hm~Vkqp+9dbY>HH%H{#+@WaP?1cnZ-1qoSfB zK=zu>?5{90eXoB@3#^a)dQph$VCacre0;r`AfOr}1fPfFVqVB&Ya1TDr<$%SKasQ_ zc@+j!pXodq<9@+YHr;hJ@VL^7 zEx>Cvlw8-)=Ah>3X45!5c5j;*GvJ{JFXiTL^qf_F80GnU0Db84gBKn;hJ!c=A-M9z zJv~ztg*W<4728j8;UZX*Xej;w`)$7 z0oT2*!vttXDC*yokSCsVdts!UU|okcj2%M^Jv7r z9l!4g34inTVeiMyA~Rk3W*u>|1~^dgf=J1C<$?YP;Wb%zcWj5S9K_Z8kornnUiMb_ z;JMM(L>;9q@Af%Nx+&)P@3Taa(=x8~bd94Zq(eT~@q>UaXhA?22NbzVtE--OKV(cj z4UT+bw#Ct$lnP3d{UG6z)8uWWVJM!rD4^+^2{+2IwG)k0&rj!aZ z&BEpK-n+mfG!Ie(2}(-+A}Lh!R7L!LO!R4rQGjv(%OQP;g18my*78cvjRd6SbE&=R z{T%LDKg^8;Y`TZJP65%SG6-5@0CPC>AwqVS6_G^30tya^=`BXv@H(6G_lNbcef(h+aLR8ZFky5$+v=54^ilF`o17?N z*D543N*(91Uzo1<8DP9K1)5rM8UxL*Oo%g4RuvQ5!2;He=(m)uAJIl(Ve6MCgc<9| ziNSu!edw^4!~p&6PPXpYM7NaIf5V08&SwW!{)V>MTfavh3Z3^)f_{_kS^dcHIYhXT zL&Q6GiT3Ka;4ZQKU<==qboq{u@8b=GB|(psOUl;9$Yhmy>~kT=%uF~P z4oW)|M(N`tS6i>UT4dMzsu&0qkrt|pb6j#a<^%7D;+o~&>C#q7$xR-c<{3u*szh*{ z>OhSv#j9dkyIExw=t$H-!Ty(tkxB{vG39MniR<}x+2u*1GcZkz16?IO)cY|tc{(FX zSb`MUrbk%w<+)t|Iu7Yir-0;%aC%9AG$4*k_#+$zS|=IQx-+%O02;}j0X8+3l@-`> z7WALXq?^;N8;cc8q~<-VwmM}6lPb7;h!B*QWV=`aDa5EE=mvmhJ8Z9PJwIA# zm2O~WR#qI8KAv0=H80yJnOEa}%}S+>O2h#D zFlDFK@jM+l&7u4R5!X3^trU3!hM(V{cG8((Tc&&8a&yU&rdNFBj}8N0?R#3yWh`Gi zw;0pV3jqhNl}u+Cdkzo%VVGfk8 zIh*J11gQST!;Xo|+(kC$!n3^oA|A@NSey00PcI1-M>RibT_~-GcSCQtP%w$UjWs~o zL6j7PX=tQ1@EhOFx0nT9=v9owPDx2L78}OV=q$X-+P%blHF+nvTxg3Yl-rk29mK&h z98Gw^w}7?D0bS!Ohl)607#uT|=ixl@+O->2A$Wbu0OGOVJs!*m)bC_=2(`SnaBFlc zc~h5%(^^Pc>||er{Xpc&pnTfpSJdTX#mf7|R-Lo!h z3pKhQ^&BII>iDEo!cm5?vRez?%mmo~J}l4_sio7uM>-TKiy$Fu#3Er2bdD}-6KG|YG9u6B77 z8$O#R`;i{F|5&jL+~}aBRPahEH3IA60an2v>z#X@4h&_@l!UeSk4PKf)N-c;Q!kf$ zvGQV-lI&m%pCtmd`96%0wmXsAE|sY$d7M)ogQ-|x7p_z-8CXO6gV_jz1*NZ zvSxRPM+!;Id8qME*5|viag^AT(W|UILzMm0?>h+CjHb{akpHGVd%oe>~vRb z^*HvC%WPTOxU_xU@V53G)jAO#&zUM$kQi3N>#_{{1i1rZC~Gx8_b0SJelM8TC6d!O z?K+Y_MEQQh=UA(h5EFY7?&Y8s`kb`^ezJVa>@sbb|(D>rexaSrZK za1eSp);-&uOH-bA><}>1nyXp34c2ZP^v;;u3SPr6|NS(|`f6B-adllbjki3lU7FWE zC~gvbt>Bhrnac2hj@iek8CK#?j8(G9deh!Nj}+N3CQ}@Wj)lJJiiuW{5ScF5XL8wJ z=n5_lR<#x84bKKfYxiJ`9jKXnDqGs#{6f;dNy%l4R|LA4mxFJ)vgps3o5GVg58$OjxejuB9rEDb2%a!5m@~Q`@bJ}Iz#&eX&#iEWy^FHi^ z_kG>z-_Zj%-DR_?-xi2}pk4)Kk;|M*|MH+X4RDuvX17VTA$kedS z9)?w6=q9=?*f#xgCKM*1tSd7$KLFRw4tLqT&N0+Y;d7^W>BtwoxmXaYY6O248_6L( z1%d0NH=)h-;Px8egg;6oXzk?BoNFf+^UEK*>RiUiv@@v+W7Z=qY^7s<@^Mrxf$yne z=AzGQIh8Wm<#dh>dR+CK%Q@*dr~-^Iuxu8+vX7m^ z^3|<_qo1@qjg5L6b%^6+~1rX zr_n;Mudg=kIu^&Hcs~skn4i#E#G|Bb9CG{OA*s_EgwPDk^q zot?L3@e0yPnLIo@M(VKpdTO%}inWl36A zTGGpf4K7B_S?GJo^pv7RozNlHQ#co$}4=GnHqn zld{28;a@Ok3VSkuox#%`s9kxz z0ZMjtt8(0W8&pZ41-+=CM3E{Ah5+s+J2lSR3T|8J!ttZ?^%3h7tX)HCMxxds&&WIAKJO`?AJ^%EWv#TItiiSOp$!Qnw}I8-UEMxr1)#29EYgu(=@GYeAVSlCfH&I-}&wr_moDD@)6MaCyz`tG%$_(><>V^y4M$5W+3wWs&Nv-uXVs$kLk`r3Xq{G8SX-cNLf z&zzEA`#6=L`=4C($ff0E`}7RVrLWBu0NHtVvf1qIVwawKNTQqAl^K|(BNb-L+mQcL z&&ZG|)*}c)t7}5RL~~x6wD-(}pf+I>{6%F6$6Y>jkN@4F{9P*TjZP%fN{^|S9rx?c z@#^b}*F#0U-R5j30q>E(BeFKsXQDLArR0`Ej_1*;PPhc4rNCAIlXiJCF&k=I7;)j* zU3pn;PqnpSGdoE1Blf4Zim8N1_?Oz{%2+VIlq0_f>92Slq)dXv*q>J!2##NrjH9*k za8=i2vs&INuZnf|@!D|N8(uaYF%7L_ZwpLED!kXzCF@sNhJ{=ibr8*mJ;pt7r8-CM z%vTxN1LR7Yc16&UJDE^YqM^a)y{BnN}V? zXUFuJkz3+O^l0xCFa>>x7EXb2T+D%_VX#%mhi~dEqLt-{_^UCBC%gabZPYSfpf;j7 zl9I`<5-t9W#yN6i(v%-#rZ>y3=V?^BXIqBHy?k?Y?=kpyNFdNxYoaqMMuUV=&7|H_ zAF08--s|AAP%ggcgr{%&Fe+8Dp=JE-#H5Lu*Y1yfDD1-vvCk#=Dp+2o9Nph>@>N%F zOZ_@zD!}sHJ?kZ4;&oDUR$u)xNJn8W`ZR1Xo9DIi7OMAHK@{k8h!NW#4S9~gbS|xj z)f{}L6XbzFS~ay8EKSw?O2p@dKak@HRO(rO$&mPt2Yqvy(BZ=g_g_qY0saFuP)HE{ zr{6S{scnMBw8loWDG;lJ3fVwWLy3WL=9(PA)QT~y z$9&^B{4S$2khOxW+v_?B%R%AVBBra2790y5~A-g^mcQd-_8G7;_< zTgGwSTFASb!LDmo5e!n}ltGdz(BPJe`|7tPxD>NR#cTXsIMVL#RIA%H(A;GWJZN`F*J~iDv8XR!gaR8P?^lXW)0dQ zjfPkOL0)zF?49#)7&!sQ z@WO+AVBbLOl5Z9Pl5>vbYM@@hCgWH!5ON;4`PgiK`QSHux!|gPx0fX|`Jyv>|B7e& z8(3C*8T>II3?~qHX++uwyP1U`?l%}k5pw#jhk(<;%be;{Ob=lTXp;#%Y0ddAx<{xa z4_w2kH0^&|wrm-yl0v{b)D&xz5DT^L;h>i@N(<-CPxm1niq}E({^1!CK>na+Db=zf zMq&IDR8BHEpF=TM?uXB4%N;G(A)!bmHk8dV@FPZDgiSs4vltJX)6ma%**OgKBjz1z zel+uY^iRkB?|PB>yAjsl?C-C;vCk6y&R3buh3M!1P!=0`o+p`zb{kQWc zJC#+h>+=ViF4v7X_s`pFA^*Fl$`PC6`QaL4aQs=b!#P?QckZB1O{Kqo_Y`HxM{s(P zTvYC6^G~-X!X+F|bOuRxO)_vc;Ne$V<3^FW{Cu z^)o7B$+E+jnhp)GiQIrtfZmdHgfIrveh^Ow=lJ z0*>mSdgRL3{anTbmu5GIzn6;R-!&s83VVTNmDFGoOL@)(o0sIAa?6Sh8xmH`K91$~ zD+5r_YQ!lQSQ_ze)ulz1*VsH3%LkiyixN-7L8qpi6>(_&jkXLeLXffem@X7juYL_) zyYV}(!;rpQT%%4K|pD{Q3p^~4ruIty(K{yyaH`P@re{@b42Ub`)ZO?FMoXO z%;glUma6C3^Xen*W>r93OYFvfrSF5D)7U;X2H;!JolT7iptV!_9xkq2GSHm8`ONO{ zNd4c?3fx0M8~DRaytDnrc|rkqhBLH-DGx1ywZ3HvFMC9DXSn$uqC3Ab-n{Xr;NB;x+1Y^Jj zK#!7jrAzNIgaA;h9{Z&pqRvRNr*zEJ3{x{Q-_*E)uQEGvq#oP^ z-Q)(al~VxT@IFqrRs5z5=mO)5xNpog*#qv7{5c(BNrx{snMF4Rj3FMr&fjzz+D9{+ z-JwJFEN_VF^krzY$e5}FF4n5gv|U@?gFulV&h78lH4>J9D1QVD7%DHi9VagBO}$iL zbz)RLGreJDLSo|fZAZhI6K5My0u_ie!S^KxM_jY z&Fjo;2~Q2V)w$|sPGHYSd2WiCp>Ubja(yv8ZwNCPfCq4Zg>5sE)A|cqe+p$Pl=;+V z>MI%=RkReS-9PDqH@wybw^lm9NR+Prhd6NO+ajLvXaTyk)o9zB6ITF)bfpnQgH1kt zxZ+?0T}7xp%3&bA_R#)o$A1|>>nHS7r_5p6(H>|Ot9fcR;fKq&A1wM;wia}h{dkP_ zAdCxc?*~O6=c(|l-e59I7oPHs`x`L+$6a=|0n;vW8CkmWLfn4PuZY@`{Y2ao%f3bh z?6eEq7#0*<&Bdcb@{9lo^9SmjK>f)`*1%I$q*5T{(~uM#NaAG z{mO{DU@ZDlzHNnnEaiKQx`kf#Sd6TsqR=aA_<5gM9Y6oaoS|phVHFgx-*_*!k6^zB zX--&Mn>Aizh%2W5u63^3wB z0*tAzlLyP9*4DzT!9pXKCVpli!iuu1*4BaI1kb%9USl6Elsdf{4?d7Do{A$2dvUEe zLK?IESzE=`;VM)F+9@F+OnnT14qr=_*EtklLc@ZP9H5ASTSHXpI`&VVhpw8{XcQBZ zQE%lUa|Y9xL_(BD4wn-cq6t>Ijz6>idszR0n(;qGDaueiTV#GMIws?@8iL^n{4-3n z1q6yn5Dr3|A8B;*)TuQ*s*@K2VSEPozExW_azM+Rg)tk-v%>|_U~|vZ_C?@a!FX|= zL*+pZyGor!&s+Qa`P@y<*iwW>ebWo)RK-Ws1Uxzw^)5u)? znPjW-yQPP;cuaVrA(8ciPM6#QoziF;YGamzA~ggVf%b_jC}0KG$o?J3G;MlRkD}+Z zX9t2n>YV^RbQmp@*Mb7hs;vbV(=Ha=LXDkhpz!B(g!J7qI`JsLSR8!B?r9DmVL9}j zIuxvwFN7(j96a@AN$}38UmRc&O4nBxQaYFpsodcMjZU>j$w_x;x7uyS^K|8$tTrwIaElB^0hpV;RmSUFzJ-9#lYNlVI5Hd|E`#*FQ%(3q2@^X z;@^L4-@W@N$z0(vBCH0F${JLx;9!JKPoJ10aqGMs7D9IIFt1Z!_I|kifKv0lsjy5A zc3GZJvjVK8@2g>}f0~;AXn+3|31TFwb%;`sUbgIy<&U;5(4S9SHxPJlZLKk6=(0K@ z3~xH~0JfxFOXp}3!C>_vWPt+Fx-?5YutlBcnQ(J-*or`A=0;MzAD$$jt5M)|fv&Aw zAmSxRJ>F6Am?(sc>~>N@yKkEg{0=hM|(e zm|z7~ry%kf)#Xp>>kGsk3PQ6~qDCQq5@L6{;!d-d-Z9dt*vKn7T2T;lx(>hPGJ~yj ze^z~7-jEEcN5p5A1`Xi04o&r)W~P(!WIlk19^y;1mI(i79=e~RnwWKJ-kvZ9p(9ES zz$4EW?q|l7&(toB#Vu`99|5zE6e{S-y8fYGd3gcrS@XbW>h1+y)FVgA*V~4Y5Uj7e zfxMd!6(V32&QXzSVl;B<)Ji2q1fv6oYKJYy??;O1u_7HoxsfZ|0pLUpqo!zEk%7UE zz{Sz=v9V$(5{KWUI(EK?$gCur2<|q5w}YPP#|Q#6((3XHh9m(`vhd}5n3HE=FfU=U zc%a(!-KDrVKfkZnoUa0v0NM$lO4R+$v9t?CW3hcR=Io9Z zj8-seh@BqqLp^cOXn0?i3>yFYlRzdwn{>W)JHnk&N()GpUB5>cpd?fTT9aZT6}VU{ z9Nq%$-vqTGj+zQ_WeCDDb1@QiG2XieAt_ACuEazi`Cj~l5la98MXk@b`NYvq>2g^> z-)@8Y)`CRbHX}*5o?x&l%g}l&U-pSS*ursQuKVq<18cL~tFqoH5 z7>bbiKP*z!UByL0#!zjZ7Say9Y$KQZ#ps6OPcmP*4TQ@!>+3D#?50`$f{XRwE4()N zOu}fKWVi`h`(eUp7V3Z70f7qnuBk=#x;%^$JIk;mLM6FW?d;ATLCaY1GGZ|v+J12g7$~u z9t3p4B!f`b3>;v&Y(;s@TF?YF(p;o4zO z)#=Y#b3?yD?uS@H|A5Yc>^*l+a=9-zYgPdBN%40we!-F6Yh+fQUfPC<-a*w z-`q88)*vlv21f+v`X{aoB6Ura4SF)gScTvsonL)-v$ZD1@1P;J3t{G`<~ubUAZ-o0 zr|Dj>g8FLY=vT#SBV!&liv0Y1hnd%?m?B=sM&>8w*!uFS)gJ1u+NQPOy$P%Q{w!YS zmZaV)E``;vCzK8#Yc$z|iyQLYuN~F^l_I{xA&$-;3|qGRlil zAy0d`&acHfql}Fx0Zp>*zW3ifyKXwe<&SR+?FoL0S#LPb3<0D}mRPPX&%uay6T!YS z-5r7&xyWh3&hs+eof7*`UjP^UG+SnK0&c{jw%dR4mK|)jqKudgX@L{oa|2rD@+wSH zaHv>KI6x`XgAEX=^riRrEoxE0N28Ik5DzR}W#*PjZ^c!C-)!3ZfLe@lL7>YH9=r)e zZv$rvjg3SADGa-yMnL{EKuK!-Is|j$voUxLj-%7M56T63Vag-#HbeP%Wk2bgPv}W2 z-5f_vU7{VfQY#7az}EG{=Uxaqz!()%7-oZ2z-YzSV0*{m*aHcw^3&Xqlz7IAWYvWO z858;wK$zVy#b%Jqn1|e=jl%#E`HT9eP!SabDnio#?oC}+c=OR5y2H_wAK96lw@6+f zNwk`$Z`qH&;2T~$xD3rklb_M2(fbevGR<((^ZeQv5T^w*1nTelgEljp(+ODO;45k6 zbJ+zvx-#N=0Nht(TjNd1JU@B)Z%)K<7<8v(q9R`Y2k+kY2cOB;Sv#Rx8fF}^;FNsJ z6?XtqIn=V~2DgXJPfFVU6I_c%iDxn9XV5H;qI6PM<>!IKmJL$#qXjg`C3Ct=%kD|8 zkJW%GFoYpR%tg0Cij0=rYXEU=0euc27EX^96MPkI=Vnd_tRS@XHXyY$s^z2fC;2{) zunkIB_iwH?h0JXkaR)>|u?|-`>0d zLf?m~s=Ll8E6TIHZ|HnLbHlbvNE{=X`)VSZEm)XRMIS&dY3a{s#5NN@QQNHy5zPg(32^1Y!9h2hfPvy)LxqfC5g zAv6%4&;R)eBvAKOj9V8P4`f?Cy%5H(E&uxR3rDEJ#5al2;4GiVmC1^X{Casmzz*O{ z6ayZBE9ViCQznv!v=A~YJQ4HrXCyBKO^p!Pp{hnX;muJ6UtdDb|7_|7+ay;qE#Q>^Tj#O39N0-}s)SU~3`_`bXaG)eNg<`5||)`B}ksr4Gxyp8`|7A!q2sLlxwK3Zf?9xSkMp3Gw)v4L3Lb!6(Fjn(1n7~g+oX7tZ2OpLlX(315g4_Nup20t7khBYmkU}K| zV@rQnFVHhHEeV#g^WFC(1PK1-2OERNvc|o`KcB8-FLS4GO{(ieZz=WCfoSz z=0}RoPE_z@vcQ_b{F%Y5i3Y253}v9`ai9 zqp$C~R_KTV)T$u18XI}>7|=r!6^}y!_5(e}Nt8T#VTAxz2nlXK((pTUByoA`OPF0D zZPV}?_@$hQlT4l&PwIBF{<8SqPv@>KIKV^whP`yPe*-F!>^i)s-Whd|4U4P(34D_v zO8kU=R4pPUgAH3Jd-YT~W)LClR!>!hiUiKrJk9$lw)fS5$vaG>s*q0)#jB&(3eD;EK*b$X|7Yx80!(D>y&2;W+DdMV_esV+lF>|vea3m%%3aGXM7$+ z10FdcTXm&NER?YgO_C(c^;;@3UU}8CmpB17d_gL3v4PBNT=~S%UZdG($)h|A6OI zmLTuv^^JeX=4xF*4wCMN6ci0;zfCu9=J zC;(4?2Vjppxu?1k!~O9Q{@ZUU(#$NA)@e@iASFVYc6Pu2zQky2o z?sNBgo@t96_?B|0d+qEt`=#n8@=N4Icm1)r(g_J?;cY%>ciCt)*lPQ8e4+j~2V#q6 zC_AclN2Y{uIZek%Z+BlSBtXDEvm0Bry5tq89ao>eE{$can!kO;;Kdi>5{t#@s4!`J ze(PTF+-y0@7FvI+hGVlM&p9U-9r5!G0;sjLN$REaI5NixSO0N{OmGp&WVzz zJ~wRR(aAgwUg^>BJsb1q$5SX&HHOY{F>yLNeGh3LQVYC=p|SAI%g4jav*E5K|a>O6!WW zD~2HptCOE0xv5U@$*-T9GjhZJ{cvSU(IAhDrsiuUd)!NCT}SeG9tI_mS}~Q6VT3Qj zR|2e%Kyp->`ATk!*42Xi2ucJw4>qoUp`Q{A5Mp5gR_hgL`>ffKQsCFK7%tyjSvfg0 zH!e-h3TY|MH?cMMIhz0j6f4jd4`!2yt(-U4{^rUL@< zzLM-3c4i1QGR%oW!LNMdIO?j=CI7ec!2gfa(EofqlMh(`{BPF){6qYIh`&IAe^`%$ z75-7WsMzWI*Z*|s(LZYEJ8A#->%RUW{y)TDpuqpE&Yxv8WHP7n`|h8wF{?8XS@cG% zgM-$Xe}4Rj8s91K4>Nox1$^@lHU9sj#;RlfVWO`cbZ^3~Z^a)xi#NZBzl1e&yafNl z?A;@+AhAbAVvp>py?e3yWU#UdyY}qC?%9*m^#%2lvj4!w&ismn+kgEB?{mx&5plE+ L>S_`XT)6%p4&u8a literal 0 HcmV?d00001 diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index d4945c0120aec..50fb6a56561cb 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -58,11 +58,12 @@ public String showCommits( FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf) .globStatus(new Path(basePath + "/.hoodie/.commits_.archive*")); List allCommits = new ArrayList<>(); + int commits = 0; for (FileStatus fs : fsStatuses) { //read the archived file HoodieLogFormat.Reader reader = HoodieLogFormat .newReader(FSUtils.getFs(basePath, HoodieCLI.conf), - new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false); + new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema()); List readRecords = new ArrayList<>(); //read the avro blocks @@ -70,10 +71,17 @@ public String showCommits( HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); List records = blk.getRecords(); readRecords.addAll(records); + if(commits == limit) { + break; + } + commits++; } List readCommits = readRecords.stream().map(r -> (GenericRecord) r) - .map(r -> readCommit(r)).limit(limit).collect(Collectors.toList()); + .map(r -> readCommit(r)).collect(Collectors.toList()); allCommits.addAll(readCommits); + if(commits == limit) { + break; + } } return HoodiePrintHelper.print( new String[]{"CommitTime", "CommitType", "CommitDetails"}, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index 75dd49d6cfadd..6579ccf9ff5dd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -105,6 +105,15 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { // Default memory size per compaction, excess spills to disk public static final String DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES = String.valueOf(1024*1024*1024L); //1GB + // used to choose a trade off between IO vs Memory when performing compaction process + // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory + public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read"; + public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false"; + + // used to choose whether to enable reverse log reading (reverse log traversal) + public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read"; + public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false"; + private HoodieCompactionConfig(Properties props) { super(props); } @@ -225,6 +234,18 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } + public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) { + props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + String.valueOf(compactionLazyBlockReadEnabled)); + return this; + } + + public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) { + props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, + String.valueOf(compactionReverseLogReadEnabled)); + return this; + } + public HoodieCompactionConfig build() { HoodieCompactionConfig config = new HoodieCompactionConfig(props); setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), @@ -262,6 +283,10 @@ public HoodieCompactionConfig build() { TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB); setDefaultOnCondition(props, !props.containsKey(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP), MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES); + setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP), + COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED); + setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP), + COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED); HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP)); Preconditions.checkArgument( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 9933f3c9c2333..dfae2a0820291 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -24,14 +24,14 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.metrics.MetricsReporterType; +import org.apache.spark.storage.StorageLevel; +import javax.annotation.concurrent.Immutable; import java.io.File; import java.io.FileReader; import java.io.IOException; import java.io.InputStream; import java.util.Map; import java.util.Properties; -import javax.annotation.concurrent.Immutable; -import org.apache.spark.storage.StorageLevel; /** * Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient} @@ -215,6 +215,14 @@ public Long getMaxMemorySizePerCompactionInBytes() { .parseLong(props.getProperty(HoodieCompactionConfig.MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP)); } + public Boolean getCompactionLazyBlockReadEnabled() { + return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP)); + } + + public Boolean getCompactionReverseLogReadEnabled() { + return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP)); + } + /** * index properties **/ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 0720b133f403d..7f00961f2c28d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -159,11 +159,14 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { return Optional.empty(); } + // TODO (NA) - Perform a schema check of current input record with the last schema on log file + // to make sure we don't append records with older (shorter) schema than already appended public void doAppend() { int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0; - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); while (recordItr.hasNext()) { HoodieRecord record = recordItr.next(); // update the new location of the record, so we know where to find it next @@ -178,7 +181,7 @@ public void doAppend() { // Recompute averageRecordSize before writing a new block and update existing value with avg of new and old logger.info("AvgRecordSize => " + averageRecordSize); averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2; - doAppend(metadata); + doAppend(header); numberOfRecords = 0; } Optional indexedRecord = getIndexedRecord(record); @@ -189,18 +192,18 @@ public void doAppend() { } numberOfRecords++; } - doAppend(metadata); + doAppend(header); } - private void doAppend(Map metadata) { + private void doAppend(Map header) { try { if (recordList.size() > 0) { - writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata)); + writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header)); recordList.clear(); } if (keysToDelete.size() > 0) { writer = writer.appendBlock( - new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata)); + new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), header)); keysToDelete.clear(); } } catch (Exception e) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 6e1f14ab74a6e..d0454e3d238b6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry; import com.uber.hoodie.avro.model.HoodieCleanMetadata; @@ -30,6 +31,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.AvroUtils; @@ -39,6 +41,7 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -47,6 +50,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -190,7 +194,9 @@ public void archive(List instants) throws HoodieCommitException { for (HoodieInstant hoodieInstant : instants) { records.add(convertToAvroRecord(commitTimeline, hoodieInstant)); } - HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); + HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); this.writer = writer.appendBlock(block); } catch (Exception e) { throw new HoodieCommitException("Failed to archive commits", e); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index c72406d126297..709c0e9fec7ec 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -154,7 +154,8 @@ private List executeCompaction(HoodieTable hoodieTable, HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(), - operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes()); + operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes(), + config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled()); if (!scanner.iterator().hasNext()) { return Lists.newArrayList(); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 88d7b8d8a8a97..9cca2ea903b41 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -265,14 +265,15 @@ public List rollback(JavaSparkContext jsc, List comm .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; // generate metadata - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); // if update belongs to an existing log file writer = writer.appendBlock(new HoodieCommandBlock( - HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, - metadata)); + header)); numRollbackBlocks++; filesToNumBlocksRollback .put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()), diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index 3add748c995a7..1e5d015d582ec 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -122,7 +122,7 @@ public void testArchiveDatasetWithArchival() throws IOException { //read the file HoodieLogFormat.Reader reader = HoodieLogFormat .newReader(fs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), - HoodieArchivedMetaEntry.getClassSchema(), false); + HoodieArchivedMetaEntry.getClassSchema()); int archivedRecordsCount = 0; List readRecords = new ArrayList<>(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java index c4a5279144f14..0c587c67199e0 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java @@ -101,6 +101,19 @@ public static Comparator getLogVersionComparator() { }; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + HoodieLogFile that = (HoodieLogFile) o; + return path != null ? path.equals(that.path) : that.path == null; + } + + @Override + public int hashCode() { + return path != null ? path.hashCode() : 0; + } + @Override public String toString() { return "HoodieLogFile {" + path + '}'; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 465b53867821d..98dd29b7872ac 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; import com.uber.hoodie.exception.HoodieIOException; +import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -48,14 +49,25 @@ import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME; /** * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of * records which will be used as a lookup table when merging the base columnar file with the redo * log file. + * NOTE: If readBlockLazily is turned on, does not merge, instead keeps reading log blocks and merges everything at once + * This is an optimization to avoid seek() back and forth to read new block (forward seek()) + * and lazily read content of seen block (reverse and forward seek()) during merge + * | | Read Block 1 Metadata | | Read Block 1 Data | + * | | Read Block 2 Metadata | | Read Block 2 Data | + * | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | + * | | Read Block N Metadata | | Read Block N Data | + * + * This results in two I/O passes over the log file. + * */ + public class HoodieCompactedLogRecordScanner implements Iterable> { @@ -77,10 +89,11 @@ public class HoodieCompactedLogRecordScanner implements // Merge strategy to use when combining records from log private String payloadClassFQN; // Store the last instant log blocks (needed to implement rollback) - Deque currentInstantLogBlocks = new ArrayDeque<>(); + private Deque currentInstantLogBlocks = new ArrayDeque<>(); public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes) { + Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, + boolean readBlocksLazily, boolean reverseReader) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); @@ -88,137 +101,141 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List(maxMemorySizeInBytes, readerSchema, + // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize + this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, readerSchema, payloadClassFQN, Optional.empty()); // iterate over the paths - Iterator logFilePathsItr = logFilePaths.iterator(); - while (logFilePathsItr.hasNext()) { - HoodieLogFile logFile = new HoodieLogFile(new Path(logFilePathsItr.next())); - log.info("Scanning log file " + logFile.getPath()); + HoodieLogFormatReader logFormatReaderWrapper = + new HoodieLogFormatReader(fs, + logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))) + .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader); + while (logFormatReaderWrapper.hasNext()) { + HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); + log.info("Scanning log file " + logFile); totalLogFiles.incrementAndGet(); - // Use the HoodieLogFormatReader to iterate through the blocks in the log file - HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, logFile, readerSchema, true); - while (reader.hasNext()) { - HoodieLogBlock r = reader.next(); - if (r.getBlockType() != CORRUPT_BLOCK && - !HoodieTimeline.compareTimestamps(r.getLogMetadata().get(INSTANT_TIME), this.latestInstantTime, - HoodieTimeline.LESSER_OR_EQUAL)) { - //hit a block with instant time greater than should be processed, stop processing further + // Use the HoodieLogFileReader to iterate through the blocks in the log file + HoodieLogBlock r = logFormatReaderWrapper.next(); + if (r.getBlockType() != CORRUPT_BLOCK && + !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME), + this.latestInstantTime, + HoodieTimeline.LESSER_OR_EQUAL)) { + //hit a block with instant time greater than should be processed, stop processing further + break; + } + switch (r.getBlockType()) { + case AVRO_DATA_BLOCK: + log.info("Reading a data block from file " + logFile.getPath()); + if (isNewInstantBlock(r) && !readBlocksLazily) { + // If this is an avro data block belonging to a different commit/instant, + // then merge the last blocks and records into the main result + merge(records, currentInstantLogBlocks); + } + // store the current block + currentInstantLogBlocks.push(r); + break; + case DELETE_BLOCK: + log.info("Reading a delete block from file " + logFile.getPath()); + if (isNewInstantBlock(r) && !readBlocksLazily) { + // If this is a delete data block belonging to a different commit/instant, + // then merge the last blocks and records into the main result + merge(records, currentInstantLogBlocks); + } + // store deletes so can be rolled back + currentInstantLogBlocks.push(r); break; - } - switch (r.getBlockType()) { - case AVRO_DATA_BLOCK: - log.info("Reading a data block from file " + logFile.getPath()); - // Consider the following scenario - // (Time 0, C1, Task T1) -> Running - // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1 - // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2) - // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2) - // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same. - // Say, commit C1 eventually failed and a rollback is triggered. - // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file, - // but in reality we need to rollback (B1 & B2) - // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2 - if(isNewInstantBlock(r)) { - // If this is a avro data block, then merge the last block records into the main result - merge(records, currentInstantLogBlocks); - } - // store the current block - currentInstantLogBlocks.push(r); - break; - case DELETE_BLOCK: - log.info("Reading a delete block from file " + logFile.getPath()); - if (isNewInstantBlock(r)) { - // Block with the keys listed as to be deleted, data and delete blocks written in different batches - // so it is safe to merge - // This is a delete block, so lets merge any records from previous data block - merge(records, currentInstantLogBlocks); - } - // store deletes so can be rolled back - currentInstantLogBlocks.push(r); - break; - case COMMAND_BLOCK: - log.info("Reading a command block from file " + logFile.getPath()); - // This is a command block - take appropriate action based on the command - HoodieCommandBlock commandBlock = (HoodieCommandBlock) r; - String targetInstantForCommandBlock = r.getLogMetadata() - .get(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME); - switch (commandBlock.getType()) { // there can be different types of command blocks - case ROLLBACK_PREVIOUS_BLOCK: - // Rollback the last read log block - // Get commit time from last record block, compare with targetCommitTime, rollback only if equal, - // this is required in scenarios of invalid/extra rollback blocks written due to failures during - // the rollback operation itself and ensures the same rollback block (R1) is used to rollback - // both B1 & B2 with same instant_time - int numBlocksRolledBack = 0; - while(!currentInstantLogBlocks.isEmpty()) { - HoodieLogBlock lastBlock = currentInstantLogBlocks.peek(); - // handle corrupt blocks separately since they may not have metadata - if (lastBlock.getBlockType() == CORRUPT_BLOCK) { - log.info( - "Rolling back the last corrupted log block read in " + logFile.getPath()); - currentInstantLogBlocks.pop(); - numBlocksRolledBack++; - } - // rollback last data block or delete block - else if (lastBlock.getBlockType() != CORRUPT_BLOCK && - targetInstantForCommandBlock - .contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) { - log.info("Rolling back the last log block read in " + logFile.getPath()); - currentInstantLogBlocks.pop(); - numBlocksRolledBack++; - } - // invalid or extra rollback block - else if(!targetInstantForCommandBlock - .contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME))) { - log.warn("Invalid or extra rollback command block in " + logFile.getPath()); - break; - } - // this should not happen ideally - else { - log.warn("Unable to apply rollback command block in " + logFile.getPath()); - } + case COMMAND_BLOCK: + // Consider the following scenario + // (Time 0, C1, Task T1) -> Running + // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1 + // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2) + // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2) + // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same. + // Say, commit C1 eventually failed and a rollback is triggered. + // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file, + // but in reality we need to rollback (B1 & B2) + // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2 + log.info("Reading a command block from file " + logFile.getPath()); + // This is a command block - take appropriate action based on the command + HoodieCommandBlock commandBlock = (HoodieCommandBlock) r; + String targetInstantForCommandBlock = r.getLogBlockHeader() + .get(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME); + switch (commandBlock.getType()) { // there can be different types of command blocks + case ROLLBACK_PREVIOUS_BLOCK: + // Rollback the last read log block + // Get commit time from last record block, compare with targetCommitTime, rollback only if equal, + // this is required in scenarios of invalid/extra rollback blocks written due to failures during + // the rollback operation itself and ensures the same rollback block (R1) is used to rollback + // both B1 & B2 with same instant_time + int numBlocksRolledBack = 0; + while (!currentInstantLogBlocks.isEmpty()) { + HoodieLogBlock lastBlock = currentInstantLogBlocks.peek(); + // handle corrupt blocks separately since they may not have metadata + if (lastBlock.getBlockType() == CORRUPT_BLOCK) { + log.info( + "Rolling back the last corrupted log block read in " + logFile.getPath()); + currentInstantLogBlocks.pop(); + numBlocksRolledBack++; + } + // rollback last data block or delete block + else if (lastBlock.getBlockType() != CORRUPT_BLOCK && + targetInstantForCommandBlock + .contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { + log.info("Rolling back the last log block read in " + logFile.getPath()); + currentInstantLogBlocks.pop(); + numBlocksRolledBack++; } - log.info("Number of applied rollback blocks " + numBlocksRolledBack); - break; + // invalid or extra rollback block + else if (!targetInstantForCommandBlock + .contentEquals( + currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME))) { + log.warn("TargetInstantTime " + targetInstantForCommandBlock + + " invalid or extra rollback command block in " + logFile.getPath()); + break; + } + // this should not happen ideally + else { + log.warn("Unable to apply rollback command block in " + logFile.getPath()); + } + } + log.info("Number of applied rollback blocks " + numBlocksRolledBack); + break; - } - break; - case CORRUPT_BLOCK: - log.info("Found a corrupt block in " + logFile.getPath()); - // If there is a corrupt block - we will assume that this was the next data block - currentInstantLogBlocks.push(r); - break; - } - } - // merge the last read block when all the blocks are done reading - if (!currentInstantLogBlocks.isEmpty()) { - log.info("Merging the final blocks in " + logFile.getPath()); - merge(records, currentInstantLogBlocks); + } + break; + case CORRUPT_BLOCK: + log.info("Found a corrupt block in " + logFile.getPath()); + // If there is a corrupt block - we will assume that this was the next data block + currentInstantLogBlocks.push(r); + break; } } + // merge the last read block when all the blocks are done reading + if (!currentInstantLogBlocks.isEmpty()) { + log.info("Merging the final data blocks"); + merge(records, currentInstantLogBlocks); + } } catch (IOException e) { - throw new HoodieIOException("IOException when reading compacting log files"); + throw new HoodieIOException("IOException when reading log file "); } this.totalRecordsToUpdate = records.size(); log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); - log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); - log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); - log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); + log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records + .getInMemoryMapNumEntries()); + log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records + .getCurrentInMemoryMapSize()); + log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records + .getDiskBasedMapNumEntries()); log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } /** * Checks if the current logblock belongs to a later instant - * @param logBlock - * @return */ private boolean isNewInstantBlock(HoodieLogBlock logBlock) { - return currentInstantLogBlocks.size() > 0 && currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK - && !logBlock.getLogMetadata().get(INSTANT_TIME) - .contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME)); + return currentInstantLogBlocks.size() > 0 + && currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK + && !logBlock.getLogBlockHeader().get(INSTANT_TIME) + .contentEquals(currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME)); } /** @@ -228,7 +245,10 @@ private boolean isNewInstantBlock(HoodieLogBlock logBlock) { */ private Map> loadRecordsFromBlock( HoodieAvroDataBlock dataBlock) throws IOException { - Map> recordsFromLastBlock = Maps.newHashMap(); + // TODO (NA) - Instead of creating a new HashMap use the spillable map + Map> recordsFromLastBlock = Maps + .newHashMap(); + // TODO (NA) - Implemnt getRecordItr() in HoodieAvroDataBlock and use that here List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); recs.forEach(rec -> { @@ -255,7 +275,7 @@ private Map> loadRecordsFrom * Merge the last seen log blocks with the accumulated records */ private void merge(Map> records, - Deque lastBlocks) throws IOException { + Deque lastBlocks) throws IOException { while (!lastBlocks.isEmpty()) { // poll the element at the bottom of the stack since that's the order it was inserted HoodieLogBlock lastBlock = lastBlocks.pollLast(); @@ -265,6 +285,7 @@ private void merge(Map> reco break; case DELETE_BLOCK: // TODO : If delete is the only block written and/or records are present in parquet file + // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove); break; case CORRUPT_BLOCK: @@ -278,7 +299,7 @@ private void merge(Map> reco * Merge the records read from a single data block with the accumulated records */ private void merge(Map> records, - Map> recordsFromLastBlock) { + Map> recordsFromLastBlock) { recordsFromLastBlock.forEach((key, hoodieRecord) -> { if (records.containsKey(key)) { // Merge and store the merged record diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java new file mode 100644 index 0000000000000..f9a01c8c85273 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -0,0 +1,410 @@ +/* + * 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.table.log; + +import com.google.common.base.Preconditions; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; +import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; +import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; +import com.uber.hoodie.exception.CorruptedLogFileException; +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieNotSupportedException; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.EOFException; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * Scans a log file and provides block level iterator on the log file Loads the entire block + * contents in memory Can emit either a DataBlock, CommandBlock, DeleteBlock or CorruptBlock (if one + * is found) + */ +class HoodieLogFileReader implements HoodieLogFormat.Reader { + + private static final int DEFAULT_BUFFER_SIZE = 4096; + private final static Logger log = LogManager.getLogger(HoodieLogFileReader.class); + + private final FSDataInputStream inputStream; + private final HoodieLogFile logFile; + private static final byte[] oldMagicBuffer = new byte[4]; + private static final byte[] magicBuffer = new byte[6]; + private final Schema readerSchema; + private HoodieLogBlock nextBlock = null; + private LogFormatVersion nextBlockVersion; + private boolean readBlockLazily; + private long reverseLogFilePosition; + private long lastReverseLogFilePosition; + private boolean reverseReader; + + HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, + boolean readBlockLazily, boolean reverseReader) throws IOException { + this.inputStream = fs.open(logFile.getPath(), bufferSize); + this.logFile = logFile; + this.readerSchema = readerSchema; + this.readBlockLazily = readBlockLazily; + this.reverseReader = reverseReader; + if(this.reverseReader) { + this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen(); + } + addShutDownHook(); + } + + HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, + boolean readBlockLazily, boolean reverseReader) throws IOException { + this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader); + } + + HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { + this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false); + } + + @Override + public HoodieLogFile getLogFile() { + return logFile; + } + + /** + * Close the inputstream when the JVM exits + */ + private void addShutDownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + try { + inputStream.close(); + } catch (Exception e) { + log.warn("unable to close input stream for log file " + logFile, e); + // fail silently for any sort of exception + } + } + }); + } + + // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows for max of Integer size + private HoodieLogBlock readBlock() throws IOException { + + int blocksize = -1; + int type = -1; + HoodieLogBlockType blockType = null; + Map header = null; + + try { + + if (isOldMagic()) { + // 1 Read the block type for a log block + type = inputStream.readInt(); + + Preconditions.checkArgument(type < HoodieLogBlockType.values().length, + "Invalid block byte type found " + type); + blockType = HoodieLogBlockType.values()[type]; + + // 2 Read the total size of the block + blocksize = inputStream.readInt(); + } else { + // 1 Read the total size of the block + blocksize = (int) inputStream.readLong(); + } + + } catch (Exception e) { + // An exception reading any of the above indicates a corrupt block + // Create a corrupt block by finding the next OLD_MAGIC marker or EOF + return createCorruptBlock(); + } + + // We may have had a crash which could have written this block partially + // Skip blocksize in the stream and we should either find a sync marker (start of the next block) or EOF + // If we did not find either of it, then this block is a corrupted block. + boolean isCorrupted = isBlockCorrupt(blocksize); + if (isCorrupted) { + return createCorruptBlock(); + } + + // 2. Read the version for this log format + this.nextBlockVersion = readVersion(); + + // 3. Read the block type for a log block + if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) { + type = inputStream.readInt(); + + Preconditions.checkArgument(type < HoodieLogBlockType.values().length, + "Invalid block byte type found " + type); + blockType = HoodieLogBlockType.values()[type]; + } + + // 4. Read the header for a log block, if present + if (nextBlockVersion.hasHeader()) { + header = HoodieLogBlock.getLogMetadata(inputStream); + } + + int contentLength = blocksize; + // 5. Read the content length for the content + if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) { + contentLength = (int) inputStream.readLong(); + } + + // 6. Read the content or skip content based on IO vs Memory trade-off by client + // TODO - have a max block size and reuse this buffer in the ByteBuffer (hard to guess max block size for now) + long contentPosition = inputStream.getPos(); + byte[] content = HoodieLogBlock.readOrSkipContent(inputStream, contentLength, readBlockLazily); + + // 7. Read footer if any + Map footer = null; + if (nextBlockVersion.hasFooter()) { + footer = HoodieLogBlock.getLogMetadata(inputStream); + } + + // 8. Read log block length, if present. This acts as a reverse pointer when traversing a log file in reverse + long logBlockLength = 0; + if (nextBlockVersion.hasLogBlockLength()) { + logBlockLength = inputStream.readLong(); + } + + // 9. Read the log block end position in the log file + long blockEndPos = inputStream.getPos(); + + switch (blockType) { + // based on type read the block + case AVRO_DATA_BLOCK: + if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) { + return HoodieAvroDataBlock.getBlock(content, readerSchema); + } else { + return HoodieAvroDataBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, readerSchema, header, footer); + } + case DELETE_BLOCK: + return HoodieDeleteBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, header, footer); + case COMMAND_BLOCK: + return HoodieCommandBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, header, footer); + default: + throw new HoodieNotSupportedException("Unsupported Block " + blockType); + } + } + + private HoodieLogBlock createCorruptBlock() throws IOException { + log.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos()); + long currentPos = inputStream.getPos(); + long nextBlockOffset = scanForNextAvailableBlockOffset(); + // Rewind to the initial start and read corrupted bytes till the nextBlockOffset + inputStream.seek(currentPos); + log.info("Next available block in " + logFile + " starts at " + nextBlockOffset); + int corruptedBlockSize = (int) (nextBlockOffset - currentPos); + long contentPosition = inputStream.getPos(); + byte[] corruptedBytes = HoodieLogBlock.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); + return HoodieCorruptBlock.getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily, + contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), new HashMap<>()); + } + + private boolean isBlockCorrupt(int blocksize) throws IOException { + long currentPos = inputStream.getPos(); + try { + inputStream.seek(currentPos + blocksize); + } catch (EOFException e) { + // this is corrupt + return true; + } + + try { + readMagic(); + // all good - either we found the sync marker or EOF. Reset position and continue + return false; + } catch (CorruptedLogFileException e) { + // This is a corrupted block + return true; + } finally { + inputStream.seek(currentPos); + } + } + + private long scanForNextAvailableBlockOffset() throws IOException { + while (true) { + long currentPos = inputStream.getPos(); + try { + boolean isEOF = readMagic(); + return isEOF ? inputStream.getPos() : currentPos; + } catch (CorruptedLogFileException e) { + // No luck - advance and try again + inputStream.seek(currentPos + 1); + } + } + } + + @Override + public void close() throws IOException { + this.inputStream.close(); + } + + @Override + /** + * hasNext is not idempotent. TODO - Fix this. It is okay for now - PR + */ + public boolean hasNext() { + try { + boolean isEOF = readMagic(); + if (isEOF) { + return false; + } + this.nextBlock = readBlock(); + return nextBlock != null; + } catch (IOException e) { + throw new HoodieIOException("IOException when reading logfile " + logFile, e); + } + } + + /** + * Read log format version from log file, if present + * For old log files written with Magic header OLD_MAGIC and without version, return DEFAULT_VERSION + * + * @throws IOException + */ + private LogFormatVersion readVersion() throws IOException { + // If not old log file format (written with Magic header OLD_MAGIC), then read log version + if (Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { + Arrays.fill(oldMagicBuffer, (byte) 0); + return new HoodieLogFormatVersion(HoodieLogFormatVersion.DEFAULT_VERSION); + } + return new HoodieLogFormatVersion(inputStream.readInt()); + } + + private boolean isOldMagic() { + return Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC); + } + + + private boolean readMagic() throws IOException { + try { + long pos = inputStream.getPos(); + // 1. Read magic header from the start of the block + inputStream.readFully(magicBuffer, 0, 6); + if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) { + inputStream.seek(pos); + // 1. Read old magic header from the start of the block + // (for backwards compatibility of older log files written without log version) + inputStream.readFully(oldMagicBuffer, 0, 4); + if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { + throw new CorruptedLogFileException( + logFile + "could not be read. Did not find the magic bytes at the start of the block"); + } + } + return false; + } catch (EOFException e) { + // We have reached the EOF + return true; + } + } + + @Override + public HoodieLogBlock next() { + if (nextBlock == null) { + // may be hasNext is not called + hasNext(); + } + return nextBlock; + } + + /** + * hasPrev is not idempotent + * + * @return + */ + public boolean hasPrev() { + try { + if(!this.reverseReader) { + throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); + } + reverseLogFilePosition = lastReverseLogFilePosition; + reverseLogFilePosition -= Long.BYTES; + lastReverseLogFilePosition = reverseLogFilePosition; + inputStream.seek(reverseLogFilePosition); + } catch (Exception e) { + // Either reached EOF while reading backwards or an exception + return false; + } + return true; + } + + /** + * This is a reverse iterator + * Note: At any point, an instance of HoodieLogFileReader should either iterate reverse (prev) + * or forward (next). Doing both in the same instance is not supported + * WARNING : Every call to prev() should be preceded with hasPrev() + * + * @return + * @throws IOException + */ + public HoodieLogBlock prev() throws IOException { + + if(!this.reverseReader) { + throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); + } + long blockSize = inputStream.readLong(); + long blockEndPos = inputStream.getPos(); + // blocksize should read everything about a block including the length as well + try { + inputStream.seek(reverseLogFilePosition - blockSize); + } catch (Exception e) { + // this could be a corrupt block + inputStream.seek(blockEndPos); + throw new CorruptedLogFileException("Found possible corrupted block, cannot read log file in reverse, " + + "fallback to forward reading of logfile"); + } + boolean hasNext = hasNext(); + reverseLogFilePosition -= blockSize; + lastReverseLogFilePosition = reverseLogFilePosition; + return this.nextBlock; + } + + /** + * Reverse pointer, does not read the block. Return the current position of the log file (in reverse) + * If the pointer (inputstream) is moved in any way, it is the job of the client of this class to + * seek/reset it back to the file position returned from the method to expect correct results + * + * @return + * @throws IOException + */ + public long moveToPrev() throws IOException { + + if(!this.reverseReader) { + throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); + } + inputStream.seek(lastReverseLogFilePosition); + long blockSize = inputStream.readLong(); + // blocksize should be everything about a block including the length as well + inputStream.seek(reverseLogFilePosition - blockSize); + reverseLogFilePosition -= blockSize; + lastReverseLogFilePosition = reverseLogFilePosition; + return reverseLogFilePosition; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Remove not supported for HoodieLogFileReader"); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java index 58734b1652920..d5deb9d03f51f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java @@ -19,17 +19,18 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; + /** - * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a MAGIC + * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a OLD_MAGIC * sync marker. A Block can either be a Data block, Command block or Delete Block. Data Block - * Contains log records serialized as Avro Binary Format Command Block - Specific commands like * RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of keys @@ -42,7 +43,21 @@ public interface HoodieLogFormat { * this file specific (generate a random 4 byte magic and stick it in the file header), but this I * think is suffice for now - PR */ - byte[] MAGIC = new byte[]{'H', 'U', 'D', 'I'}; + byte[] OLD_MAGIC = new byte[]{'H', 'U', 'D', 'I'}; + + /** + * Magic 6 bytes we put at the start of every block in the log file. + * This is added to maintain backwards compatiblity due to lack of log format/block + * version in older log files. All new log block will now write this OLD_MAGIC value + */ + byte[] MAGIC = new byte[]{'#', 'H', 'U', 'D', 'I', '#'}; + + /** + * The current version of the log format. Anytime the log format changes + * this version needs to be bumped and corresponding changes need to be made to + * {@link HoodieLogFormatVersion} + */ + int currentVersion = 1; /** * Writer interface to allow appending block to this file format @@ -196,9 +211,8 @@ static WriterBuilder newWriterBuilder() { return new WriterBuilder(); } - static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, - boolean readMetadata) + static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { - return new HoodieLogFormatReader(fs, logFile, readerSchema, readMetadata); + return new HoodieLogFileReader(fs, logFile, readerSchema, false, false); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index 4168e27cf10e9..f62c9f0e504f2 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -16,192 +16,85 @@ package com.uber.hoodie.common.table.log; -import com.google.common.base.Preconditions; import com.uber.hoodie.common.model.HoodieLogFile; -import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; -import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; -import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; -import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; -import com.uber.hoodie.exception.CorruptedLogFileException; import com.uber.hoodie.exception.HoodieIOException; -import com.uber.hoodie.exception.HoodieNotSupportedException; -import java.io.EOFException; -import java.io.IOException; -import java.util.Arrays; import org.apache.avro.Schema; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -/** - * Scans a log file and provides block level iterator on the log file Loads the entire block - * contents in memory Can emit either a DataBlock, CommandBlock, DeleteBlock or CorruptBlock (if one - * is found) - */ -public class HoodieLogFormatReader implements HoodieLogFormat.Reader { +import java.io.IOException; +import java.util.List; - private static final int DEFAULT_BUFFER_SIZE = 4096; - private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class); +public class HoodieLogFormatReader implements HoodieLogFormat.Reader { - private final FSDataInputStream inputStream; - private final HoodieLogFile logFile; - private static final byte[] magicBuffer = new byte[4]; + private final List logFiles; + private HoodieLogFileReader currentReader; + private final FileSystem fs; private final Schema readerSchema; - private HoodieLogBlock nextBlock = null; - private boolean readMetadata = true; + private final boolean readBlocksLazily; + private final boolean reverseLogReader; - HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, - boolean readMetadata) throws IOException { - this.inputStream = fs.open(logFile.getPath(), bufferSize); - this.logFile = logFile; + HoodieLogFormatReader(FileSystem fs, List logFiles, + Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { + this.logFiles = logFiles; + this.fs = fs; this.readerSchema = readerSchema; - this.readMetadata = readMetadata; + this.readBlocksLazily = readBlocksLazily; + this.reverseLogReader = reverseLogReader; + if(logFiles.size() > 0) { + HoodieLogFile nextLogFile = logFiles.remove(0); + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, + false); + } } - HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, - boolean readMetadata) throws IOException { - this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readMetadata); + HoodieLogFormatReader(FileSystem fs, List logFiles, + Schema readerSchema) throws IOException { + this(fs, logFiles, readerSchema, false, false); } @Override - public HoodieLogFile getLogFile() { - return logFile; - } - - private HoodieLogBlock readBlock() throws IOException { - // 2. Read the block type - int ordinal = inputStream.readInt(); - Preconditions.checkArgument(ordinal < HoodieLogBlockType.values().length, - "Invalid block byte ordinal found " + ordinal); - HoodieLogBlockType blockType = HoodieLogBlockType.values()[ordinal]; - - // 3. Read the size of the block - int blocksize = inputStream.readInt(); - - // We may have had a crash which could have written this block partially - // Skip blocksize in the stream and we should either find a sync marker (start of the next block) or EOF - // If we did not find either of it, then this block is a corrupted block. - boolean isCorrupted = isBlockCorrupt(blocksize); - if (isCorrupted) { - return createCorruptBlock(); - } - - // 4. Read the content - // TODO - have a max block size and reuse this buffer in the ByteBuffer (hard to guess max block size for now) - byte[] content = new byte[blocksize]; - inputStream.readFully(content, 0, blocksize); - - switch (blockType) { - // based on type read the block - case AVRO_DATA_BLOCK: - return HoodieAvroDataBlock.fromBytes(content, readerSchema, readMetadata); - case DELETE_BLOCK: - return HoodieDeleteBlock.fromBytes(content, readMetadata); - case COMMAND_BLOCK: - return HoodieCommandBlock.fromBytes(content, readMetadata); - default: - throw new HoodieNotSupportedException("Unsupported Block " + blockType); + public void close() throws IOException { + if (currentReader != null) { + currentReader.close(); } } - private HoodieLogBlock createCorruptBlock() throws IOException { - log.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos()); - long currentPos = inputStream.getPos(); - long nextBlockOffset = scanForNextAvailableBlockOffset(); - // Rewind to the initial start and read corrupted bytes till the nextBlockOffset - inputStream.seek(currentPos); - log.info("Next available block in " + logFile + " starts at " + nextBlockOffset); - int corruptedBlockSize = (int) (nextBlockOffset - currentPos); - byte[] content = new byte[corruptedBlockSize]; - inputStream.readFully(content, 0, corruptedBlockSize); - return HoodieCorruptBlock.fromBytes(content, corruptedBlockSize, true); - } - - private boolean isBlockCorrupt(int blocksize) throws IOException { - long currentPos = inputStream.getPos(); - try { - inputStream.seek(currentPos + blocksize); - } catch (EOFException e) { - // this is corrupt - return true; - } + @Override + public boolean hasNext() { - try { - readMagic(); - // all good - either we found the sync marker or EOF. Reset position and continue + if(currentReader == null) { return false; - } catch (CorruptedLogFileException e) { - // This is a corrupted block + } + else if (currentReader.hasNext()) { return true; - } finally { - inputStream.seek(currentPos); } - } - - private long scanForNextAvailableBlockOffset() throws IOException { - while (true) { - long currentPos = inputStream.getPos(); + else if (logFiles.size() > 0) { try { - boolean isEOF = readMagic(); - return isEOF ? inputStream.getPos() : currentPos; - } catch (CorruptedLogFileException e) { - // No luck - advance and try again - inputStream.seek(currentPos + 1); + HoodieLogFile nextLogFile = logFiles.remove(0); + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, + false); + } catch (IOException io) { + throw new HoodieIOException("unable to initialize read with log file ", io); } + return this.currentReader.hasNext(); } + return false; } @Override - public void close() throws IOException { - this.inputStream.close(); - } - - @Override - /** - * hasNext is not idempotent. TODO - Fix this. It is okay for now - PR - */ - public boolean hasNext() { - try { - boolean isEOF = readMagic(); - if (isEOF) { - return false; - } - this.nextBlock = readBlock(); - return nextBlock != null; - } catch (IOException e) { - throw new HoodieIOException("IOException when reading logfile " + logFile, e); - } - } - - private boolean readMagic() throws IOException { - try { - // 1. Read magic header from the start of the block - inputStream.readFully(magicBuffer, 0, 4); - if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) { - throw new CorruptedLogFileException( - logFile + "could not be read. Did not find the magic bytes at the start of the block"); - } - return false; - } catch (EOFException e) { - // We have reached the EOF - return true; - } + public HoodieLogBlock next() { + HoodieLogBlock block = currentReader.next(); + return block; } @Override - public HoodieLogBlock next() { - if (nextBlock == null) { - // may be hasNext is not called - hasNext(); - } - return nextBlock; + public HoodieLogFile getLogFile() { + return currentReader.getLogFile(); } @Override public void remove() { - throw new UnsupportedOperationException("Remove not supported for HoodieLogFormatReader"); } -} + +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index f326295710959..9ea4600a5326c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -22,7 +22,6 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; -import java.io.IOException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -32,6 +31,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; + /** * HoodieLogFormatWriter can be used to append blocks to a log file Use * HoodieLogFormat.WriterBuilder to construct @@ -117,16 +118,39 @@ public long getSizeThreshold() { @Override public Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException { - byte[] content = block.getBytes(); - // 1. write the magic header for the start of the block + + // Find current version + LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.currentVersion); + long currentSize = this.output.size(); + + // 1. Write the magic header for the start of the block this.output.write(HoodieLogFormat.MAGIC); - // 2. Write the block type + + // bytes for header + byte [] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); + // content bytes + byte [] content = block.getContentBytes(); + // bytes for footer + byte [] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); + + // 2. Write the total size of the block (excluding Magic) + this.output.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); + + // 3. Write the version of this log block + this.output.writeInt(currentLogFormatVersion.getVersion()); + // 4. Write the block type this.output.writeInt(block.getBlockType().ordinal()); - // 3. Write the size of the block - this.output.writeInt(content.length); - // 4. Write the contents of the block - this.output.write(content); + // 5. Write the headers for the log block + this.output.write(headerBytes); + // 6. Write the size of the content block + this.output.writeLong(content.length); + // 7. Write the contents of the data block + this.output.write(content); + // 8. Write the footers for the log block + this.output.write(footerBytes); + // 9. Write the total size of the log block (including magic) which is everything written until now (for reverse pointer) + this.output.writeLong(this.output.size() - currentSize); // Flush every block to disk flush(); @@ -134,6 +158,32 @@ public Writer appendBlock(HoodieLogBlock block) return rolloverIfNeeded(); } + /** + * + * This method returns the total LogBlock Length which is the sum of + * 1. Number of bytes to write version + * 2. Number of bytes to write ordinal + * 3. Length of the headers + * 4. Number of bytes used to write content length + * 5. Length of the content + * 6. Length of the footers + * 7. Number of bytes to write totalLogBlockLength + * @param contentLength + * @param headerLength + * @param footerLength + * @return + */ + private int getLogBlockLength(int contentLength, int headerLength, int footerLength) { + return + Integer.BYTES + // Number of bytes to write version + Integer.BYTES + // Number of bytes to write ordinal + headerLength + // Length of the headers + Long.BYTES + // Number of bytes used to write content length + contentLength + // Length of the content + footerLength + // Length of the footers + Long.BYTES; // Number of bytes to write totalLogBlockLength at end of block (for reverse pointer) + } + private Writer rolloverIfNeeded() throws IOException, InterruptedException { // Roll over if the size is past the threshold if (getCurrentSize() > sizeThreshold) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java new file mode 100644 index 0000000000000..8bba078c74322 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java @@ -0,0 +1,131 @@ +/* + * 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.table.log; + +/** + * A set of feature flags associated with a log format. + * Versions are changed when the log format changes. + * TODO(na) - Implement policies around major/minor versions + */ +abstract class LogFormatVersion { + private final int version; + + LogFormatVersion(int version) { + this.version = version; + } + + public int getVersion() { + return version; + } + + public abstract boolean hasMagicHeader(); + + public abstract boolean hasContent(); + + public abstract boolean hasContentLength(); + + public abstract boolean hasOrdinal(); + + public abstract boolean hasHeader(); + + public abstract boolean hasFooter(); + + public abstract boolean hasLogBlockLength(); +} + +/** + * Implements logic to determine behavior for feature flags for {@link LogFormatVersion} + */ +final class HoodieLogFormatVersion extends LogFormatVersion { + + public final static int DEFAULT_VERSION = 0; + + HoodieLogFormatVersion(int version) { + super(version); + } + @Override + public boolean hasMagicHeader() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } + + @Override + public boolean hasContent() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } + + @Override + public boolean hasContentLength() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } + + @Override + public boolean hasOrdinal() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } + + @Override + public boolean hasHeader() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return false; + default: + return true; + } + } + + @Override + public boolean hasFooter() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return false; + case 1: + return true; + } + return false; + } + + @Override + public boolean hasLogBlockLength() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return false; + case 1: + return true; + } + return false; + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java index ebb72bb4bdc91..94e89e793a99f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java @@ -16,6 +16,8 @@ package com.uber.hoodie.common.table.log.block; +import com.google.common.annotations.VisibleForTesting; +import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.exception.HoodieIOException; @@ -27,71 +29,95 @@ import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.Encoder; import org.apache.avro.io.EncoderFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import javax.annotation.Nonnull; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; /** * DataBlock contains a list of records serialized using Avro. * The Datablock contains - * 1. Compressed Writer Schema length - * 2. Compressed Writer Schema content - * 3. Total number of records in the block - * 4. Size of a record - * 5. Actual avro serialized content of the record + * 1. Data Block version + * 2. Total number of records in the block + * 3. Size of a record + * 4. Actual avro serialized content of the record */ public class HoodieAvroDataBlock extends HoodieLogBlock { private List records; private Schema schema; - public HoodieAvroDataBlock(List records, Schema schema, Map metadata) { - super(metadata); + public HoodieAvroDataBlock(@Nonnull List records, + @Nonnull Map header, + @Nonnull Map footer) { + super(header, footer, Optional.empty(), Optional.empty(), null, false); this.records = records; - this.schema = schema; + this.schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); } - public HoodieAvroDataBlock(List records, Schema schema) { - this(records, schema, null); + public HoodieAvroDataBlock(@Nonnull List records, + @Nonnull Map header) { + this(records, header, new HashMap<>()); } - //TODO : (na) lazily create IndexedRecords only when required - public List getRecords() { - return records; + private HoodieAvroDataBlock(Optional content, @Nonnull FSDataInputStream inputStream, + boolean readBlockLazily, Optional blockContentLocation, + Schema readerSchema, @Nonnull Map headers, + @Nonnull Map footer) { + super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily); + this.schema = readerSchema; } - public Schema getSchema() { - return schema; + public static HoodieLogBlock getBlock(HoodieLogFile logFile, + FSDataInputStream inputStream, + Optional content, + boolean readBlockLazily, + long position, + long blockSize, + long blockEndpos, + Schema readerSchema, + Map header, + Map footer) { + + return new HoodieAvroDataBlock(content, inputStream, readBlockLazily, + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), + readerSchema, header, footer); + } @Override - public byte[] getBytes() throws IOException { + public byte[] getContentBytes() throws IOException { + + // In case this method is called before realizing records from content + if (getContent().isPresent()) { + return getContent().get(); + } else if (readBlockLazily && !getContent().isPresent() && records == null) { + // read block lazily + createRecordsFromContentBytes(); + } + Schema schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); GenericDatumWriter writer = new GenericDatumWriter<>(schema); ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); - // 1. Write out metadata - if (super.getLogMetadata() != null) { - output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata())); - } + // 1. Write out the log block version + output.writeInt(HoodieLogBlock.version); - // 2. Compress and Write schema out - byte[] schemaContent = HoodieAvroUtils.compress(schema.toString()); - output.writeInt(schemaContent.length); - output.write(schemaContent); - - // 3. Write total number of records + // 2. Write total number of records output.writeInt(records.size()); - // 4. Write the records + // 3. Write the records Iterator itr = records.iterator(); while (itr.hasNext()) { IndexedRecord s = itr.next(); @@ -113,7 +139,6 @@ public byte[] getBytes() throws IOException { throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e); } } - output.close(); return baos.toByteArray(); } @@ -123,15 +148,94 @@ public HoodieLogBlockType getBlockType() { return HoodieLogBlockType.AVRO_DATA_BLOCK; } + public List getRecords() { + if (records == null) { + try { + // in case records are absent, read content lazily and then convert to IndexedRecords + createRecordsFromContentBytes(); + } catch (IOException io) { + throw new HoodieIOException("Unable to convert content bytes to records", io); + } + } + return records; + } + + public Schema getSchema() { + return schema; + } + //TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used - public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) throws IOException { + //TODO (na) - Implement a recordItr instead of recordList + private void createRecordsFromContentBytes() throws IOException { + + if (readBlockLazily && !getContent().isPresent()) { + // read log block contents from disk + inflate(); + } - SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); - Map metadata = null; - // 1. Read the metadata written out, if applicable - if (readMetadata) { - metadata = HoodieLogBlock.getLogMetadata(dis); + SizeAwareDataInputStream dis = + new SizeAwareDataInputStream( + new DataInputStream(new ByteArrayInputStream(getContent().get()))); + + // 1. Read version for this data block + int version = dis.readInt(); + HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version); + + // Get schema from the header + Schema writerSchema = new Schema.Parser() + .parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); + + // If readerSchema was not present, use writerSchema + if (schema == null) { + schema = writerSchema; + } + + GenericDatumReader reader = new GenericDatumReader<>(writerSchema, schema); + // 2. Get the total records + int totalRecords = 0; + if (logBlockVersion.hasRecordCount()) { + totalRecords = dis.readInt(); } + List records = new ArrayList<>(totalRecords); + + // 3. Read the content + for (int i = 0; i < totalRecords; i++) { + int recordLength = dis.readInt(); + Decoder decoder = DecoderFactory.get() + .binaryDecoder(getContent().get(), dis.getNumberOfBytesRead(), recordLength, null); + IndexedRecord record = reader.read(null, decoder); + records.add(record); + dis.skipBytes(recordLength); + } + dis.close(); + this.records = records; + // Free up content to be GC'd, deflate + deflate(); + } + + /*****************************************************DEPRECATED METHODS**********************************************/ + + @Deprecated + @VisibleForTesting + /** + * This constructor is retained to provide backwards compatibility to HoodieArchivedLogs + * which were written using HoodieLogFormat V1 + */ + public HoodieAvroDataBlock(List records, Schema schema) { + super(new HashMap<>(), new HashMap<>(), Optional.empty(), Optional.empty(), null, false); + this.records = records; + this.schema = schema; + } + + @Deprecated + /** + * This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using HoodieLogFormat V1 + */ + public static HoodieLogBlock getBlock(byte[] content, Schema readerSchema) throws IOException { + + SizeAwareDataInputStream dis = new SizeAwareDataInputStream( + new DataInputStream(new ByteArrayInputStream(content))); + // 1. Read the schema written out int schemaLength = dis.readInt(); byte[] compressedSchema = new byte[schemaLength]; @@ -142,21 +246,65 @@ public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, bool readerSchema = writerSchema; } - //TODO : (na) lazily create IndexedRecords only when required GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema); // 2. Get the total records int totalRecords = dis.readInt(); List records = new ArrayList<>(totalRecords); // 3. Read the content - for (int i=0;i writer = new GenericDatumWriter<>(schema); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream output = new DataOutputStream(baos); + + // 2. Compress and Write schema out + byte[] schemaContent = HoodieAvroUtils.compress(schema.toString()); + output.writeInt(schemaContent.length); + output.write(schemaContent); + + // 3. Write total number of records + output.writeInt(records.size()); + + // 4. Write the records + Iterator itr = records.iterator(); + while (itr.hasNext()) { + IndexedRecord s = itr.next(); + ByteArrayOutputStream temp = new ByteArrayOutputStream(); + Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null); + try { + // Encode the record into bytes + writer.write(s, encoder); + encoder.flush(); + + // Get the size of the bytes + int size = temp.toByteArray().length; + // Write the record size + output.writeInt(size); + // Write the content + output.write(temp.toByteArray()); + itr.remove(); + } catch (IOException e) { + throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e); + } + } + + output.close(); + return baos.toByteArray(); + } + } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java index 0c6dc9a52fba5..b9b11cc8f59c5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java @@ -16,14 +16,12 @@ package com.uber.hoodie.common.table.log.block; -import com.uber.hoodie.common.storage.SizeAwareDataInputStream; +import com.uber.hoodie.common.model.HoodieLogFile; +import org.apache.hadoop.fs.FSDataInputStream; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; +import java.util.HashMap; import java.util.Map; +import java.util.Optional; /** * Command block issues a specific command to the scanner @@ -34,27 +32,16 @@ public class HoodieCommandBlock extends HoodieLogBlock { public enum HoodieCommandBlockTypeEnum {ROLLBACK_PREVIOUS_BLOCK} - public HoodieCommandBlock(HoodieCommandBlockTypeEnum type, - Map metadata) { - super(metadata); - this.type = type; + public HoodieCommandBlock(Map header) { + this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>()); } - public HoodieCommandBlock(HoodieCommandBlockTypeEnum type) { - this(type, null); - } - - @Override - public byte[] getBytes() throws IOException { - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream output = new DataOutputStream(baos); - if (super.getLogMetadata() != null) { - output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata())); - } - output.writeInt(type.ordinal()); - output.close(); - return baos.toByteArray(); + private HoodieCommandBlock(Optional content, FSDataInputStream inputStream, + boolean readBlockLazily, Optional blockContentLocation, + Map header, Map footer) { + super(header, footer, blockContentLocation, content, inputStream, readBlockLazily); + this.type = HoodieCommandBlockTypeEnum.values()[Integer + .parseInt(header.get(HeaderMetadataType.COMMAND_BLOCK_TYPE))]; } public HoodieCommandBlockTypeEnum getType() { @@ -66,13 +53,23 @@ public HoodieLogBlockType getBlockType() { return HoodieLogBlockType.COMMAND_BLOCK; } - public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException { - SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); - Map metadata = null; - if (readMetadata) { - metadata = HoodieLogBlock.getLogMetadata(dis); - } - int ordinal = dis.readInt(); - return new HoodieCommandBlock(HoodieCommandBlockTypeEnum.values()[ordinal], metadata); + @Override + public byte[] getContentBytes() { + return new byte[0]; + } + + public static HoodieLogBlock getBlock(HoodieLogFile logFile, + FSDataInputStream inputStream, + Optional content, + boolean readBlockLazily, + long position, + long blockSize, + long blockEndpos, + Map header, + Map footer) { + + return new HoodieCommandBlock(content, inputStream, readBlockLazily, + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), + header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java index 5819c99e1d9e8..c75c8ea622325 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java @@ -16,14 +16,12 @@ package com.uber.hoodie.common.table.log.block; -import com.uber.hoodie.common.storage.SizeAwareDataInputStream; +import com.uber.hoodie.common.model.HoodieLogFile; +import org.apache.hadoop.fs.FSDataInputStream; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.util.Map; +import java.util.Optional; /** * Corrupt block is emitted whenever the scanner finds the length of the block written at the @@ -31,26 +29,20 @@ */ public class HoodieCorruptBlock extends HoodieLogBlock { - private final byte[] corruptedBytes; - - private HoodieCorruptBlock(byte[] corruptedBytes, Map metadata) { - super(metadata); - this.corruptedBytes = corruptedBytes; - } - - private HoodieCorruptBlock(byte[] corruptedBytes) { - this(corruptedBytes, null); + private HoodieCorruptBlock(Optional corruptedBytes, FSDataInputStream inputStream, + boolean readBlockLazily, Optional blockContentLocation, + Map header, Map footer) { + super(header, footer, blockContentLocation, corruptedBytes, inputStream, readBlockLazily); } @Override - public byte[] getBytes() throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream output = new DataOutputStream(baos); - if (super.getLogMetadata() != null) { - output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata())); + public byte[] getContentBytes() throws IOException { + + if (!getContent().isPresent() && readBlockLazily) { + // read content from disk + inflate(); } - output.write(corruptedBytes); - return baos.toByteArray(); + return getContent().get(); } @Override @@ -58,26 +50,17 @@ public HoodieLogBlockType getBlockType() { return HoodieLogBlockType.CORRUPT_BLOCK; } - public byte[] getCorruptedBytes() { - return corruptedBytes; - } + public static HoodieLogBlock getBlock(HoodieLogFile logFile, + FSDataInputStream inputStream, + Optional corruptedBytes, + boolean readBlockLazily, + long position, + long blockSize, + long blockEndPos, + Map header, + Map footer) throws IOException { - public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata) - throws IOException { - SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); - Map metadata = null; - int bytesRemaining = blockSize; - if (readMetadata) { - try { //attempt to read metadata - metadata = HoodieLogBlock.getLogMetadata(dis); - bytesRemaining = blockSize - HoodieLogBlock.getLogMetadataBytes(metadata).length; - } catch (IOException e) { - // unable to read metadata, possibly corrupted - metadata = null; - } - } - byte[] corruptedBytes = new byte[bytesRemaining]; - dis.readFully(corruptedBytes); - return new HoodieCorruptBlock(corruptedBytes, metadata); + return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily, + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index 3751124fec96a..4de25b5daef7f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -16,48 +16,82 @@ package com.uber.hoodie.common.table.log.block; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.storage.SizeAwareDataInputStream; +import com.uber.hoodie.exception.HoodieIOException; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FSDataInputStream; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.nio.charset.Charset; +import java.util.HashMap; import java.util.Map; - -import com.uber.hoodie.common.storage.SizeAwareDataInputStream; -import org.apache.commons.lang3.StringUtils; +import java.util.Optional; /** * Delete block contains a list of keys to be deleted from scanning the blocks so far */ public class HoodieDeleteBlock extends HoodieLogBlock { - private final String[] keysToDelete; + private String[] keysToDelete; - public HoodieDeleteBlock(String[] keysToDelete, Map metadata) { - super(metadata); + public HoodieDeleteBlock(String[] keysToDelete, + Map header) { + this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>()); this.keysToDelete = keysToDelete; } - public HoodieDeleteBlock(String[] keysToDelete) { - this(keysToDelete, null); + + private HoodieDeleteBlock(Optional content, FSDataInputStream inputStream, + boolean readBlockLazily, Optional blockContentLocation, + Map header, Map footer) { + super(header, footer, blockContentLocation, content, inputStream, readBlockLazily); } @Override - public byte[] getBytes() throws IOException { + public byte[] getContentBytes() throws IOException { + + // In case this method is called before realizing keys from content + if (getContent().isPresent()) { + return getContent().get(); + } else if (readBlockLazily && !getContent().isPresent() && keysToDelete == null) { + // read block lazily + getKeysToDelete(); + } + ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); - if (super.getLogMetadata() != null) { - output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata())); - } - byte[] bytesToWrite = StringUtils.join(keysToDelete, ',').getBytes(Charset.forName("utf-8")); + byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',').getBytes(Charset.forName("utf-8")); + output.writeInt(HoodieLogBlock.version); output.writeInt(bytesToWrite.length); output.write(bytesToWrite); return baos.toByteArray(); } public String[] getKeysToDelete() { - return keysToDelete; + try { + if (keysToDelete == null) { + if (!getContent().isPresent() && readBlockLazily) { + // read content from disk + inflate(); + } + SizeAwareDataInputStream dis = + new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get()))); + int version = dis.readInt(); + int dataLength = dis.readInt(); + byte[] data = new byte[dataLength]; + dis.readFully(data); + this.keysToDelete = new String(data).split(","); + deflate(); + } + return keysToDelete; + } catch (IOException io) { + throw new HoodieIOException("Unable to generate keys to delete from block content", io); + } } @Override @@ -65,15 +99,17 @@ public HoodieLogBlockType getBlockType() { return HoodieLogBlockType.DELETE_BLOCK; } - public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException { - SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content))); - Map metadata = null; - if (readMetadata) { - metadata = HoodieLogBlock.getLogMetadata(dis); - } - int dataLength = dis.readInt(); - byte[] data = new byte[dataLength]; - dis.readFully(data); - return new HoodieDeleteBlock(new String(data).split(","), metadata); + public static HoodieLogBlock getBlock(HoodieLogFile logFile, + FSDataInputStream inputStream, + Optional content, + boolean readBlockLazily, + long position, + long blockSize, + long blockEndPos, + Map header, + Map footer) throws IOException { + + return new HoodieDeleteBlock(content, inputStream, readBlockLazily, + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index d21332f2fd18f..e7735b0dbda46 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -17,21 +17,66 @@ package com.uber.hoodie.common.table.log.block; import com.google.common.collect.Maps; -import com.uber.hoodie.common.storage.SizeAwareDataInputStream; +import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; import java.io.IOException; import java.util.Map; +import java.util.Optional; +import javax.annotation.Nonnull; +import org.apache.hadoop.fs.FSDataInputStream; /** * Abstract class defining a block in HoodieLogFile */ public abstract class HoodieLogBlock { - public byte[] getBytes() throws IOException { + /** + * The current version of the log block. Anytime the logBlock format changes + * this version needs to be bumped and corresponding changes need to be made to + * {@link HoodieLogBlockVersion} + * TODO : Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override operations there + */ + public static int version = 1; + // Header for each log block + private final Map logBlockHeader; + // Footer for each log block + private final Map logBlockFooter; + // Location of a log block on disk + private final Optional blockContentLocation; + // data for a specific block + private Optional content; + // TODO : change this to just InputStream so this works for any FileSystem + // create handlers to return specific type of inputstream based on FS + // input stream corresponding to the log file where this logBlock belongs + protected FSDataInputStream inputStream; + // Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive) + protected boolean readBlockLazily; + + public HoodieLogBlock(@Nonnull Map logBlockHeader, + @Nonnull Map logBlockFooter, + @Nonnull Optional blockContentLocation, + @Nonnull Optional content, + FSDataInputStream inputStream, + boolean readBlockLazily) { + this.logBlockHeader = logBlockHeader; + this.logBlockFooter = logBlockFooter; + this.blockContentLocation = blockContentLocation; + this.content = content; + this.inputStream = inputStream; + this.readBlockLazily = readBlockLazily; + } + + // Return the bytes representation of the data belonging to a LogBlock + public byte[] getContentBytes() throws IOException { + throw new HoodieException("No implementation was provided"); + } + + public byte [] getMagic() { throw new HoodieException("No implementation was provided"); } @@ -39,8 +84,25 @@ public HoodieLogBlockType getBlockType() { throw new HoodieException("No implementation was provided"); } - //log metadata for each log block - private Map logMetadata; + public long getLogBlockLength() { + throw new HoodieException("No implementation was provided"); + } + + public Optional getBlockContentLocation() { + return this.blockContentLocation; + } + + public Map getLogBlockHeader() { + return logBlockHeader; + } + + public Map getLogBlockFooter() { + return logBlockFooter; + } + + public Optional getContent() { + return content; + } /** * Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at @@ -54,32 +116,71 @@ public enum HoodieLogBlockType { } /** - * Metadata abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. + * Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. * Only add new enums at the end. */ - public enum LogMetadataType { + public enum HeaderMetadataType { INSTANT_TIME, - TARGET_INSTANT_TIME + TARGET_INSTANT_TIME, + SCHEMA, + COMMAND_BLOCK_TYPE } - public HoodieLogBlock(Map logMetadata) { - this.logMetadata = logMetadata; + /** + * Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. + * Only add new enums at the end. + */ + public enum FooterMetadataType { } - public Map getLogMetadata() { - return logMetadata; + /** + * This class is used to store the Location of the Content of a Log Block. It's used when a client chooses for a + * IO intensive CompactedScanner, the location helps to lazily read contents from the log file + */ + public static final class HoodieLogBlockContentLocation { + // The logFile that contains this block + private final HoodieLogFile logFile; + // The filePosition in the logFile for the contents of this block + private final long contentPositionInLogFile; + // The number of bytes / size of the contents of this block + private final long blockSize; + // The final position where the complete block ends + private final long blockEndPos; + + HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, long blockSize, long blockEndPos) { + this.logFile = logFile; + this.contentPositionInLogFile = contentPositionInLogFile; + this.blockSize = blockSize; + this.blockEndPos = blockEndPos; + } + + public HoodieLogFile getLogFile() { + return logFile; + } + + public long getContentPositionInLogFile() { + return contentPositionInLogFile; + } + + public long getBlockSize() { + return blockSize; + } + + public long getBlockEndPos() { + return blockEndPos; + } } /** * Convert log metadata to bytes 1. Write size of metadata 2. Write enum ordinal 3. Write actual * bytes */ - public static byte[] getLogMetadataBytes(Map metadata) + public static byte[] getLogMetadataBytes(Map metadata) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); output.writeInt(metadata.size()); - for (Map.Entry entry : metadata.entrySet()) { + for (Map.Entry entry : metadata.entrySet()) { output.writeInt(entry.getKey().ordinal()); byte[] bytes = entry.getValue().getBytes(); output.writeInt(bytes.length); @@ -91,10 +192,10 @@ public static byte[] getLogMetadataBytes(Map metadata) /** * Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes} */ - public static Map getLogMetadata(SizeAwareDataInputStream dis) + public static Map getLogMetadata(DataInputStream dis) throws IOException { - Map metadata = Maps.newHashMap(); + Map metadata = Maps.newHashMap(); // 1. Read the metadata written out int metadataCount = dis.readInt(); try { @@ -103,7 +204,7 @@ public static Map getLogMetadata(SizeAwareDataInputStre int metadataEntrySize = dis.readInt(); byte[] metadataEntry = new byte[metadataEntrySize]; dis.readFully(metadataEntry, 0, metadataEntrySize); - metadata.put(LogMetadataType.values()[metadataEntryIndex], new String(metadataEntry)); + metadata.put(HeaderMetadataType.values()[metadataEntryIndex], new String(metadataEntry)); metadataCount--; } return metadata; @@ -111,4 +212,60 @@ public static Map getLogMetadata(SizeAwareDataInputStre throw new IOException("Could not read metadata fields ", eof); } } + + /** + * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in + * {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner} + * + * @param inputStream + * @param contentLength + * @param readBlockLazily + * @return + * @throws IOException + */ + public static byte [] readOrSkipContent(FSDataInputStream inputStream, + Integer contentLength, boolean readBlockLazily) throws IOException { + byte [] content = null; + if (!readBlockLazily) { + // Read the contents in memory + content = new byte[contentLength]; + inputStream.readFully(content, 0, contentLength); + } else { + // Seek to the end of the content block + inputStream.seek(inputStream.getPos() + contentLength); + } + return content; + } + + /** + * When lazyReading of blocks is turned on, inflate the content of a log block from disk + * @throws IOException + */ + protected void inflate() throws IOException { + + try { + content = Optional.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]); + inputStream.seek(this.getBlockContentLocation().get().getContentPositionInLogFile()); + inputStream.readFully(content.get(), 0, content.get().length); + inputStream.seek(this.getBlockContentLocation().get().getBlockEndPos()); + } catch(IOException e) { + try { + // TODO : fs.open() and return inputstream again, need to pass FS configuration + // because the inputstream might close/timeout for large number of log blocks to be merged + inflate(); + } catch(IOException io) { + throw new HoodieIOException("unable to lazily read log block from disk", io); + } + } + } + + /** + * After the content bytes is converted into the required DataStructure by a logBlock, deflate the content + * to release byte [] and relieve memory pressure when GC kicks in. + * NOTE: This still leaves the heap fragmented + */ + protected void deflate() { + content = Optional.empty(); + } + } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java new file mode 100644 index 0000000000000..1a984420935dc --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java @@ -0,0 +1,79 @@ +/* + * 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.table.log.block; + +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.version; + +abstract class HoodieLogBlockVersion { + + private final int currentVersion; + + public final static int DEFAULT_VERSION = 0; + + HoodieLogBlockVersion(int version) { + this.currentVersion = version; + } + + int getVersion() { + return currentVersion; + } +} + +/** + * A set of feature flags associated with a data log block format. + * Versions are changed when the log block format changes. + * TODO(na) - Implement policies around major/minor versions + */ +final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion { + + HoodieAvroDataBlockVersion(int version) { + super(version); + } + + public boolean hasRecordCount() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } +} + +/** + * A set of feature flags associated with a command log block format. + * Versions are changed when the log block format changes. + * TODO(na) - Implement policies around major/minor versions + */ +final class HoodieCommandBlockVersion extends HoodieLogBlockVersion { + + HoodieCommandBlockVersion(int version) { + super(version); + } +} + +/** + * A set of feature flags associated with a delete log block format. + * Versions are changed when the log block format changes. + * TODO(na) - Implement policies around major/minor versions + */ +final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion { + + HoodieDeleteBlockVersion(int version) { + super(version); + } +} \ No newline at end of file diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index db0e87d6fc666..4db0a58b78d0f 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -291,8 +291,9 @@ public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema .overBaseCommit(location.getCommitTime()) .withFs(fs).build(); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, location.getCommitTime()); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getCommitTime()); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> { try { GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); @@ -304,7 +305,7 @@ public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema } catch (IOException e) { return null; } - }).collect(Collectors.toList()), schema, metadata)); + }).collect(Collectors.toList()), header)); logWriter.close(); } catch (Exception e) { fail(e.toString()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index e57c7ebe7335c..0acd77fb79056 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -27,7 +27,6 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; -import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum; import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; @@ -35,7 +34,7 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; -import com.uber.hoodie.common.util.collection.DiskBasedMap; +import com.uber.hoodie.exception.CorruptedLogFileException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -49,12 +48,15 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.IOException; import java.net.URISyntaxException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -68,12 +70,26 @@ import static org.junit.Assert.fail; @SuppressWarnings("Duplicates") +@RunWith(Parameterized.class) public class HoodieLogFormatTest { private FileSystem fs; private Path partitionPath; private static String basePath; + private Boolean readBlocksLazily = true; + + public HoodieLogFormatTest(Boolean readBlocksLazily) { + this.readBlocksLazily = readBlocksLazily; + } + + @Parameterized.Parameters(name = "LogBlockReadMode") + public static Collection data() { + return Arrays.asList(new Boolean[][]{ + {true},{false} + }); + } + @BeforeClass public static void setUpClass() throws IOException, InterruptedException { // Append is not supported in LocalFileSystem. HDFS needs to be setup. @@ -119,10 +135,10 @@ public void testBasicAppend() throws IOException, InterruptedException, URISynta .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records = SchemaTestUtil.generateTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size = writer.getCurrentSize(); assertTrue("We just wrote a block - size should be > 0", size > 0); @@ -138,10 +154,10 @@ public void testRollover() throws IOException, InterruptedException, URISyntaxEx .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records = SchemaTestUtil.generateTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); // Write out a block writer = writer.appendBlock(dataBlock); // Get the size of the block @@ -153,8 +169,8 @@ public void testRollover() throws IOException, InterruptedException, URISyntaxEx .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build(); records = SchemaTestUtil.generateTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); assertEquals("This should be a new log file and hence size should be 0", 0, writer.getCurrentSize()); @@ -168,10 +184,10 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records = SchemaTestUtil.generateTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 = writer.getCurrentSize(); writer.close(); @@ -180,8 +196,8 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); @@ -195,8 +211,8 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size3 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2); @@ -220,10 +236,10 @@ public void testLeaseRecovery() throws IOException, URISyntaxException, Interrup .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records = SchemaTestUtil.generateTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 = writer.getCurrentSize(); // do not close this writer - this simulates a data note appending to a log dying without closing the file @@ -233,8 +249,8 @@ public void testLeaseRecovery() throws IOException, URISyntaxException, Interrup .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); @@ -256,10 +272,10 @@ public void testAppendNotSupported() // Some data & append two times. List records = SchemaTestUtil.generateTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); for (int i = 0; i < 2; i++) { HoodieLogFormat.newWriterBuilder().onParentPath(testPath) @@ -284,15 +300,15 @@ public void testBasicWriteAndScan() List copyOfRecords = records.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true); + .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("We wrote a block, we should be able to read it", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK, @@ -316,10 +332,10 @@ public void testBasicAppendAndRead() List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -330,8 +346,8 @@ public void testBasicAppendAndRead() List copyOfRecords2 = records2.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - dataBlock = new HoodieAvroDataBlock(records2, - getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -343,13 +359,13 @@ public void testBasicAppendAndRead() List copyOfRecords3 = records3.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - dataBlock = new HoodieAvroDataBlock(records3, - getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true); + .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; @@ -373,6 +389,51 @@ public void testBasicAppendAndRead() dataBlockRead.getRecords()); } + @SuppressWarnings("unchecked") + @Test + public void testBasicAppendAndScanMultipleFiles() + throws IOException, URISyntaxException, InterruptedException { + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withSizeThreshold(1024).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + + Set logFiles = new HashSet<>(); + List> allRecords = new ArrayList<>(); + // create 4 log files + while(writer.getLogFile().getLogVersion() != 4) { + logFiles.add(writer.getLogFile()); + List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + allRecords.add(copyOfRecords1); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); + writer = writer.appendBlock(dataBlock); + } + writer.close(); + + // scan all log blocks (across multiple log files) + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, + logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), + schema, "100", 10240L, readBlocksLazily, false); + + List scannedRecords = new ArrayList<>(); + for(HoodieRecord record: scanner) { + scannedRecords.add((IndexedRecord) record.getData().getInsertValue(schema).get()); + } + + assertEquals("Scanner records count should be the same as appended records", + scannedRecords.size(), allRecords.stream().flatMap(records -> records.stream()) + .collect(Collectors.toList()).size()); + + } + + @Test public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxException, InterruptedException { @@ -380,10 +441,10 @@ public void testAppendAndReadOnCorruptedLog() .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records = SchemaTestUtil.generateTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, - getSimpleSchema(), metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -392,19 +453,20 @@ public void testAppendAndReadOnCorruptedLog() FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); + // Write out a length that does not confirm with the content + outputStream.writeLong(1000); outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.currentVersion); // Write out a length that does not confirm with the content - outputStream.writeInt(1000); - // Write out some metadata - // TODO : test for failure to write metadata - NA ? - outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata)); + outputStream.writeLong(500); + // Write out some bytes outputStream.write("something-random".getBytes()); outputStream.flush(); outputStream.close(); // First round of reads - we should be able to read the first block and then EOF Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true); + .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); reader.next(); assertTrue("We should have corrupted block next", reader.hasNext()); @@ -412,19 +474,20 @@ public void testAppendAndReadOnCorruptedLog() assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block; - assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes())); + //assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes())); assertFalse("There should be no more block left", reader.hasNext()); // Simulate another failure back to back outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); + // Write out a length that does not confirm with the content + outputStream.writeLong(1000); outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.currentVersion); // Write out a length that does not confirm with the content - outputStream.writeInt(1000); - // Write out some metadata - // TODO : test for failure to write metadata - NA ? - outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata)); + outputStream.writeLong(500); + // Write out some bytes outputStream.write("something-else-random".getBytes()); outputStream.flush(); outputStream.close(); @@ -434,13 +497,14 @@ public void testAppendAndReadOnCorruptedLog() .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema(), metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); // Second round of reads - we should be able to read the first and last block reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true); + .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); reader.next(); assertTrue("We should get the 1st corrupted block next", reader.hasNext()); @@ -450,7 +514,7 @@ public void testAppendAndReadOnCorruptedLog() assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); corruptBlock = (HoodieCorruptBlock) block; - assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes())); + //assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes())); assertTrue("We should get the last block next", reader.hasNext()); reader.next(); assertFalse("We should have no more blocks left", reader.hasNext()); @@ -471,10 +535,10 @@ public void testAvroLogRecordReaderBasic() HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); // Write 2 @@ -482,7 +546,8 @@ public void testAvroLogRecordReaderBasic() List copyOfRecords2 = records2.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -493,7 +558,7 @@ public void testAvroLogRecordReaderBasic() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "100", 10240L); + schema, "100", 10240L, readBlocksLazily, false); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -521,32 +586,35 @@ public void testAvroLogRecordReaderWithRollbackTombstone() List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); // Write 2 - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); // Rollback the last write - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); // Write 3 - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords3 = records3.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - dataBlock = new HoodieAvroDataBlock(records3, schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -557,8 +625,9 @@ public void testAvroLogRecordReaderWithRollbackTombstone() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "102", 10240L); - assertEquals("We only read 200 records, but only 200 of them are valid", 200, scanner.getTotalLogRecords()); + schema, "102", 10240L, readBlocksLazily, false); + assertEquals("We read 200 records from 2 write batches", 200, + scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); @@ -585,48 +654,53 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); writer.close(); // Write 2 - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); - outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); // Write out a length that does not confirm with the content - outputStream.writeInt(100); - // Write out some metadata - // TODO : test for failure to write metadata - NA ? - outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata)); + outputStream.writeLong(1000); + + outputStream.writeInt(HoodieLogFormat.currentVersion); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + + // Write out some header + outputStream.write(HoodieLogBlock.getLogMetadataBytes(header)); + outputStream.writeLong("something-random".getBytes().length); outputStream.write("something-random".getBytes()); outputStream.flush(); outputStream.close(); // Rollback the last write - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); writer = writer.appendBlock(commandBlock); // Write 3 - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords3 = records3.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - dataBlock = new HoodieAvroDataBlock(records3, schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -637,8 +711,9 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "103", 10240L); - assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); + schema, "103", 10240L, true, false); + assertEquals("We would read 200 records", 200, + scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); @@ -665,19 +740,19 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); // Write 2 - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords2 = records2.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); + dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); copyOfRecords1.addAll(copyOfRecords2); @@ -689,9 +764,9 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - metadata); + header); writer = writer.appendBlock(deleteBlock); List allLogFiles = FSUtils @@ -701,8 +776,9 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, - schema, "102", 10240L); - assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); + schema, "102", 10240L, readBlocksLazily, false); + assertEquals("We still would read 200 records", 200, + scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 150, readKeys.size()); @@ -713,14 +789,16 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() readKeys); // Rollback the last block - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "102"); - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "102"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); readKeys.clear(); - scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L); + scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, + false); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); @@ -741,17 +819,18 @@ public void testAvroLogRecordReaderWithFailedRollbacks() List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - dataBlock = new HoodieAvroDataBlock(records2, schema, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); List originalKeys = copyOfRecords1.stream() @@ -762,12 +841,13 @@ public void testAvroLogRecordReaderWithFailedRollbacks() // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - metadata); + header); writer = writer.appendBlock(deleteBlock); // Attempt 1 : Write rollback block for a failed write - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); try { writer = writer.appendBlock(commandBlock); // Say job failed, retry writing 2 rollback in the next rollback(..) attempt @@ -785,7 +865,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks() // all data must be rolled back before merge HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L); + allLogFiles, schema, "100", 10240L, readBlocksLazily, false); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(); @@ -809,11 +889,11 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); List originalKeys = copyOfRecords1.stream() @@ -824,12 +904,13 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - metadata); + header); writer = writer.appendBlock(deleteBlock); // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); writer = writer.appendBlock(commandBlock); @@ -839,7 +920,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L); + allLogFiles, schema, "100", 10240L, readBlocksLazily, false); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -855,16 +936,17 @@ public void testAvroLogRecordReaderWithInvalidRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); // Write invalid rollback for a failed write (possible for in-flight commits) - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101"); - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); List allLogFiles = FSUtils @@ -873,7 +955,7 @@ public void testAvroLogRecordReaderWithInvalidRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L); + allLogFiles, schema, "100", 10240L, readBlocksLazily, false); assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); @@ -897,11 +979,11 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() List copyOfRecords1 = records1.stream().map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) .collect(Collectors.toList()); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); @@ -914,13 +996,14 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - metadata); + header); writer = writer.appendBlock(deleteBlock); // Write 1 rollback block for a failed write - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); List allLogFiles = FSUtils @@ -929,7 +1012,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101", 10240L); + allLogFiles, schema, "101", 10240L, readBlocksLazily, false); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -947,11 +1030,11 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); - metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100"); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, - schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); @@ -962,9 +1045,11 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeLong(1000); outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.currentVersion); // Write out a length that does not confirm with the content - outputStream.writeInt(100); + outputStream.writeLong(100); outputStream.flush(); outputStream.close(); @@ -973,9 +1058,11 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeLong(1000); outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.currentVersion); // Write out a length that does not confirm with the content - outputStream.writeInt(100); + outputStream.writeLong(100); outputStream.flush(); outputStream.close(); @@ -991,9 +1078,11 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeLong(1000); outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.currentVersion); // Write out a length that does not confirm with the content - outputStream.writeInt(100); + outputStream.writeLong(100); outputStream.flush(); outputStream.close(); @@ -1001,9 +1090,10 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); // Write 1 rollback block for a failed write - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101"); - HoodieCommandBlock commandBlock = new HoodieCommandBlock( - HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); List allLogFiles = FSUtils @@ -1012,9 +1102,282 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() .collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101", 10240L); + allLogFiles, schema, "101", 10240L, readBlocksLazily, false); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } + @Test + public void testMagicAndLogVersionsBackwardsCompatibility() + throws IOException, InterruptedException, URISyntaxException { + // Create the log file + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + List records = SchemaTestUtil.generateHoodieTestRecords(0, 100); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + writer.close(); + + // Write 1 with OLD_MAGIC and no log format version + // Append a log block to end of the log (mimics a log block with old format + // fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.OLD_MAGIC); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + // Write out a length that does not confirm with the content + records = SchemaTestUtil.generateHoodieTestRecords(0, 100); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, new String(HoodieAvroUtils.compress(schema.toString()))); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema); + byte [] content = dataBlock.getBytes(schema); + outputStream.writeInt(content.length); + // Write out some content + outputStream.write(content); + outputStream.flush(); + outputStream.hflush(); + outputStream.close(); + + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + + // Write 2 with MAGIC and latest log format version + records = SchemaTestUtil.generateHoodieTestRecords(0, 100); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records, header); + writer = writer.appendBlock(dataBlock); + + // Write 3 with MAGIC and latest log format version + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + records = SchemaTestUtil.generateHoodieTestRecords(0, 100); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = new HoodieAvroDataBlock(records, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + Reader reader = HoodieLogFormat + .newReader(fs, writer.getLogFile(), schema); + + // Read the first block written with latest version and magic + reader.hasNext(); + HoodieLogBlock block = reader.next(); + assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK); + HoodieAvroDataBlock dBlock = (HoodieAvroDataBlock) block; + assertEquals(dBlock.getRecords().size(), 100); + + // Read second block written with old magic and no version + reader.hasNext(); + block = reader.next(); + assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK); + dBlock = (HoodieAvroDataBlock) block; + assertEquals(dBlock.getRecords().size(), 100); + + //Read third block written with latest version and magic + reader.hasNext(); + block = reader.next(); + assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK); + dBlock = (HoodieAvroDataBlock) block; + assertEquals(dBlock.getRecords().size(), 100); + + } + + @SuppressWarnings("unchecked") + @Test + public void testBasicAppendAndReadInReverse() + throws IOException, URISyntaxException, InterruptedException { + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + Schema schema = getSimpleSchema(); + List records1 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + List records2 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords2 = records2.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + dataBlock = new HoodieAvroDataBlock(records2, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + // Close and Open again and append 100 more records + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + List records3 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords3 = records3.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + dataBlock = new HoodieAvroDataBlock(records3, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + HoodieLogFileReader reader = + new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily, + true); + + assertTrue("Last block should be available", reader.hasPrev()); + HoodieLogBlock prevBlock = reader.prev(); + HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock; + + assertEquals("Third records size should be equal to the written records size", + copyOfRecords3.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, + dataBlockRead.getRecords()); + + assertTrue("Second block should be available", reader.hasPrev()); + prevBlock = reader.prev(); + dataBlockRead = (HoodieAvroDataBlock) prevBlock; + assertEquals("Read records size should be equal to the written records size", + copyOfRecords2.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, + dataBlockRead.getRecords()); + + assertTrue("First block should be available", reader.hasPrev()); + prevBlock = reader.prev(); + dataBlockRead = (HoodieAvroDataBlock) prevBlock; + assertEquals("Read records size should be equal to the written records size", + copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, + dataBlockRead.getRecords()); + + assertFalse(reader.hasPrev()); + } + + @Test + public void testAppendAndReadOnCorruptedLogInReverse() + throws IOException, URISyntaxException, InterruptedException { + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + Schema schema = getSimpleSchema(); + List records = SchemaTestUtil.generateTestRecords(0, 100); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.OLD_MAGIC); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + // Write out a length that does not confirm with the content + outputStream.writeInt(1000); + // Write out footer length + outputStream.writeInt(1); + // Write out some metadata + // TODO : test for failure to write metadata - NA ? + outputStream.write(HoodieLogBlock.getLogMetadataBytes(header)); + outputStream.write("something-random".getBytes()); + outputStream.flush(); + outputStream.close(); + + // Should be able to append a new block + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + records = SchemaTestUtil.generateTestRecords(0, 100); + dataBlock = new HoodieAvroDataBlock(records, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + // First round of reads - we should be able to read the first block and then EOF + HoodieLogFileReader reader = + new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true); + + assertTrue("Last block should be available", reader.hasPrev()); + HoodieLogBlock block = reader.prev(); + assertTrue("Last block should be datablock", block instanceof HoodieAvroDataBlock); + + assertTrue("Last block should be available", reader.hasPrev()); + try { + reader.prev(); + } catch(CorruptedLogFileException e) { + e.printStackTrace(); + // We should have corrupted block + } + } + + @SuppressWarnings("unchecked") + @Test + public void testBasicAppendAndTraverseInReverse() + throws IOException, URISyntaxException, InterruptedException { + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + Schema schema = getSimpleSchema(); + List records1 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords1 = records1.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + List records2 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords2 = records2.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + dataBlock = new HoodieAvroDataBlock(records2, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + // Close and Open again and append 100 more records + writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") + .overBaseCommit("100").withFs(fs).build(); + List records3 = SchemaTestUtil.generateTestRecords(0, 100); + List copyOfRecords3 = records3.stream().map(record -> + HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) + .collect(Collectors.toList()); + dataBlock = new HoodieAvroDataBlock(records3, header); + writer = writer.appendBlock(dataBlock); + writer.close(); + + HoodieLogFileReader reader = + new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily, + true); + + assertTrue("Third block should be available", reader.hasPrev()); + reader.moveToPrev(); + + assertTrue("Second block should be available", reader.hasPrev()); + reader.moveToPrev(); + + // After moving twice, this last reader.prev() should read the First block written + assertTrue("First block should be available", reader.hasPrev()); + HoodieLogBlock prevBlock = reader.prev(); + HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock; + assertEquals("Read records size should be equal to the written records size", + copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, + dataBlockRead.getRecords()); + + assertFalse(reader.hasPrev()); + } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 9f231084a53ef..81cae359ba30e 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -72,6 +72,11 @@ public class HoodieRealtimeRecordReader implements RecordReader deltaRecordMap; @@ -132,7 +137,8 @@ private void readAndCompactLog(JobConf jobConf) throws IOException { split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime(), (long) Math.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) - *jobConf.getMemoryForMapTask())); + *jobConf.getMemoryForMapTask()), + Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), false); // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit // but can return records for completed commits > the commit we are trying to read (if using readCommit() API) for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { @@ -140,6 +146,7 @@ private void readAndCompactLog(JobConf jobConf) throws IOException { .get(); String key = hoodieRecord.getRecordKey(); // we assume, a later safe record in the log, is newer than what we have in the map & replace it. + // TODO : handle deletes here ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema); deltaRecordMap.put(key, aWritable); if (LOG.isDebugEnabled()) { @@ -302,6 +309,7 @@ key, arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); } if (deltaRecordMap.containsKey(key)) { + // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? Writable[] replaceValue = deltaRecordMap.get(key).get(); Writable[] originalValue = arrayWritable.get(); System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 2bef7780a1ed9..12c010e9860d5 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -91,9 +91,10 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); } Schema writeSchema = records.get(0).getSchema(); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, newCommit); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, writeSchema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size = writer.getCurrentSize(); return writer; diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 95d1d58212a0c..69ae7aff54c28 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -395,7 +395,7 @@ private MessageType readSchemaFromLastCompaction(Optional lastCom @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private MessageType readSchemaFromLogFile(Optional lastCompactionCommitOpt, Path path) throws IOException { - Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null, true); + Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); HoodieAvroDataBlock lastBlock = null; while (reader.hasNext()) { HoodieLogBlock block = reader.next(); @@ -404,6 +404,7 @@ private MessageType readSchemaFromLogFile(Optional lastCompaction } } if (lastBlock != null) { + lastBlock.getRecords(); return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema()); } // Fall back to read the schema from last compaction diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java index f2eb5e4f1ffe6..10b8c11f80b75 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java @@ -314,9 +314,10 @@ private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLog List records = (isLogSchemaSimple ? SchemaTestUtil .generateTestRecords(0, 100) : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); - Map metadata = Maps.newHashMap(); - metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, dataFile.getCommitTime()); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema, metadata); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime()); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); logWriter.appendBlock(dataBlock); logWriter.close(); return logWriter.getLogFile(); From 4b5179180710ca9d682038388cba6a7acd17eaaa Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 28 Feb 2018 15:58:19 -0800 Subject: [PATCH 027/374] Re-factoring Compaction as first level API in WriteClient similar to upsert/insert --- .../com/uber/hoodie/HoodieWriteClient.java | 118 +++++++++++- .../hoodie/config/HoodieCompactionConfig.java | 2 +- .../hoodie/io/compact/HoodieCompactor.java | 4 +- .../compact/HoodieRealtimeTableCompactor.java | 174 ++++++++---------- .../hoodie/table/HoodieCopyOnWriteTable.java | 13 +- .../hoodie/table/HoodieMergeOnReadTable.java | 22 +-- .../com/uber/hoodie/table/HoodieTable.java | 17 +- .../uber/hoodie/io/TestHoodieCompactor.java | 34 +--- .../hoodie/table/TestMergeOnReadTable.java | 127 ++++++++++--- 9 files changed, 318 insertions(+), 193 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 81224fab767f5..42778e7dd6771 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -40,6 +40,7 @@ import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCommitException; +import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieInsertException; import com.uber.hoodie.exception.HoodieRollbackException; @@ -353,10 +354,11 @@ private JavaRDD bulkInsertInternal( return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); } - private void commitOnAutoCommit(String commitTime, JavaRDD resultRDD) { + private void commitOnAutoCommit(String commitTime, JavaRDD resultRDD, + String actionType) { if (config.shouldAutoCommit()) { logger.info("Auto commit enabled: Committing " + commitTime); - boolean commitResult = commit(commitTime, resultRDD); + boolean commitResult = commit(commitTime, resultRDD, Optional.empty(), actionType); if (!commitResult) { throw new HoodieCommitException("Failed to commit " + commitTime); } @@ -454,7 +456,7 @@ private JavaRDD updateIndexAndCommitIfNeeded(JavaRDD w JavaRDD statuses = index.updateLocation(writeStatusRDD, table); // Trigger the insert and collect statuses statuses = statuses.persist(config.getWriteStatusStorageLevel()); - commitOnAutoCommit(commitTime, statuses); + commitOnAutoCommit(commitTime, statuses, table.getCommitActionType()); return statuses; } @@ -482,6 +484,14 @@ public boolean commit(String commitTime, JavaRDD writeStatuses) { public boolean commit(String commitTime, JavaRDD writeStatuses, Optional> extraMetadata) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType()); + } + + private boolean commit(String commitTime, JavaRDD writeStatuses, + Optional> extraMetadata, String actionType) { logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible @@ -519,7 +529,6 @@ public boolean commit(String commitTime, } try { - String actionType = table.getCommitActionType(); activeTimeline.saveAsComplete( new HoodieInstant(true, actionType, commitTime), Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); @@ -917,17 +926,80 @@ public void startCommitWithTime(String commitTime) { new HoodieInstant(true, commitActionType, commitTime)); } + /** + * Provides a new commit time for a compaction (commit) operation + */ + public String startCompaction() { + String commitTime = HoodieActiveTimeline.createNewCommitTime(); + logger.info("Generate a new commit time " + commitTime); + startCompactionWithTime(commitTime); + return commitTime; + } + + /** Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, + * we need to explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction + */ + public void startCompactionWithTime(String commitTime) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + String commitActionType = HoodieTimeline.COMMIT_ACTION; + activeTimeline.createInflight( + new HoodieInstant(true, commitActionType, commitTime)); + } + /** * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed * asynchronously. Please always use this serially before or after an insert/upsert action. */ - private void compact(String compactionCommitTime) throws IOException { + public JavaRDD compact(String commitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - Optional compactionMetadata = table.compact(jsc, compactionCommitTime); - if (compactionMetadata.isPresent()) { + JavaRDD statuses = table.compact(jsc, commitTime); + // Trigger the insert and collect statuses + statuses = statuses.persist(config.getWriteStatusStorageLevel()); + String actionType = HoodieActiveTimeline.COMMIT_ACTION; + commitOnAutoCommit(commitTime, statuses, actionType); + return statuses; + } + + /** + * Commit a compaction operation + * @param commitTime + * @param writeStatuses + * @param extraMetadata + */ + public void commitCompaction(String commitTime, JavaRDD writeStatuses, + Optional> extraMetadata) { + String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION; + commit(commitTime, writeStatuses, extraMetadata, commitCompactionActionType); + } + + /** + * Commit a compaction operation + * @param commitTime + * @param writeStatuses + */ + public void commitCompaction(String commitTime, JavaRDD writeStatuses) { + String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION; + commit(commitTime, writeStatuses, Optional.empty(), commitCompactionActionType); + } + + /** + * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed + * asynchronously. Please always use this serially before or after an insert/upsert action. + */ + private void forceCompact(String compactionCommitTime) throws IOException { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTableMetaClient metaClient = + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + JavaRDD compactedStatuses = table.compact(jsc, compactionCommitTime); + if (!compactedStatuses.isEmpty()) { + commitForceCompaction(compactedStatuses, metaClient, compactionCommitTime); logger.info("Compacted successfully on commit " + compactionCommitTime); } else { logger.info("Compaction did not run for commit " + compactionCommitTime); @@ -938,12 +1010,38 @@ private void compact(String compactionCommitTime) throws IOException { * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed * asynchronously. Please always use this serially before or after an insert/upsert action. */ - public String forceCompact() throws IOException { - String compactionCommitTime = HoodieActiveTimeline.createNewCommitTime(); - compact(compactionCommitTime); + private String forceCompact() throws IOException { + String compactionCommitTime = startCompaction(); + forceCompact(compactionCommitTime); return compactionCommitTime; } + private void commitForceCompaction(JavaRDD writeStatuses, + HoodieTableMetaClient metaClient, + String compactionCommitTime) { + List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) + .collect(); + + HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); + for (HoodieWriteStat stat : updateStatusMap) { + metadata.addWriteStat(stat.getPartitionPath(), stat); + } + + logger.info("Compaction finished with result " + metadata); + + logger.info("Committing Compaction " + compactionCommitTime); + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + + try { + activeTimeline.saveAsComplete( + new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, compactionCommitTime), + Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieCompactionException( + "Failed to commit " + metaClient.getBasePath() + " at time " + compactionCommitTime, e); + } + } + public static SparkConf registerClasses(SparkConf conf) { conf.registerKryoClasses( new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index 6579ccf9ff5dd..56257fdc397d3 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -42,7 +42,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { // Turn on inline compaction - after fw delta commits a inline compaction will be run public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline"; - private static final String DEFAULT_INLINE_COMPACT = "true"; + private static final String DEFAULT_INLINE_COMPACT = "false"; // Run a compaction every N delta commits public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits"; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index fe8227f2d6864..9011a82b29df3 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -16,12 +16,14 @@ package com.uber.hoodie.io.compact; +import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import java.io.Serializable; @@ -35,7 +37,7 @@ public interface HoodieCompactor extends Serializable { /** * Compact the delta files with the data files */ - HoodieCommitMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config, + JavaRDD compact(JavaSparkContext jsc, final HoodieWriteConfig config, HoodieTable hoodieTable, String compactionCommitTime) throws Exception; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 709c0e9fec7ec..62a606efa8fd0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -16,43 +16,35 @@ package com.uber.hoodie.io.compact; +import static java.util.stream.Collectors.toList; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; -import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; -import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; -import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; -import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FlatMapFunction; - import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.StreamSupport; - -import static java.util.stream.Collectors.toList; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; /** * HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all @@ -66,73 +58,29 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { private static Logger log = LogManager.getLogger(HoodieRealtimeTableCompactor.class); @Override - public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws IOException { - Preconditions.checkArgument( - hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, - "HoodieRealtimeTableCompactor can only compact table of type " - + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() - .getTableType().name()); - - //TODO : check if maxMemory is not greater than JVM or spark.executor memory - // TODO - rollback any compactions in flight - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); - List partitionPaths = - FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), - config.shouldAssumeDatePartitioning()); - - TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); - log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - List operations = - jsc.parallelize(partitionPaths, partitionPaths.size()) - .flatMap((FlatMapFunction) partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath) - .map(s -> new CompactionOperation(s.getDataFile().get(), - partitionPath, s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed()) - .collect(Collectors.toList()), config)) - .filter(c -> !c.getDeltaFilePaths().isEmpty()) - .collect(toList()).iterator()).collect(); - log.info("Total of " + operations.size() + " compactions are retrieved"); - - // Filter the compactions with the passed in filter. This lets us choose most effective compactions only - operations = config.getCompactionStrategy().orderAndFilter(config, operations); - if (operations.isEmpty()) { - log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); - return null; - } + public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig config, + HoodieTable hoodieTable, String compactionCommitTime) throws IOException { - log.info("After filtering, Compacting " + operations + " files"); - List updateStatusMap = - jsc.parallelize(operations, operations.size()) - .map(s -> executeCompaction(hoodieTable, config, s, compactionCommitTime)) - .flatMap(writeStatList -> writeStatList.iterator()) - .collect(); - - HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); - for (HoodieWriteStat stat : updateStatusMap) { - metadata.addWriteStat(stat.getPartitionPath(), stat); + List operations = getCompactionWorkload(jsc, hoodieTable, config, + compactionCommitTime); + if (operations == null) { + return jsc.emptyRDD(); } - - log.info("Compaction finished with result " + metadata); - - //noinspection ConstantConditions - if (isCompactionSucceeded(metadata)) { - log.info("Compaction succeeded " + compactionCommitTime); - commitCompaction(compactionCommitTime, metaClient, metadata); - } else { - log.info("Compaction failed " + compactionCommitTime); - } - return metadata; + return executeCompaction(jsc, operations, hoodieTable, config, compactionCommitTime); } - private boolean isCompactionSucceeded(HoodieCommitMetadata result) { - //TODO figure out a success factor for a compaction - return true; - } + private JavaRDD executeCompaction(JavaSparkContext jsc, + List operations, + HoodieTable hoodieTable, + HoodieWriteConfig config, String compactionCommitTime) throws IOException { + log.info("After filtering, Compacting " + operations + " files"); + return jsc.parallelize(operations, operations.size()) + .map(s -> compact(hoodieTable, config, s, compactionCommitTime)) + .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); + } - private List executeCompaction(HoodieTable hoodieTable, + private List compact(HoodieTable hoodieTable, HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException { FileSystem fs = hoodieTable.getMetaClient().getFs(); @@ -153,11 +101,11 @@ private List executeCompaction(HoodieTable hoodieTable, .filterCompletedInstants().lastInstant().get().getTimestamp(); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, - metaClient.getBasePath(), - operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes(), - config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled()); + metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime, + config.getMaxMemorySizePerCompactionInBytes(), config.getCompactionLazyBlockReadEnabled(), + config.getCompactionReverseLogReadEnabled()); if (!scanner.iterator().hasNext()) { - return Lists.newArrayList(); + return Lists.newArrayList(); } // Compacting is very similar to applying updates to existing file @@ -167,30 +115,56 @@ private List executeCompaction(HoodieTable hoodieTable, Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false) .flatMap(Collection::stream) - .map(WriteStatus::getStat) .map(s -> { - s.setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate()); - s.setTotalLogFiles(scanner.getTotalLogFiles()); - s.setTotalLogRecords(scanner.getTotalLogRecords()); - s.setPartitionPath(operation.getPartitionPath()); - return s;}) + s.getStat().setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate()); + s.getStat().setTotalLogFiles(scanner.getTotalLogFiles()); + s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); + s.getStat().setPartitionPath(operation.getPartitionPath()); + return s; + }) .collect(toList()); } - public boolean commitCompaction(String commitTime, HoodieTableMetaClient metaClient, - HoodieCommitMetadata metadata) { - log.info("Committing Compaction " + commitTime); - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - - try { - activeTimeline.saveAsComplete( - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - } catch (IOException e) { - throw new HoodieCompactionException( - "Failed to commit " + metaClient.getBasePath() + " at time " + commitTime, e); + private List getCompactionWorkload(JavaSparkContext jsc, + HoodieTable hoodieTable, + HoodieWriteConfig config, String compactionCommitTime) + throws IOException { + + Preconditions.checkArgument( + hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, + "HoodieRealtimeTableCompactor can only compact table of type " + + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() + .getTableType().name()); + + //TODO : check if maxMemory is not greater than JVM or spark.executor memory + // TODO - rollback any compactions in flight + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); + List partitionPaths = + FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), + config.shouldAssumeDatePartitioning()); + + TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); + log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); + List operations = + jsc.parallelize(partitionPaths, partitionPaths.size()) + .flatMap((FlatMapFunction) partitionPath -> fileSystemView + .getLatestFileSlices(partitionPath) + .map(s -> new CompactionOperation(s.getDataFile().get(), + partitionPath, + s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed()) + .collect(Collectors.toList()), config)) + .filter(c -> !c.getDeltaFilePaths().isEmpty()) + .collect(toList()).iterator()).collect(); + log.info("Total of " + operations.size() + " compactions are retrieved"); + + // Filter the compactions with the passed in filter. This lets us choose most effective compactions only + operations = config.getCompactionStrategy().orderAndFilter(config, operations); + if (operations.isEmpty()) { + log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); + return null; } - return true; + return operations; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 5b4f176b6b483..ea80630fad118 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -35,6 +35,7 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.func.LazyInsertIterable; import com.uber.hoodie.io.HoodieCleanHelper; @@ -64,6 +65,7 @@ import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetReader; import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; @@ -414,6 +416,11 @@ public boolean isWorkloadProfileNeeded() { return true; } + @Override + public JavaRDD compact(JavaSparkContext jsc, String commitTime) { + throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); + } + public Iterator> handleUpdate(String commitTime, String fileLoc, Iterator> recordItr) @@ -513,12 +520,6 @@ public Iterator> handleInsertPartition(String commitTime, Inte return handleUpsertPartition(commitTime, partition, recordItr, partitioner); } - @Override - public Optional compact(JavaSparkContext jsc, String commitCompactionTime) { - logger.info("Nothing to compact in COW storage format"); - return Optional.empty(); - } - /** * Performs cleaning of partition paths according to cleaning policy and returns the number of * files cleaned. Handles skews in partitions to clean by making files to clean as the unit of diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 9cca2ea903b41..0787477743b23 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -41,14 +41,6 @@ import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; @@ -60,6 +52,14 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; /** * Implementation of a more real-time read-optimized Hoodie Table where @@ -167,7 +167,7 @@ public Iterator> handleUpdate(String commitTime, String fileId } @Override - public Optional compact(JavaSparkContext jsc, String compactionCommitTime) { + public JavaRDD compact(JavaSparkContext jsc, String compactionCommitTime) { logger.info("Checking if compaction needs to be run on " + config.getBasePath()); Optional lastCompaction = getActiveTimeline().getCommitTimeline() .filterCompletedInstants().lastInstant(); @@ -182,13 +182,13 @@ public Optional compact(JavaSparkContext jsc, String compa logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction + " delta commits was found since last compaction " + deltaCommitsSinceTs + ". Waiting for " + config.getInlineCompactDeltaCommitMax()); - return Optional.empty(); + return jsc.emptyRDD(); } logger.info("Compacting merge on read table " + config.getBasePath()); HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); try { - return Optional.of(compactor.compact(jsc, config, this, compactionCommitTime)); + return compactor.compact(jsc, config, this, compactionCommitTime); } catch (IOException e) { throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index c3b9d1f702298..728fc560ad031 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -20,7 +20,6 @@ import com.uber.hoodie.avro.model.HoodieSavepointMetadata; import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieRollbackStat; -import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; @@ -43,9 +42,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import scala.Tuple2; @@ -57,8 +55,6 @@ public abstract class HoodieTable implements Seri protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; - private static Logger logger = LogManager.getLogger(HoodieTable.class); - protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { this.config = config; this.metaClient = metaClient; @@ -240,7 +236,6 @@ public abstract Iterator> handleUpsertPartition(String commitT public abstract Iterator> handleInsertPartition(String commitTime, Integer partition, Iterator> recordIterator, Partitioner partitioner); - public static HoodieTable getHoodieTable( HoodieTableMetaClient metaClient, HoodieWriteConfig config) { switch (metaClient.getTableType()) { @@ -254,11 +249,10 @@ public static HoodieTable getHoodieTable( } /** - * Run Compaction on the table. Compaction arranges the data so that it is optimized for data - * access + * Run Compaction on the table. + * Compaction arranges the data so that it is optimized for data access */ - public abstract Optional compact(JavaSparkContext jsc, - String commitCompactionTime); + public abstract JavaRDD compact(JavaSparkContext jsc, String commitTime); /** * Clean partition paths according to cleaning policy and returns the number of files cleaned. @@ -279,5 +273,6 @@ public abstract List rollback(JavaSparkContext jsc, List finalizeWrite(JavaSparkContext jsc, List> writeStatuses); + public abstract Optional finalizeWrite(JavaSparkContext jsc, + List> writeStatuses); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index e05e73364d859..c1dd84034d48b 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -24,12 +24,10 @@ import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.FileSlice; -import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; -import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -125,15 +123,14 @@ public void testCompactionEmpty() throws Exception { JavaRDD recordsRDD = jsc.parallelize(records, 1); writeClient.insert(recordsRDD, newCommitTime).collect(); - HoodieCommitMetadata result = + JavaRDD result = compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); - String basePath = table.getMetaClient().getBasePath(); assertTrue("If there is nothing to compact, result will be empty", - result == null || result.getFileIdAndFullPaths(basePath).isEmpty()); + result.isEmpty()); } @Test - public void testLogFileCountsAfterCompaction() throws Exception { + public void testWriteStatusContentsAfterCompaction() throws Exception { // insert 100 records HoodieWriteConfig config = getConfig(); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); @@ -179,28 +176,15 @@ public void testLogFileCountsAfterCompaction() throws Exception { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); - HoodieCommitMetadata result = + JavaRDD result = compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); - // Verify that recently written compacted data file has no log file - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - - assertTrue("Compaction commit should be > than last insert", - HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime, - HoodieTimeline.GREATER)); - + // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = table.getRTFileSystemView() - .getLatestFileSlices(partitionPath) - .collect(Collectors.toList()); - for (FileSlice slice : groupedLogFiles) { - assertTrue( - "After compaction there should be no log files visiable on a Realtime view", - slice.getLogFiles().collect(Collectors.toList()).isEmpty()); - } - assertTrue(result.getPartitionToWriteStats().containsKey(partitionPath)); + List writeStatuses = result.collect(); + assertTrue(writeStatuses.stream() + .filter(writeStatus -> writeStatus.getStat().getPartitionPath() + .contentEquals(partitionPath)).count() > 0); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 38ee2101a0abc..3de5654d819f4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -19,6 +19,11 @@ package com.uber.hoodie.table; +import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieClientTestUtils; @@ -26,6 +31,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus; import com.uber.hoodie.common.minicluster.HdfsTestService; +import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -42,8 +48,15 @@ import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; -import com.uber.hoodie.io.compact.HoodieCompactor; -import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; +import com.uber.hoodie.index.bloom.HoodieBloomIndex; +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -57,29 +70,15 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - public class TestMergeOnReadTable { private transient JavaSparkContext jsc = null; private transient SQLContext sqlContext; private static String basePath = null; - private HoodieCompactor compactor; //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) //The implementation and gurantees of many API's differ, for example check rename(src,dst) @@ -125,7 +124,6 @@ public void init() throws IOException { HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); sqlContext = new SQLContext(jsc); // SQLContext stuff - compactor = new HoodieRealtimeTableCompactor(); } @After @@ -207,10 +205,8 @@ public void testSimpleInsertAndUpdate() throws Exception { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - HoodieCompactor compactor = new HoodieRealtimeTableCompactor(); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true)); - - compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime()); + String compactionCommitTime = client.startCompaction(); + client.compact(compactionCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), @@ -219,7 +215,7 @@ public void testSimpleInsertAndUpdate() throws Exception { assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit - table = HoodieTable.getHoodieTable( + HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false)); HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); @@ -508,12 +504,10 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieCompactor compactor = new HoodieRealtimeTableCompactor(); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, getConfig(true)); - compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime()); + String compactionCommit = client.startCompaction(); + client.compact(compactionCommit); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); @@ -630,6 +624,83 @@ public void testUpsertPartitioner() throws Exception { assertEquals("Must contain 40 records", 40, recordsRead.size()); } + @Test + @Ignore + public void testLogFileCountsAfterCompaction() throws Exception { + // insert 100 records + HoodieWriteConfig config = getConfig(true); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); + + // Update all the 100 records + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + + newCommitTime = "101"; + writeClient.startCommitWithTime(newCommitTime); + + List updatedRecords = dataGen.generateUpdates(newCommitTime, records); + JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); + HoodieIndex index = new HoodieBloomIndex<>(config, jsc); + updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); + + // Write them to corresponding avro logfiles + HoodieTestUtils + .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), + HoodieTestDataGenerator.avroSchema, updatedRecords); + + // Verify that all data file has one log file + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient, config); + for (String partitionPath : dataGen.getPartitionPaths()) { + List groupedLogFiles = + table.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toList()); + for (FileSlice fileSlice : groupedLogFiles) { + assertEquals("There should be 1 log file written for every data file", 1, + fileSlice.getLogFiles().count()); + } + } + + // Do a compaction + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient, config); + + String commitTime = writeClient.startCompaction(); + JavaRDD result = + writeClient.compact(commitTime); + + // Verify that recently written compacted data file has no log file + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient, config); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + + assertTrue("Compaction commit should be > than last insert", + HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime, + HoodieTimeline.GREATER)); + + for (String partitionPath : dataGen.getPartitionPaths()) { + List groupedLogFiles = table.getRTFileSystemView() + .getLatestFileSlices(partitionPath) + .collect(Collectors.toList()); + for (FileSlice slice : groupedLogFiles) { + assertTrue( + "After compaction there should be no log files visiable on a Realtime view", + slice.getLogFiles().collect(Collectors.toList()).isEmpty()); + } + List writeStatuses = result.collect(); + assertTrue(writeStatuses.stream() + .filter(writeStatus -> writeStatus.getStat().getPartitionPath() + .contentEquals(partitionPath)).count() > 0); + } + } private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit).build(); @@ -642,7 +713,7 @@ private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { .withAssumeDatePartitioning(true) .withCompactionConfig( HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).build()) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) .forTable("test-trip-table").withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); From 64615c4b8750bf6944a79424fda4cf0069291f9c Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 7 Mar 2018 21:04:00 -0800 Subject: [PATCH 028/374] [maven-release-plugin] prepare release hoodie-0.4.1 --- hoodie-cli/pom.xml | 6 ++---- hoodie-client/pom.xml | 6 ++---- hoodie-common/pom.xml | 6 ++---- hoodie-hadoop-mr/pom.xml | 6 ++---- hoodie-hive/pom.xml | 6 ++---- hoodie-spark/pom.xml | 6 ++---- hoodie-utilities/pom.xml | 6 ++---- pom.xml | 8 +++----- 8 files changed, 17 insertions(+), 33 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index d590153a88bf7..375fa01f52608 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -15,13 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index d082f2f60b172..2e2a0ce0e96cf 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -15,13 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 181fed895818d..eb1b15515bd6b 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -15,13 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 73ee95e5116c4..060f433e7d601 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -15,13 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index dfefe9bfbad26..0b1d150113731 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -15,13 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 5f3763220063b..d673add29a6d3 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -17,13 +17,11 @@ ~ --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index a6df1595f9c89..8b470e2dd34e1 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -15,13 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.1-SNAPSHOT + 0.4.1 4.0.0 diff --git a/pom.xml b/pom.xml index b19b801d29525..4533c96960c43 100644 --- a/pom.xml +++ b/pom.xml @@ -15,15 +15,13 @@ ~ limitations under the License. --> - + 4.0.0 com.uber.hoodie hoodie pom - 0.4.1-SNAPSHOT + 0.4.1 Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -135,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - HEAD + hoodie-0.4.1 From f5a86e8e67d04ff36eeb9efff16e08237cd1a695 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 7 Mar 2018 21:04:10 -0800 Subject: [PATCH 029/374] [maven-release-plugin] prepare for next development iteration --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- pom.xml | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 375fa01f52608..5712dd42da064 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index 2e2a0ce0e96cf..b027b357e5905 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index eb1b15515bd6b..91a9452035580 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 060f433e7d601..7bf618011ea94 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index 0b1d150113731..a4f084e91315b 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index d673add29a6d3..545ff9a59f288 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 8b470e2dd34e1..f752e0f90a47e 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.1 + 0.4.2-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index 4533c96960c43..7ed46636b6e5e 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.1 + 0.4.2-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - hoodie-0.4.1 + HEAD From aad6416bc0890212dbaf2ec08435d880385ce45d Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Thu, 8 Mar 2018 17:21:11 -0800 Subject: [PATCH 030/374] Use hadoopConf in HoodieTableMetaClient and related tests --- hoodie-client/src/test/java/HoodieClientExample.java | 2 +- .../hoodie/TestHoodieClientOnCopyOnWriteStorage.java | 4 ++-- .../src/test/java/com/uber/hoodie/TestMultiFS.java | 6 ++---- .../com/uber/hoodie/func/TestUpdateMapFunction.java | 2 +- .../java/com/uber/hoodie/index/TestHbaseIndex.java | 2 +- .../hoodie/index/bloom/TestHoodieBloomIndex.java | 2 +- .../uber/hoodie/io/TestHoodieCommitArchiveLog.java | 9 ++++++--- .../java/com/uber/hoodie/io/TestHoodieCompactor.java | 9 ++++++--- .../com/uber/hoodie/table/TestCopyOnWriteTable.java | 2 +- .../com/uber/hoodie/table/TestMergeOnReadTable.java | 6 +++--- .../hoodie/common/table/HoodieTableMetaClient.java | 11 +++++++---- .../uber/hoodie/common/model/HoodieTestUtils.java | 12 ++++++------ .../hoodie/common/table/log/HoodieLogFormatTest.java | 2 +- .../com/uber/hoodie/hadoop/InputFormatTestUtil.java | 8 ++------ .../realtime/HoodieRealtimeRecordReaderTest.java | 10 ++++++---- .../src/test/java/com/uber/hoodie/hive/TestUtil.java | 6 +++--- .../main/scala/com/uber/hoodie/DefaultSource.scala | 3 ++- .../uber/hoodie/utilities/HDFSParquetImporter.java | 2 +- .../utilities/deltastreamer/HoodieDeltaStreamer.java | 3 +-- .../hoodie/utilities/TestHoodieSnapshotCopier.java | 6 ++++-- 20 files changed, 57 insertions(+), 50 deletions(-) diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index 14c1e6eac239f..166bd22de21b9 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -85,7 +85,7 @@ public void run() throws Exception { FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration()); if (!fs.exists(path)) { HoodieTableMetaClient - .initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName, + .initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName, HoodieAvroPayload.class.getName()); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 1bf9ca506fb2d..60a6a1db2a313 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -105,7 +105,7 @@ public void init() throws IOException { folder.create(); basePath = folder.getRoot().getAbsolutePath(); fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration()); - HoodieTestUtils.init(fs, basePath); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); dataGen = new HoodieTestDataGenerator(); } @@ -1258,7 +1258,7 @@ public void testKeepLatestFileVersionsMOR() throws IOException { .retainFileVersions(1).build()).build(); HoodieTableMetaClient metaClient = HoodieTestUtils - .initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); + .initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java index 676b970d9bf9c..0dc7a816f9921 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java @@ -103,9 +103,8 @@ public void readLocalWriteHDFS() throws Exception { HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); // Initialize table and filesystem - FileSystem hdfs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration()); HoodieTableMetaClient - .initTableType(hdfs, dfsBasePath, HoodieTableType.valueOf(tableType), tableName, + .initTableType(jsc.hadoopConfiguration(), dfsBasePath, HoodieTableType.valueOf(tableType), tableName, HoodieAvroPayload.class.getName()); //Create write client to write some records in @@ -133,9 +132,8 @@ public void readLocalWriteHDFS() throws Exception { assertEquals("Should contain 100 records", readRecords.count(), records.size()); // Write to local - FileSystem local = FSUtils.getFs(tablePath, jsc.hadoopConfiguration()); HoodieTableMetaClient - .initTableType(local, tablePath, HoodieTableType.valueOf(tableType), tableName, + .initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName, HoodieAvroPayload.class.getName()); HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index 352a0036b85dd..0316372cad5cf 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -48,7 +48,7 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()), basePath); + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath); } @Test diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java index 990e62d611c44..89f31fa2e9bc5 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -112,7 +112,7 @@ public void before() throws Exception { basePath = folder.getRoot().getAbsolutePath(); // Initialize table metaClient = HoodieTableMetaClient - .initTableType(utility.getTestFileSystem(), basePath, HoodieTableType.COPY_ON_WRITE, + .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName, HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index c3ace401eff67..c0b7f4a2749e6 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -88,7 +88,7 @@ public void init() throws IOException { folder.create(); basePath = folder.getRoot().getAbsolutePath(); fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); - HoodieTestUtils.init(fs, basePath); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); // We have some records to be tagged (two different partitions) schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index 1e5d015d582ec..d171a80d997d3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -39,6 +39,7 @@ import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.Before; @@ -49,14 +50,16 @@ public class TestHoodieCommitArchiveLog { private String basePath; private FileSystem fs; + private Configuration hadoopConf; @Before public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - HoodieTestUtils.init(fs, basePath); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); + fs = FSUtils.getFs(basePath, hadoopConf); + HoodieTestUtils.init(hadoopConf, basePath); } @Test @@ -76,7 +79,7 @@ public void testArchiveDatasetWithArchival() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build()) .forTable("test-trip-table").build(); - HoodieTestUtils.init(fs, basePath); + HoodieTestUtils.init(hadoopConf, basePath); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "102"); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index c1dd84034d48b..5a1ff7ba80a0d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -43,6 +43,7 @@ import java.io.IOException; import java.util.List; import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -58,6 +59,7 @@ public class TestHoodieCompactor { private HoodieCompactor compactor; private transient HoodieTestDataGenerator dataGen = null; private transient FileSystem fs; + private Configuration hadoopConf; @Before public void init() throws IOException { @@ -68,8 +70,9 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); + fs = FSUtils.getFs(basePath, hadoopConf); + HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); dataGen = new HoodieTestDataGenerator(); compactor = new HoodieRealtimeTableCompactor(); @@ -102,7 +105,7 @@ private HoodieWriteConfig.Builder getConfigBuilder() { @Test(expected = IllegalArgumentException.class) public void testCompactionOnCopyOnWriteFail() throws Exception { - HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.COPY_ON_WRITE); + HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index 9432be72d01c2..ab9ba8be4ed16 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -76,7 +76,7 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(FSUtils.getFs(basePath, jsc.hadoopConfiguration()), basePath); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); } @Test diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 3de5654d819f4..23f201b451e30 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -121,7 +121,7 @@ public void init() throws IOException { jsc.hadoopConfiguration().addResource(dfs.getConf()); dfs.mkdirs(new Path(basePath)); - HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); sqlContext = new SQLContext(jsc); // SQLContext stuff } @@ -346,7 +346,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { public void testCOWToMORConvertedDatasetRollback() throws Exception { //Set TableType to COW - HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.COPY_ON_WRITE); + HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE); HoodieWriteConfig cfg = getConfig(true); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); @@ -385,7 +385,7 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { assertNoWriteErrors(statuses); //Set TableType to MOR - HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); //rollback a COW commit when TableType is MOR client.rollback(newCommitTime); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 4c5168f60dd60..1388823af2f51 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -193,7 +193,7 @@ public synchronized HoodieArchivedTimeline getArchivedTimeline() { /** * Helper method to initialize a given path, as a given storage type and table name */ - public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, + public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType, String tableName, String payloadClassName) throws IOException { Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName); @@ -201,7 +201,7 @@ public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath if (tableType == HoodieTableType.MERGE_ON_READ) { properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName); } - return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties); + return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); } /** @@ -210,10 +210,11 @@ public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath * * @return Instance of HoodieTableMetaClient */ - public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs, + public static HoodieTableMetaClient initializePathAsHoodieDataset(Configuration hadoopConf, String basePath, Properties props) throws IOException { log.info("Initializing " + basePath + " as hoodie dataset " + basePath); Path basePathDir = new Path(basePath); + final FileSystem fs = FSUtils.getFs(basePath, hadoopConf); if (!fs.exists(basePathDir)) { fs.mkdirs(basePathDir); } @@ -239,7 +240,9 @@ public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs, fs.mkdirs(temporaryFolder); } HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + // We should not use fs.getConf as this might be different from the original configuration + // used to create the fs in unit tests + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath); log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() + " from " + basePath); return metaClient; diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 4db0a58b78d0f..bd698e7c8ff15 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -80,11 +80,12 @@ public static Configuration getDefaultHadoopConf() { return new Configuration(); } - public static HoodieTableMetaClient init(FileSystem fs, String basePath) throws IOException { - return initTableType(fs, basePath, HoodieTableType.COPY_ON_WRITE); + public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) + throws IOException { + return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); } - public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, + public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType) throws IOException { Properties properties = new Properties(); @@ -92,7 +93,7 @@ public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName()); - return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties); + return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); } public static HoodieTableMetaClient initOnTemp() throws IOException { @@ -100,8 +101,7 @@ public static HoodieTableMetaClient initOnTemp() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); String basePath = folder.getRoot().getAbsolutePath(); - return HoodieTestUtils - .init(FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()), basePath); + return HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath); } public static String makeNewCommitTime() { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 0acd77fb79056..3ef9f23bf7f9c 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -109,7 +109,7 @@ public void setUp() throws IOException, InterruptedException { assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath()))); this.partitionPath = new Path(folder.getRoot().getPath()); this.basePath = folder.getRoot().getParent(); - HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(MiniClusterUtil.configuration, basePath, HoodieTableType.MERGE_ON_READ); } @After diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java index 4b060a132bd26..c8305e3a3f4aa 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java @@ -39,9 +39,7 @@ public class InputFormatTestUtil { public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, String commitNumber) throws IOException { basePath.create(); - HoodieTestUtils - .init(FSUtils.getFs(basePath.getRoot().toString(), HoodieTestUtils.getDefaultHadoopConf()), - basePath.getRoot().toString()); + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); for (int i = 0; i < numberOfFiles; i++) { File dataFile = @@ -101,9 +99,7 @@ public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { basePath.create(); - HoodieTestUtils - .init(FSUtils.getFs(basePath.getRoot().toString(), HoodieTestUtils.getDefaultHadoopConf()), - basePath.getRoot().toString()); + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 12c010e9860d5..653e122d305d1 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -41,6 +41,7 @@ import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; @@ -66,12 +67,13 @@ public class HoodieRealtimeRecordReaderTest { private JobConf jobConf; private FileSystem fs; + private Configuration hadoopConf; @Before public void setUp() { jobConf = new JobConf(); - fs = FSUtils - .getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf()); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); + fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf); } @Rule @@ -105,7 +107,7 @@ public void testReader() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); HoodieTestUtils - .initTableType(fs, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + .initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; File partitionDir = InputFormatTestUtil .prepareParquetDataset(basePath, schema, 1, 100, commitTime); @@ -163,7 +165,7 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema()); HoodieTestUtils - .initTableType(fs, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + .initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java index 10b8c11f80b75..ff260f893e714 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java @@ -118,7 +118,7 @@ public static void setUp() throws IOException, InterruptedException, URISyntaxEx static void clear() throws IOException { fileSystem.delete(new Path(hiveSyncConfig.basePath), true); HoodieTableMetaClient - .initTableType(fileSystem, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, + .initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, hiveSyncConfig.tableName, HoodieAvroPayload.class.getName()); HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), @@ -154,7 +154,7 @@ static void createCOWDataset(String commitTime, int numberOfPartitions) Path path = new Path(hiveSyncConfig.basePath); FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient - .initTableType(fileSystem, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, + .initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, hiveSyncConfig.tableName, HoodieAvroPayload.class.getName()); boolean result = fileSystem.mkdirs(path); checkResult(result); @@ -170,7 +170,7 @@ static void createMORDataset(String commitTime, String deltaCommitTime, int numb Path path = new Path(hiveSyncConfig.basePath); FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient - .initTableType(fileSystem, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ, + .initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ, hiveSyncConfig.tableName, HoodieAvroPayload.class.getName()); boolean result = fileSystem.mkdirs(path); diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 285ba87b1b021..dec9160e22d9e 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -190,7 +190,8 @@ class DefaultSource extends RelationProvider properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tblName.get); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, storageType); properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived"); - HoodieTableMetaClient.initializePathAsHoodieDataset(fs, path.get, properties); + HoodieTableMetaClient.initializePathAsHoodieDataset( + sqlContext.sparkContext.hadoopConfiguration, path.get, properties); } // Create a HoodieWriteClient & issue the write. diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 1d3a2b1c10eae..77184971ce061 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -232,7 +232,7 @@ protected int dataImport(JavaSparkContext jsc) throws IOException { Properties properties = new Properties(); properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType); - HoodieTableMetaClient.initializePathAsHoodieDataset(fs, cfg.targetPath, properties); + HoodieTableMetaClient.initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index 6a63c33fc84ed..eadb5cd491ff9 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -202,8 +202,7 @@ private void sync() throws Exception { Properties properties = new Properties(); properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName); HoodieTableMetaClient - .initializePathAsHoodieDataset( - FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), cfg.targetBasePath, + .initializePathAsHoodieDataset(jssc.hadoopConfiguration(), cfg.targetBasePath, properties); } log.info("Checkpoint to resume from : " + resumeCheckpointStr); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java index 326894dfeab0f..f522333393640 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.util.FSUtils; import java.io.File; import java.io.IOException; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.SparkConf; @@ -52,8 +53,9 @@ public void init() throws IOException { basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME; outputPath = rootPath + "/output"; - fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - HoodieTestUtils.init(fs, basePath); + final Configuration hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); + fs = FSUtils.getFs(basePath, hadoopConf); + HoodieTestUtils.init(hadoopConf, basePath); // Start a local Spark job SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); jsc = new JavaSparkContext(conf); From eca2e10d80880f9d1f77d9c7a4873fef81f0270d Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Tue, 6 Mar 2018 11:25:26 -0800 Subject: [PATCH 031/374] Add more options in HoodieWriteConfig --- .../java/com/uber/hoodie/config/HoodieIndexConfig.java | 9 +++++++++ .../java/com/uber/hoodie/config/HoodieStorageConfig.java | 2 +- .../java/com/uber/hoodie/config/HoodieWriteConfig.java | 8 ++++++-- .../com/uber/hoodie/index/bloom/HoodieBloomIndex.java | 2 +- .../com/uber/hoodie/io/storage/HoodieParquetConfig.java | 6 +++--- 5 files changed, 20 insertions(+), 7 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java index a84c57b11537c..9a9687a3dcb66 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java @@ -44,6 +44,8 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true"; public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching"; public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true"; + public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level"; + public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; // ***** HBase Index Configs ***** public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; @@ -143,6 +145,11 @@ public Builder hbaseIndexPutBatchSize(int putBatchSize) { return this; } + public Builder withBloomIndexInputStorageLevel(String level) { + props.setProperty(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level); + return this; + } + public HoodieIndexConfig build() { HoodieIndexConfig config = new HoodieIndexConfig(props); setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), @@ -161,6 +168,8 @@ public HoodieIndexConfig build() { HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); + setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL), + BLOOM_INDEX_INPUT_STORAGE_LEVEL, DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL); // Throws IllegalArgumentException if the value set is not a known Hoodie Index Type HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP)); return config; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java index c9ca6aa4b8d31..168174c63c1e5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java @@ -68,7 +68,7 @@ public Builder fromProperties(Properties props) { return this; } - public Builder limitFileSize(int maxFileSize) { + public Builder limitFileSize(long maxFileSize) { props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize)); return this; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index dfae2a0820291..30badf2a029d6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -275,11 +275,15 @@ public int getNumBucketsPerPartition() { return Integer.parseInt(props.getProperty(HoodieIndexConfig.BUCKETED_INDEX_NUM_BUCKETS_PROP)); } + public StorageLevel getBloomIndexInputStorageLevel() { + return StorageLevel.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL)); + } + /** * storage properties **/ - public int getParquetMaxFileSize() { - return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES)); + public long getParquetMaxFileSize() { + return Long.parseLong(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES)); } public int getParquetBlockSize() { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index bf452bc8f30f0..73a07fc4cd554 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -75,7 +75,7 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, // Step 0: cache the input record RDD if (config.getBloomIndexUseCaching()) { - recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); + recordRDD.persist(config.getBloomIndexInputStorageLevel()); } // Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java index d2af03047136a..f0aa544da6234 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java @@ -26,11 +26,11 @@ public class HoodieParquetConfig { private CompressionCodecName compressionCodecName; private int blockSize; private int pageSize; - private int maxFileSize; + private long maxFileSize; private Configuration hadoopConf; public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport, - CompressionCodecName compressionCodecName, int blockSize, int pageSize, int maxFileSize, + CompressionCodecName compressionCodecName, int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf) { this.writeSupport = writeSupport; this.compressionCodecName = compressionCodecName; @@ -56,7 +56,7 @@ public int getPageSize() { return pageSize; } - public int getMaxFileSize() { + public long getMaxFileSize() { return maxFileSize; } From b8e60974edad53e6ad1e0bbf6e8308a9d37e71cd Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 21 Jan 2018 21:09:52 -0800 Subject: [PATCH 032/374] Adding a tool to read/inspect a HoodieLogFile --- hoodie-cli/hoodie-cli.sh | 6 +- .../cli/commands/HoodieLogFileCommand.java | 221 ++++++++++++++++++ .../table/log/block/HoodieAvroDataBlock.java | 25 +- .../uber/hoodie/hive/HoodieHiveClient.java | 20 +- .../com/uber/hoodie/hive/util/SchemaUtil.java | 29 +++ 5 files changed, 275 insertions(+), 26 deletions(-) create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java diff --git a/hoodie-cli/hoodie-cli.sh b/hoodie-cli/hoodie-cli.sh index 70a54a4c753a8..73ffe333f54db 100755 --- a/hoodie-cli/hoodie-cli.sh +++ b/hoodie-cli/hoodie-cli.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash + DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" HOODIE_JAR=`ls $DIR/target/hoodie-cli-*-SNAPSHOT.jar` if [ -z "$HADOOP_CONF_DIR" ]; then @@ -9,4 +10,7 @@ if [ -z "$SPARK_CONF_DIR" ]; then echo "setting spark conf dir" SPARK_CONF_DIR="/etc/spark/conf" fi -java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HOODIE_JAR org.springframework.shell.Bootstrap +if [ -z "$CLIENT_JAR" ]; then + echo "client jar location not set" +fi +java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HOODIE_JAR:${CLIENT_JAR} org.springframework.shell.Bootstrap diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java new file mode 100644 index 0000000000000..e12ed4b462036 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -0,0 +1,221 @@ +/* + * 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.cli.commands; + +import com.beust.jcommander.internal.Maps; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; +import com.uber.hoodie.common.table.log.HoodieLogFormat; +import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.hive.util.SchemaUtil; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; +import parquet.avro.AvroSchemaConverter; +import scala.Tuple2; +import scala.Tuple3; + +@Component +public class HoodieLogFileCommand implements CommandMarker { + + @CliAvailabilityIndicator({"show logfiles"}) + public boolean isShowArchivedLogFileAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files") + public String showLogFileCommits( + @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") + final String logFilePathPattern) throws IOException { + + FileSystem fs = HoodieCLI.tableMetadata.getFs(); + List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) + .map(status -> status.getPath().toString()).collect(Collectors.toList()); + Map, Map>, Integer>>> commitCountAndMetadata = Maps + .newHashMap(); + int totalEntries = 0; + int numCorruptBlocks = 0; + + for (String logFilePath : logFilePaths) { + FileStatus[] fsStatus = fs.listStatus( + new Path(logFilePath)); + Schema writerSchema = new AvroSchemaConverter() + .convert(SchemaUtil + .readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath))); + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, + new HoodieLogFile(fsStatus[0].getPath()), writerSchema); + + // read the avro blocks + while (reader.hasNext()) { + HoodieLogBlock n = reader.next(); + String instantTime; + int recordCount = 0; + if (n instanceof HoodieCorruptBlock) { + try { + instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME); + } catch (Exception e) { + numCorruptBlocks++; + instantTime = "corrupt_block_" + numCorruptBlocks; + // could not read metadata for corrupt block + } + } else { + instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME); + if (n instanceof HoodieAvroDataBlock) { + recordCount = ((HoodieAvroDataBlock) n).getRecords().size(); + } + } + if (commitCountAndMetadata.containsKey(instantTime)) { + commitCountAndMetadata.get(instantTime) + .add(new Tuple3<>(n.getBlockType(), + new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); + totalEntries++; + } else { + List, Map>, Integer>> list + = new ArrayList<>(); + list.add(new Tuple3<>(n.getBlockType(), + new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); + commitCountAndMetadata.put(instantTime, list); + totalEntries++; + } + } + } + String[][] rows = new String[totalEntries + 1][]; + int i = 0; + ObjectMapper objectMapper = new ObjectMapper(); + for (Map.Entry, Map>, Integer>>> entry : commitCountAndMetadata + .entrySet()) { + String instantTime = entry.getKey().toString(); + for (Tuple3, Map>, Integer> tuple3 : entry + .getValue()) { + String[] output = new String[5]; + output[0] = instantTime; + output[1] = String.valueOf(tuple3._3()); + output[2] = tuple3._1().toString(); + output[3] = objectMapper.writeValueAsString(tuple3._2()._1()); + output[4] = objectMapper.writeValueAsString(tuple3._2()._2()); + rows[i] = output; + i++; + } + } + return HoodiePrintHelper.print( + new String[]{"InstantTime", "RecordCount", "BlockType", "HeaderMetadata", "FooterMetadata"}, + rows); + } + + @CliCommand(value = "show logfile records", help = "Read records from log files") + public String showLogFileRecords( + @CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") + final Integer limit, + @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified paths for the log files") + final String logFilePathPattern, + @CliOption(key = "mergeRecords", mandatory = false, help = "If the records in the log files should be merged", + unspecifiedDefaultValue = "false") + final Boolean shouldMerge) throws IOException { + + System.out + .println("===============> Showing only " + limit + " records <==============="); + + FileSystem fs = HoodieCLI.tableMetadata.getFs(); + List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) + .map(status -> status.getPath().toString()).collect(Collectors.toList()); + + // TODO : readerSchema can change across blocks/log files, fix this inside Scanner + AvroSchemaConverter converter = new AvroSchemaConverter(); + // get schema from last log file + Schema readerSchema = converter + .convert(SchemaUtil + .readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))); + + List allRecords = new ArrayList<>(); + + if (shouldMerge) { + System.out.println("===========================> MERGING RECORDS <==================="); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, + HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema, + HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get() + .getTimestamp(), + Long.valueOf(HoodieCompactionConfig.DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES), + Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), + Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED)); + for (HoodieRecord hoodieRecord : scanner) { + Optional record = hoodieRecord.getData().getInsertValue(readerSchema); + if (allRecords.size() >= limit) { + break; + } + allRecords.add(record.get()); + } + } else { + for (String logFile : logFilePaths) { + Schema writerSchema = new AvroSchemaConverter() + .convert(SchemaUtil + .readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile))); + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, + new HoodieLogFile(new Path(logFile)), writerSchema); + // read the avro blocks + while (reader.hasNext()) { + HoodieLogBlock n = reader.next(); + if (n instanceof HoodieAvroDataBlock) { + HoodieAvroDataBlock blk = (HoodieAvroDataBlock) n; + List records = blk.getRecords(); + allRecords.addAll(records); + if (allRecords.size() >= limit) { + break; + } + } + } + if (allRecords.size() >= limit) { + break; + } + } + } + String[][] rows = new String[allRecords.size() + 1][]; + int i = 0; + for (IndexedRecord record : allRecords) { + String[] data = new String[1]; + data[0] = record.toString(); + rows[i] = data; + i++; + } + return HoodiePrintHelper.print( + new String[]{"Records"}, rows); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java index 94e89e793a99f..eb2b5f45ffd25 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java @@ -21,17 +21,6 @@ import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.io.Decoder; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.io.Encoder; -import org.apache.avro.io.EncoderFactory; -import org.apache.hadoop.fs.FSDataInputStream; - -import javax.annotation.Nonnull; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -43,6 +32,16 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import javax.annotation.Nonnull; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.hadoop.fs.FSDataInputStream; /** * DataBlock contains a list of records serialized using Avro. @@ -161,6 +160,10 @@ public List getRecords() { } public Schema getSchema() { + // if getSchema was invoked before converting byte [] to records + if (records == null) { + getRecords(); + } return schema; } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 69ae7aff54c28..1ac16ad431004 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -395,21 +395,13 @@ private MessageType readSchemaFromLastCompaction(Optional lastCom @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private MessageType readSchemaFromLogFile(Optional lastCompactionCommitOpt, Path path) throws IOException { - Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); - HoodieAvroDataBlock lastBlock = null; - while (reader.hasNext()) { - HoodieLogBlock block = reader.next(); - if (block instanceof HoodieAvroDataBlock) { - lastBlock = (HoodieAvroDataBlock) block; - } - } - if (lastBlock != null) { - lastBlock.getRecords(); - return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema()); - } + MessageType messageType = SchemaUtil.readSchemaFromLogFile(fs, path); // Fall back to read the schema from last compaction - LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt); - return readSchemaFromLastCompaction(lastCompactionCommitOpt); + if (messageType == null) { + LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt); + return readSchemaFromLastCompaction(lastCompactionCommitOpt); + } + return messageType; } /** diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index 9f16c777fc8ac..9eda294d9afde 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -18,13 +18,22 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.log.HoodieLogFormat; +import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader; +import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.hive.HiveSyncConfig; import com.uber.hoodie.hive.HoodieHiveSyncException; import com.uber.hoodie.hive.SchemaDifference; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import parquet.schema.DecimalMetadata; @@ -416,4 +425,24 @@ private static String getPartitionKeyType(Map hiveSchema, String // TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that return "String"; } + + /** + * Read the schema from the log file on path + */ + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") + public static MessageType readSchemaFromLogFile(FileSystem fs, + Path path) throws IOException { + Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); + HoodieAvroDataBlock lastBlock = null; + while (reader.hasNext()) { + HoodieLogBlock block = reader.next(); + if (block instanceof HoodieAvroDataBlock) { + lastBlock = (HoodieAvroDataBlock) block; + } + } + if (lastBlock != null) { + return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema()); + } + return null; + } } From 82d9612df9cf42f632249b09f1f5b7d6fac499c9 Mon Sep 17 00:00:00 2001 From: Omkar Joshi Date: Wed, 14 Mar 2018 16:00:47 -0700 Subject: [PATCH 033/374] Spawning parallel writer thread to separate reading records from spark and writing records to parquet file --- .../uber/hoodie/config/HoodieWriteConfig.java | 12 + .../uber/hoodie/func/BufferedIterator.java | 221 ++++++++++++++++++ .../uber/hoodie/func/LazyInsertIterable.java | 88 +++++-- .../uber/hoodie/io/HoodieAppendHandle.java | 4 +- .../uber/hoodie/io/HoodieCreateHandle.java | 14 +- .../com/uber/hoodie/io/HoodieIOHandle.java | 7 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 4 +- .../hoodie/func/TestBufferedIterator.java | 207 ++++++++++++++++ 8 files changed, 529 insertions(+), 28 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 30badf2a029d6..aa0dc101d1a45 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -25,6 +25,7 @@ import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.metrics.MetricsReporterType; import org.apache.spark.storage.StorageLevel; + import javax.annotation.concurrent.Immutable; import java.io.File; import java.io.FileReader; @@ -46,6 +47,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism"; private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism"; + private static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes"; + private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4*1024*1024); private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert"; private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false"; private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert"; @@ -104,6 +107,10 @@ public int getUpsertShuffleParallelism() { return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM)); } + public int getWriteBufferLimitBytes() { + return Integer.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES)); + } + public boolean shouldCombineBeforeInsert() { return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP)); } @@ -391,6 +398,11 @@ public Builder withParallelism(int insertShuffleParallelism, int upsertShufflePa return this; } + public Builder withWriteBufferLimitBytes(int writeBufferLimit) { + props.setProperty(WRITE_BUFFER_LIMIT_BYTES, String.valueOf(writeBufferLimit)); + return this; + } + public Builder combineInput(boolean onInsert, boolean onUpsert) { props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert)); props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert)); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java new file mode 100644 index 0000000000000..08cea4dc71dbe --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java @@ -0,0 +1,221 @@ +/* + * 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.func; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.exception.HoodieException; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.util.SizeEstimator; + +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It internally samples + * every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in buffer accordingly. This is done to + * ensure that we don't OOM. + */ +public class BufferedIterator> + implements Iterator> { + + private static Logger logger = LogManager.getLogger(BufferedIterator.class); + // interval used for polling records in the queue. + public static final int RECORD_POLL_INTERVAL_SEC = 5; + // rate used for sampling records to determine avg record size in bytes. + public static final int RECORD_SAMPLING_RATE = 64; + // maximum records that will be cached + private static final int RECORD_CACHING_LIMIT = 128 * 1024; + // It indicates number of records to cache. We will be using sampled record's average size to determine how many + // records we should cache and will change (increase/decrease) permits accordingly. + @VisibleForTesting + public final Semaphore rateLimiter = new Semaphore(1); + // used for sampling records with "RECORD_SAMPLING_RATE" frequency. + public final AtomicLong samplingRecordCounter = new AtomicLong(-1); + // indicates rate limit (number of records to cache). it is updated whenever there is a change in avg record size. + @VisibleForTesting + public int currentRateLimit = 1; + // internal buffer to cache buffered records. + private final LinkedBlockingQueue>> buffer = new LinkedBlockingQueue<>(); + // maximum amount of memory to be used for buffering records. + private final long bufferMemoryLimit; + // indicates avg record size in bytes. It is updated whenever a new record is sampled. + @VisibleForTesting + public long avgRecordSizeInBytes = 0; + // indicates number of samples collected so far. + private long numSamples = 0; + // original iterator from where records are read for buffering. + private final Iterator inputIterator; + // it holds the root cause of the exception in case either buffering records (reading from inputIterator) fails or + // thread reading records from buffer fails. + private final AtomicReference hasFailed = new AtomicReference(null); + // used for indicating that all the records from buffer are read successfully. + private final AtomicBoolean isDone = new AtomicBoolean(false); + // next record to be read from buffer. + private BufferedIteratorPayload nextRecord; + // schema used for fetching insertValue from HoodieRecord. + private final Schema schema; + + public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, final Schema schema) { + this.inputIterator = iterator; + this.bufferMemoryLimit = bufferMemoryLimit; + this.schema = schema; + } + + @VisibleForTesting + public int size() { + return this.buffer.size(); + } + + // It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in bytes. It is used + // for determining how many maximum records to buffer. Based on change in avg size it may increase or decrease + // available permits. + private void adjustBufferSizeIfNeeded(final T record) throws InterruptedException { + if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { + return; + } + final long recordSizeInBytes = SizeEstimator.estimate(record); + final long newAvgRecordSizeInBytes = + Math.max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); + final int newRateLimit = + (int) Math.min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes)); +// System.out.println("recordSizeInBytes:" + recordSizeInBytes + ":newAvgRecordSizeInBytes:" + newAvgRecordSizeInBytes +// + ":newRateLimit:" + newRateLimit + ":currentRateLimit:" + currentRateLimit + ":numSamples:" + numSamples +// + ":avgRecordSizeInBytes:" + avgRecordSizeInBytes); + + // If there is any change in number of records to cache then we will either release (if it increased) or acquire + // (if it decreased) to adjust rate limiting to newly computed value. + if (newRateLimit > currentRateLimit) { + rateLimiter.release(newRateLimit - currentRateLimit); + } else if (newRateLimit < currentRateLimit) { + rateLimiter.acquire(currentRateLimit - newRateLimit); + } + currentRateLimit = newRateLimit; + avgRecordSizeInBytes = newAvgRecordSizeInBytes; + numSamples++; + } + + // inserts record into internal buffer. It also fetches insert value from the record to offload computation work on to + // buffering thread. + private void insertRecord(T t) throws Exception { + rateLimiter.acquire(); + adjustBufferSizeIfNeeded(t); + // We are retrieving insert value in the record buffering thread to offload computation around schema validation + // and record creation to it. + final BufferedIteratorPayload payload = new BufferedIteratorPayload<>(t, this.schema); + buffer.put(Optional.of(payload)); + } + + private void readNextRecord() { + rateLimiter.release(); + Optional> newRecord; + while (true) { + try { + throwExceptionIfFailed(); + newRecord = buffer.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS); + if (newRecord != null) { + break; + } + } catch (InterruptedException e) { + logger.error("error reading records from BufferedIterator", e); + throw new HoodieException(e); + } + } + if (newRecord.isPresent()) { + this.nextRecord = newRecord.get(); + } else { + // We are done reading all the records from internal iterator. + this.isDone.set(true); + this.nextRecord = null; + } + } + + public void startBuffering() throws Exception { + logger.info("starting to buffer records"); + try { + while (inputIterator.hasNext()) { + // We need to stop buffering if buffer-reader has failed and exited. + throwExceptionIfFailed(); + insertRecord(inputIterator.next()); + } + // done buffering records notifying buffer-reader. + buffer.put(Optional.empty()); + } catch (Exception e) { + logger.error("error buffering records", e); + // Used for notifying buffer-reader thread of the failed operation. + markAsFailed(e); + throw e; + } + logger.info("finished buffering records"); + } + + @Override + public boolean hasNext() { + if (this.nextRecord == null && !this.isDone.get()) { + readNextRecord(); + } + return !this.isDone.get(); + } + + @Override + public BufferedIteratorPayload next() { + Preconditions.checkState(hasNext() && this.nextRecord != null); + final BufferedIteratorPayload ret = this.nextRecord; + this.nextRecord = null; + return ret; + } + + private void throwExceptionIfFailed() { + if (this.hasFailed.get() != null) { + throw new HoodieException("operation has failed", this.hasFailed.get()); + } + } + + public void markAsFailed(Exception e) { + this.hasFailed.set(e); + // release the permits so that if the buffering thread is waiting for permits then it will get it. + this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); + } + + // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread. + static class BufferedIteratorPayload { + public T record; + public Optional insertValue; + // It caches the exception seen while fetching insert value. + public Optional exception = Optional.empty(); + + public BufferedIteratorPayload(T record, Schema schema) { + this.record = record; + try { + this.insertValue = record.getData().getInsertValue(schema); + } catch (Exception e) { + this.exception = Optional.of(e); + } + } + } +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index 8b49897e7e783..2406762a99302 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -20,15 +20,24 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; + import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Set; -import org.apache.spark.TaskContext; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -37,6 +46,7 @@ public class LazyInsertIterable extends LazyIterableIterator, List> { + private static Logger logger = LogManager.getLogger(LazyInsertIterable.class); private final HoodieWriteConfig hoodieConfig; private final String commitTime; private final HoodieTable hoodieTable; @@ -56,57 +66,95 @@ public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWrite protected void start() { } - @Override protected List computeNext() { - List statuses = new ArrayList<>(); + // Need to set current spark thread's TaskContext into newly launched thread so that new thread can access + // TaskContext properties. + final TaskContext sparkThreadTaskContext = TaskContext.get(); + // Executor service used for launching writer thread. + final ExecutorService writerService = Executors.newFixedThreadPool(1); + try { + // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. + final BufferedIterator> bufferedIterator = + new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(), + HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)); + Future> writerResult = + writerService.submit( + () -> { + logger.info("starting hoodie writer thread"); + // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext + // properties. + TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + List statuses = new LinkedList<>(); + try { + statuses.addAll(handleWrite(bufferedIterator)); + logger.info("hoodie write is done; notifying reader thread"); + return statuses; + } catch (Exception e) { + logger.error("error writing hoodie records", e); + bufferedIterator.markAsFailed(e); + throw e; + } + }); + // Buffering records into internal buffer. This can throw exception either if reading records from spark fails or + // if writing buffered records into parquet file fails. + bufferedIterator.startBuffering(); + logger.info("waiting for hoodie write to finish"); + final List result = writerResult.get(); + assert result != null && !result.isEmpty() && !bufferedIterator.hasNext(); + return result; + } catch (Exception e) { + throw new HoodieException(e); + } finally { + writerService.shutdownNow(); + } + } - while (inputItr.hasNext()) { - HoodieRecord record = inputItr.next(); + private List handleWrite(final BufferedIterator> bufferedIterator) { + List statuses = new ArrayList<>(); + while (bufferedIterator.hasNext()) { + final BufferedIterator.BufferedIteratorPayload> payload = bufferedIterator.next(); // clean up any partial failures - if (!partitionsCleaned.contains(record.getPartitionPath())) { + if (!partitionsCleaned.contains(payload.record.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with // the same data again. Thus, before we open any files under a given partition, we // first delete any files in the same partitionPath written by same Spark partition HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, - record.getPartitionPath(), + payload.record.getPartitionPath(), TaskContext.getPartitionId(), hoodieTable); - partitionsCleaned.add(record.getPartitionPath()); + partitionsCleaned.add(payload.record.getPartitionPath()); } // lazily initialize the handle, for the first time if (handle == null) { handle = - new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, - record.getPartitionPath()); + new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath()); } - if (handle.canWrite(record)) { - // write the record, if the handle has capacity - handle.write(record); + if (handle.canWrite(payload.record)) { + // write the payload, if the handle has capacity + handle.write(payload.record, payload.insertValue, payload.exception); } else { // handle is full. statuses.add(handle.close()); - // Need to handle the rejected record & open new handle + // Need to handle the rejected payload & open new handle handle = - new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, - record.getPartitionPath()); - handle.write(record); // we should be able to write 1 record. - break; + new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath()); + handle.write(payload.record, payload.insertValue, payload.exception); // we should be able to write 1 payload. } } // If we exited out, because we ran out of records, just close the pending handle. - if (!inputItr.hasNext()) { + if (!bufferedIterator.hasNext()) { if (handle != null) { statuses.add(handle.close()); } } - assert statuses.size() > 0; // should never return empty statuses + assert statuses.size() > 0 && !bufferedIterator.hasNext(); // should never return empty statuses return statuses; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 7f00961f2c28d..4e62818b16d38 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -149,8 +149,10 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { recordsDeleted++; } - hoodieRecord.deflate(); writeStatus.markSuccess(hoodieRecord, recordMetadata); + // deflate record payload after recording success. This will help users access payload as a part of marking + // record successful. + hoodieRecord.deflate(); return avroRecord; } catch (Exception e) { logger.error("Error writing record " + hoodieRecord, e); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 1fa52de256e49..9c7fac34ceead 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -93,11 +93,15 @@ public boolean canWrite(HoodieRecord record) { /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record) { + public void write(HoodieRecord record, Optional insertValue, + Optional getInsertValueException) { Optional recordMetadata = record.getData().getMetadata(); try { - Optional avroRecord = record.getData().getInsertValue(schema); - + // throws exception if there was any exception while fetching insert value + if (getInsertValueException.isPresent()) { + throw getInsertValueException.get(); + } + Optional avroRecord = insertValue; if (avroRecord.isPresent()) { storageWriter.writeAvroWithMetadata(avroRecord.get(), record); // update the new location of record, so we know where to find it next @@ -106,8 +110,10 @@ public void write(HoodieRecord record) { } else { recordsDeleted++; } - record.deflate(); status.markSuccess(record, recordMetadata); + // deflate record payload after recording success. This will help users access payload as a part of marking + // record successful. + record.deflate(); } catch (Throwable t) { // Not throwing exception from here, since we don't want to fail the entire job // for a single record diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index f207ea41a7cca..41cf3644ce640 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -50,8 +50,7 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, this.fs = hoodieTable.getMetaClient().getFs(); this.hoodieTable = hoodieTable; this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline(); - this.schema = - HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + this.schema = createHoodieWriteSchema(config); } public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) { @@ -101,4 +100,8 @@ public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, public Schema getSchema() { return schema; } + + public static Schema createHoodieWriteSchema(HoodieWriteConfig config) { + return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 5dbe39d1f4ce6..df79cb83db7e8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -167,8 +167,10 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, recordsDeleted++; } - hoodieRecord.deflate(); writeStatus.markSuccess(hoodieRecord, recordMetadata); + // deflate record payload after recording success. This will help users access payload as a part of marking + // record successful. + hoodieRecord.deflate(); return true; } catch (Exception e) { logger.error("Error writing record " + hoodieRecord, e); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java new file mode 100644 index 0000000000000..35e36ef1dd5dd --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java @@ -0,0 +1,207 @@ +/* + * 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.func; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.exception.HoodieException; +import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.io.FileUtils; +import org.apache.spark.util.SizeEstimator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestBufferedIterator { + + private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); + private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); + private ExecutorService recordReader = null; + + @Before + public void beforeTest() { + this.recordReader = Executors.newFixedThreadPool(1); + } + + @After + public void afterTest() { + if (this.recordReader != null) { + this.recordReader.shutdownNow(); + this.recordReader = null; + } + } + + // Test to ensure that we are reading all records from buffered iterator in the same order without any exceptions. + @Test(timeout = 60000) + public void testRecordReading() throws IOException, ExecutionException, InterruptedException { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final BufferedIterator bufferedIterator = + new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, HoodieTestDataGenerator.avroSchema); + Future result = + recordReader.submit( + () -> { + bufferedIterator.startBuffering(); + return true; + } + ); + final Iterator originalRecordIterator = hoodieRecords.iterator(); + int recordsRead = 0; + while (bufferedIterator.hasNext()) { + final HoodieRecord originalRecord = originalRecordIterator.next(); + final Optional originalInsertValue = + originalRecord.getData().getInsertValue(HoodieTestDataGenerator.avroSchema); + final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next(); + // Ensure that record ordering is guaranteed. + Assert.assertEquals(originalRecord, payload.record); + // cached insert value matches the expected insert value. + Assert.assertEquals(originalInsertValue, payload.insertValue); + recordsRead++; + } + Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); + // all the records should be read successfully. + Assert.assertEquals(numRecords, recordsRead); + // should not throw any exceptions. + Assert.assertTrue(result.get()); + } + + // Test to ensure that record buffering is throttled when we hit memory limit. + @Test(timeout = 60000) + public void testMemoryLimitForBuffering() throws IOException, InterruptedException { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // maximum number of records to keep in memory. + final int recordLimit = 5; + final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); + final BufferedIterator bufferedIterator = + new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); + Future result = + recordReader.submit( + () -> { + bufferedIterator.startBuffering(); + return true; + } + ); + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator.rateLimiter)) { + Thread.sleep(10); + } + Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); + Assert.assertEquals(recordLimit, bufferedIterator.size()); + Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); + + // try to read 2 records. + Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record); + Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record); + + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator.rateLimiter)) { + Thread.sleep(10); + } + // No change is expected in rate limit or number of buffered records. We only expect buffering thread to read + // 2 more records into the buffer. + Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); + Assert.assertEquals(recordLimit, bufferedIterator.size()); + Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); + } + + // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread is propagated to + // another thread. + @Test(timeout = 60000) + public void testException() throws IOException, InterruptedException { + final int numRecords = 256; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // buffer memory limit + final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); + + // first let us throw exception from bufferIterator reader and test that buffering thread stops and throws + // correct exception back. + BufferedIterator bufferedIterator1 = + new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); + Future result = + recordReader.submit( + () -> { + bufferedIterator1.startBuffering(); + return true; + } + ); + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator1.rateLimiter)) { + Thread.sleep(10); + } + // notify buffering thread of an exception and ensure that it exits. + final Exception e = new Exception("Failing it :)"); + bufferedIterator1.markAsFailed(e); + try { + result.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e1) { + Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); + Assert.assertEquals(e, e1.getCause().getCause()); + } + + // second let us raise an exception while doing record buffering. this exception should get propagated to + // buffered iterator reader. + final RuntimeException expectedException = new RuntimeException("failing record reading"); + final Iterator mockHoodieRecordsIterator = mock(Iterator.class); + when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); + when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); + BufferedIterator bufferedIterator2 = + new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); + Future result2 = + recordReader.submit( + () -> { + bufferedIterator2.startBuffering(); + return true; + } + ); + try { + bufferedIterator2.hasNext(); + Assert.fail("exception is expected"); + } catch (Exception e1) { + Assert.assertEquals(expectedException, e1.getCause()); + } + // buffering thread should also have exited. make sure that it is not running. + try { + result2.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e2) { + Assert.assertEquals(expectedException, e2.getCause()); + } + } + + private boolean isQueueFull(Semaphore rateLimiter) { + return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); + } +} From 15a6b936af202497d8fd36f4c19567376f519e04 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Wed, 14 Mar 2018 12:53:12 -0700 Subject: [PATCH 034/374] Add back UseTempFolder changes in HoodieMergeHandle --- .../java/com/uber/hoodie/io/HoodieMergeHandle.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index df79cb83db7e8..ce5f1e9e42a4c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -109,6 +109,10 @@ private void init(String fileId, String partitionPath) { String relativePath = new Path(partitionPath + "/" + FSUtils .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); newFilePath = new Path(config.getBasePath(), relativePath); + if (config.shouldUseTempFolderForCopyOnWriteForMerge()) { + this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), + fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); + } // handle cases of partial failures, for update task if (fs.exists(newFilePath)) { @@ -116,15 +120,15 @@ private void init(String fileId, String partitionPath) { } logger.info(String.format("Merging new data into oldPath %s, as newPath %s", - oldFilePath.toString(), newFilePath.toString())); + oldFilePath.toString(), getStorageWriterPath().toString())); // file name is same for all records, in this bunch writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); writeStatus.getStat().setFileId(fileId); - writeStatus.getStat().setPath(relativePath); + writeStatus.getStat().setPaths(new Path(config.getBasePath()), newFilePath, tempPath); // Create the writer for writing the new version file storageWriter = HoodieStorageWriterFactory - .getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema); + .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); } catch (IOException io) { logger.error("Error in update task at commit " + commitTime, io); writeStatus.setGlobalError(io); From b83b8e459eaf77675f9b6798985ca5b1dcc8380e Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 13 Mar 2018 22:56:29 -0700 Subject: [PATCH 035/374] - Fixing memory leak due to HoodieLogFileReader holding on to a logblock - Removed inMemory HashMap usage in merge(..) code in LogScanner --- .../log/HoodieCompactedLogRecordScanner.java | 42 +++++-------------- .../common/table/log/HoodieLogFileReader.java | 16 +++---- .../table/log/HoodieLogFormatReader.java | 8 +++- .../common/table/log/HoodieLogFormatTest.java | 2 + 4 files changed, 27 insertions(+), 41 deletions(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 98dd29b7872ac..c8fe0e9e3c3d4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -240,14 +240,12 @@ private boolean isNewInstantBlock(HoodieLogBlock logBlock) { /** * Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge - * with the application specific payload if the same key was found before Sufficient to just merge - * the log records since the base data is merged on previous compaction + * with the application specific payload if the same key was found before. Sufficient to just merge + * the log records since the base data is merged on previous compaction. + * Finally, merge this log block with the accumulated records */ - private Map> loadRecordsFromBlock( + private Map> merge( HoodieAvroDataBlock dataBlock) throws IOException { - // TODO (NA) - Instead of creating a new HashMap use the spillable map - Map> recordsFromLastBlock = Maps - .newHashMap(); // TODO (NA) - Implemnt getRecordItr() in HoodieAvroDataBlock and use that here List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); @@ -256,19 +254,19 @@ private Map> loadRecordsFrom .toString(); HoodieRecord hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); - if (recordsFromLastBlock.containsKey(key)) { + if (records.containsKey(key)) { // Merge and store the merged record - HoodieRecordPayload combinedValue = recordsFromLastBlock.get(key).getData() + HoodieRecordPayload combinedValue = records.get(key).getData() .preCombine(hoodieRecord.getData()); - recordsFromLastBlock + records .put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); } else { // Put the record as is - recordsFromLastBlock.put(key, hoodieRecord); + records.put(key, hoodieRecord); } }); - return recordsFromLastBlock; + return records; } /** @@ -277,11 +275,12 @@ private Map> loadRecordsFrom private void merge(Map> records, Deque lastBlocks) throws IOException { while (!lastBlocks.isEmpty()) { + log.info("Number of remaining logblocks to merge " + lastBlocks.size()); // poll the element at the bottom of the stack since that's the order it was inserted HoodieLogBlock lastBlock = lastBlocks.pollLast(); switch (lastBlock.getBlockType()) { case AVRO_DATA_BLOCK: - merge(records, loadRecordsFromBlock((HoodieAvroDataBlock) lastBlock)); + merge((HoodieAvroDataBlock) lastBlock); break; case DELETE_BLOCK: // TODO : If delete is the only block written and/or records are present in parquet file @@ -295,25 +294,6 @@ private void merge(Map> reco } } - /** - * Merge the records read from a single data block with the accumulated records - */ - private void merge(Map> records, - Map> recordsFromLastBlock) { - recordsFromLastBlock.forEach((key, hoodieRecord) -> { - if (records.containsKey(key)) { - // Merge and store the merged record - HoodieRecordPayload combinedValue = records.get(key).getData() - .preCombine(hoodieRecord.getData()); - records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), - combinedValue)); - } else { - // Put the record as is - records.put(key, hoodieRecord); - } - }); - } - @Override public Iterator> iterator() { return records.iterator(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java index f9a01c8c85273..1d4cda06121f5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -56,7 +56,6 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { private static final byte[] oldMagicBuffer = new byte[4]; private static final byte[] magicBuffer = new byte[6]; private final Schema readerSchema; - private HoodieLogBlock nextBlock = null; private LogFormatVersion nextBlockVersion; private boolean readBlockLazily; private long reverseLogFilePosition; @@ -271,8 +270,8 @@ public boolean hasNext() { if (isEOF) { return false; } - this.nextBlock = readBlock(); - return nextBlock != null; + // If not hasNext(), we either we reach EOF or throw an exception on invalid magic header + return true; } catch (IOException e) { throw new HoodieIOException("IOException when reading logfile " + logFile, e); } @@ -322,11 +321,12 @@ private boolean readMagic() throws IOException { @Override public HoodieLogBlock next() { - if (nextBlock == null) { - // may be hasNext is not called - hasNext(); + try { + // hasNext() must be called before next() + return readBlock(); + } catch(IOException io) { + throw new HoodieIOException("IOException when reading logblock from log file " + logFile, io); } - return nextBlock; } /** @@ -378,7 +378,7 @@ public HoodieLogBlock prev() throws IOException { boolean hasNext = hasNext(); reverseLogFilePosition -= blockSize; lastReverseLogFilePosition = reverseLogFilePosition; - return this.nextBlock; + return next(); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index f62c9f0e504f2..e5f8f5c99be65 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -24,6 +24,8 @@ import java.io.IOException; import java.util.List; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; public class HoodieLogFormatReader implements HoodieLogFormat.Reader { @@ -34,6 +36,8 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final boolean readBlocksLazily; private final boolean reverseLogReader; + private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class); + HoodieLogFormatReader(FileSystem fs, List logFiles, Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { this.logFiles = logFiles; @@ -77,6 +81,7 @@ else if (logFiles.size() > 0) { } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); } + log.info("Moving to the next reader for logfile " + currentReader.getLogFile()); return this.currentReader.hasNext(); } return false; @@ -84,8 +89,7 @@ else if (logFiles.size() > 0) { @Override public HoodieLogBlock next() { - HoodieLogBlock block = currentReader.next(); - return block; + return currentReader.next(); } @Override diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 3ef9f23bf7f9c..209a53cb40ab5 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -374,6 +374,7 @@ public void testBasicAppendAndRead() assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); + reader.hasNext(); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", @@ -381,6 +382,7 @@ public void testBasicAppendAndRead() assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, dataBlockRead.getRecords()); + reader.hasNext(); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", From 474a1c71cdadb881c351951dd37e76734312ff1a Mon Sep 17 00:00:00 2001 From: Kaushik Devarajaiah Date: Mon, 12 Mar 2018 19:06:52 -0700 Subject: [PATCH 036/374] DeduplicateRecords based on recordKey if global index is used --- .../com/uber/hoodie/HoodieWriteClient.java | 19 ++++++++-- .../TestHoodieClientOnCopyOnWriteStorage.java | 35 ++++++++++++++++++- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 42778e7dd6771..bf81a1faa4640 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -17,6 +17,7 @@ package com.uber.hoodie; import com.codahale.metrics.Timer; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.uber.hoodie.avro.model.HoodieCleanMetadata; @@ -111,10 +112,16 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) */ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) { + this(jsc, clientConfig, rollbackInFlight, HoodieIndex.createIndex(clientConfig, jsc)); + } + + @VisibleForTesting + HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, + boolean rollbackInFlight, HoodieIndex index) { this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration()); this.jsc = jsc; this.config = clientConfig; - this.index = HoodieIndex.createIndex(config, jsc); + this.index = index; this.metrics = new HoodieMetrics(config, config.getTableName()); if (rollbackInFlight) { @@ -1051,10 +1058,16 @@ public static SparkConf registerClasses(SparkConf conf) { /** * Deduplicate Hoodie records, using the given deduplication funciton. */ - private JavaRDD> deduplicateRecords(JavaRDD> records, + JavaRDD> deduplicateRecords(JavaRDD> records, int parallelism) { + boolean isIndexingGlobal = index.isGlobal(); return records - .mapToPair(record -> new Tuple2<>(record.getKey(), record)) + .mapToPair(record -> { + HoodieKey hoodieKey = record.getKey(); + // If index used is global, then records are expected to differ in their partitionPath + Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; + return new Tuple2<>(key, record); + }) .reduceByKey((rec1, rec2) -> { @SuppressWarnings("unchecked") T reducedData = (T) rec1.getData().preCombine(rec2.getData()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 60a6a1db2a313..c5544760d8625 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -20,6 +20,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.collect.Iterables; import com.uber.hoodie.common.HoodieCleanStat; @@ -48,13 +50,13 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.index.HoodieIndex; -import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -65,6 +67,7 @@ import java.util.Optional; import java.util.Set; import java.util.TreeSet; +import java.util.UUID; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.commons.io.IOUtils; @@ -218,6 +221,36 @@ public void testUpserts() throws Exception { testUpsertsInternal(getConfig()); } + @Test + public void testDeduplication() throws Exception { + String newCommitTime = "001"; + + String recordKey = UUID.randomUUID().toString(); + HoodieKey keyOne = new HoodieKey(recordKey, "2018-01-01"); + HoodieRecord recordOne = new HoodieRecord(keyOne, + HoodieTestDataGenerator.generateRandomValue(keyOne, newCommitTime)); + + HoodieKey keyTwo = new HoodieKey(recordKey, "2018-02-01"); + HoodieRecord recordTwo = new HoodieRecord(keyTwo, + HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime)); + + JavaRDD records = jsc.parallelize(Arrays.asList(recordOne, recordTwo), 1); + + // dedup should be done based on recordKey only + HoodieWriteClient clientWithDummyGlobalIndex = getWriteClientWithDummyIndex(true); + assertEquals(1, clientWithDummyGlobalIndex.deduplicateRecords(records, 1).collect().size()); + + // dedup should be done based on both recordKey and partitionPath + HoodieWriteClient clientWithDummyNonGlobalIndex = getWriteClientWithDummyIndex(false); + assertEquals(2, clientWithDummyNonGlobalIndex.deduplicateRecords(records, 1).collect().size()); + } + + private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception { + HoodieIndex index = mock(HoodieIndex.class); + when(index.isGlobal()).thenReturn(isGlobal); + return new HoodieWriteClient(jsc, getConfigBuilder().build(), false, index); + } + private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exception { HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig); From 947ba915fe0fa83e96bc0a343fa0a62d04d7b215 Mon Sep 17 00:00:00 2001 From: Jian Xu Date: Tue, 20 Mar 2018 12:06:15 -0700 Subject: [PATCH 037/374] Checking storage level before persisting preppedRecords --- .../src/main/java/com/uber/hoodie/HoodieWriteClient.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index bf81a1faa4640..bb5613f2a61b9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -421,7 +421,12 @@ private JavaRDD upsertRecordsInternal(JavaRDD> prep final boolean isUpsert) { // Cache the tagged records, so we don't end up computing both - preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER()); + // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling + if (preppedRecords.getStorageLevel() == StorageLevel.NONE()) { + preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER()); + } else { + logger.info("RDD PreppedRecords was persisted at: " + preppedRecords.getStorageLevel()); + } WorkloadProfile profile = null; if (hoodieTable.isWorkloadProfileNeeded()) { From 4ffa5c85bd2b1a8ded2cafb353a6624fbeec2fd4 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 23 Mar 2018 21:50:11 -0700 Subject: [PATCH 038/374] Adding config for parquet compression ratio --- .../main/scala/com/uber/hoodie/cli/SparkHelpers.scala | 2 +- .../com/uber/hoodie/config/HoodieStorageConfig.java | 10 ++++++++++ .../java/com/uber/hoodie/config/HoodieWriteConfig.java | 4 ++++ .../uber/hoodie/io/storage/HoodieParquetConfig.java | 8 +++++++- .../uber/hoodie/io/storage/HoodieParquetWriter.java | 4 +--- .../hoodie/io/storage/HoodieStorageWriterFactory.java | 3 ++- .../uber/hoodie/index/bloom/TestHoodieBloomIndex.java | 4 +++- 7 files changed, 28 insertions(+), 7 deletions(-) diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala index 8e1eccfeedb46..f383e5fba4fae 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala @@ -42,7 +42,7 @@ object SparkHelpers { val schema: Schema = sourceRecords.get(0).getSchema val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble) val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter) - val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf) + val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble) val writer = new HoodieParquetWriter[HoodieJsonPayload, IndexedRecord](commitTime, destinationFile, parquetConfig, schema) for (rec <- sourceRecords) { val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java index 168174c63c1e5..0eb0a5c0eb45f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java @@ -40,6 +40,9 @@ public class HoodieStorageConfig extends DefaultHoodieConfig { // used to size data blocks in log file public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size"; public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256*1024*1024); // 256 MB + public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio"; + // Default compression ratio for parquet + public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1); private HoodieStorageConfig(Properties props) { super(props); @@ -93,6 +96,11 @@ public Builder logFileMaxSize(int logFileSize) { return this; } + public Builder parquetCompressionRatio(double parquetCompressionRatio) { + props.setProperty(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio)); + return this; + } + public HoodieStorageConfig build() { HoodieStorageConfig config = new HoodieStorageConfig(props); setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), @@ -105,6 +113,8 @@ public HoodieStorageConfig build() { LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES); setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES), LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES); + setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO), + PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index aa0dc101d1a45..d0911dca1ec22 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -309,6 +309,10 @@ public int getLogFileMaxSize() { return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES)); } + public double getParquetCompressionRatio() { + return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO)); + } + /** * metrics properties diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java index f0aa544da6234..985e2cc280f61 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetConfig.java @@ -28,16 +28,18 @@ public class HoodieParquetConfig { private int pageSize; private long maxFileSize; private Configuration hadoopConf; + private double compressionRatio; public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize, long maxFileSize, - Configuration hadoopConf) { + Configuration hadoopConf, double compressionRatio) { this.writeSupport = writeSupport; this.compressionCodecName = compressionCodecName; this.blockSize = blockSize; this.pageSize = pageSize; this.maxFileSize = maxFileSize; this.hadoopConf = hadoopConf; + this.compressionRatio = compressionRatio; } public HoodieAvroWriteSupport getWriteSupport() { @@ -63,4 +65,8 @@ public long getMaxFileSize() { public Configuration getHadoopConf() { return hadoopConf; } + + public double getCompressionRatio() { + return compressionRatio; + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java index 48bdfda59e6c7..9978c074737b3 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java @@ -39,10 +39,8 @@ public class HoodieParquetWriter extends ParquetWriter implements HoodieStorageWriter { - private static double STREAM_COMPRESSION_RATIO = 0.1; private static AtomicLong recordIndex = new AtomicLong(1); - private final Path file; private final HoodieWrapperFileSystem fs; private final long maxFileSize; @@ -75,7 +73,7 @@ public HoodieParquetWriter(String commitTime, Path file, // We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10% // TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS this.maxFileSize = parquetConfig.getMaxFileSize() + Math - .round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO); + .round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio()); this.writeSupport = parquetConfig.getWriteSupport(); this.commitTime = commitTime; this.schema = schema; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java index 17842b888e215..0bf07482f8ec6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java @@ -50,7 +50,8 @@ private static HoodieSt HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, config.getParquetBlockSize(), config.getParquetPageSize(), - config.getParquetMaxFileSize(), hoodieTable.getHadoopConf()); + config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(), + config.getParquetCompressionRatio()); return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index c0b7f4a2749e6..4bf91001f3b24 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -37,6 +37,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.storage.HoodieParquetConfig; import com.uber.hoodie.io.storage.HoodieParquetWriter; @@ -513,7 +514,8 @@ private String writeParquetFile(String partitionPath, String filename, List Date: Tue, 27 Mar 2018 11:36:01 -0400 Subject: [PATCH 039/374] Replace deprecated jackson version --- hoodie-common/pom.xml | 9 ------- .../hoodie/avro/MercifulJsonConverter.java | 2 +- .../uber/hoodie/common/HoodieJsonPayload.java | 6 ++--- .../common/model/HoodieCommitMetadata.java | 24 ++++++++++--------- .../uber/hoodie/common/util/TestRecord.java | 8 +++---- hoodie-spark/pom.xml | 4 ---- 6 files changed, 21 insertions(+), 32 deletions(-) diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 91a9452035580..0d29828ab0dbd 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -89,10 +89,6 @@ com.fasterxml.jackson.core jackson-annotations - - org.codehaus.jackson - jackson-mapper-asl - org.apache.parquet parquet-avro @@ -114,11 +110,6 @@ hadoop-common tests - - org.codehaus.jackson - jackson-core-asl - 1.9.13 - org.apache.commons commons-lang3 diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java index b39d641746f8b..827da3093531b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java @@ -24,7 +24,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.codehaus.jackson.map.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectMapper; /** * Marjority of this is copied from https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java index c8d89d96bf037..95b2b399250c7 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java @@ -16,6 +16,8 @@ package com.uber.hoodie.common; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.uber.hoodie.avro.MercifulJsonConverter; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.exception.HoodieException; @@ -30,8 +32,6 @@ import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.commons.io.IOUtils; -import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.map.ObjectMapper; public class HoodieJsonPayload implements HoodieRecordPayload { @@ -98,7 +98,7 @@ private String getFieldFromJsonOrFail(String field) throws IOException { if (!node.has(field)) { throw new HoodieException("Field :" + field + " not found in payload => " + node.toString()); } - return node.get(field).getTextValue(); + return node.get(field).textValue(); } public String getRowKey(String keyColumnField) throws IOException { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index 8a49c5c39cfc4..21c1816f5bccc 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -16,7 +16,11 @@ package com.uber.hoodie.common.model; +import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.Serializable; import java.nio.charset.Charset; @@ -27,10 +31,6 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.codehaus.jackson.annotate.JsonAutoDetect; -import org.codehaus.jackson.annotate.JsonMethod; -import org.codehaus.jackson.map.DeserializationConfig.Feature; -import org.codehaus.jackson.map.ObjectMapper; /** * All the metadata that gets stored along with a commit. @@ -116,9 +116,7 @@ public String toJsonString() throws IOException { log.info("partition path is null for " + partitionToWriteStats.get(null)); partitionToWriteStats.remove(null); } - ObjectMapper mapper = new ObjectMapper(); - mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); - return mapper.defaultPrettyPrintingWriter().writeValueAsString(this); + return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); } public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOException { @@ -126,10 +124,7 @@ public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOExcep // For empty commit file (no data or somethings bad happen). return new HoodieCommitMetadata(); } - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); - return mapper.readValue(jsonStr, HoodieCommitMetadata.class); + return getObjectMapper().readValue(jsonStr, HoodieCommitMetadata.class); } // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. @@ -235,4 +230,11 @@ public int hashCode() { public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException { return fromJsonString(new String(bytes, Charset.forName("utf-8"))); } + + private static ObjectMapper getObjectMapper() { + ObjectMapper mapper = new ObjectMapper(); + mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); + return mapper; + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestRecord.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestRecord.java index 5df7f3252c340..ecbc94c856a3a 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestRecord.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestRecord.java @@ -16,14 +16,14 @@ package com.uber.hoodie.common.util; +import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.codehaus.jackson.annotate.JsonAutoDetect; -import org.codehaus.jackson.annotate.JsonMethod; -import org.codehaus.jackson.map.ObjectMapper; @JsonIgnoreProperties(ignoreUnknown = true) @SuppressWarnings({"unused", "FieldCanBeLocal", "MismatchedQueryAndUpdateOfCollection"}) @@ -97,7 +97,7 @@ public TestRecord(String commitTime, int recordNumber, String fileId) { public String toJsonString() throws IOException { ObjectMapper mapper = new ObjectMapper(); - mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); + mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(this); } } diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 545ff9a59f288..3e8878ea94549 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -148,10 +148,6 @@ com.fasterxml.jackson.core jackson-annotations - - org.codehaus.jackson - jackson-mapper-asl - org.apache.hadoop From 58ffd63fbaf2f53ef732df806f2afb4eac213de4 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Thu, 15 Mar 2018 00:20:16 -0700 Subject: [PATCH 040/374] Making ExternalSpillableMap generic for any datatype - Introduced concept of converters to be able to serde generic datatype for SpillableMap - Fixed/Added configs to Hoodie Configs - Changed HoodieMergeHandle to start using SpillableMap --- docs/configurations.md | 11 +- .../cli/commands/HoodieLogFileCommand.java | 3 +- .../hoodie/config/HoodieCompactionConfig.java | 20 +-- .../hoodie/config/HoodieMemoryConfig.java | 147 +++++++++++++++++ .../uber/hoodie/config/HoodieWriteConfig.java | 58 +++++-- .../com/uber/hoodie/io/HoodieIOHandle.java | 7 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 35 ++-- .../compact/HoodieRealtimeTableCompactor.java | 5 +- hoodie-common/pom.xml | 5 + .../log/HoodieCompactedLogRecordScanner.java | 9 +- .../hoodie/common/util/SpillableMapUtils.java | 38 ++--- .../common/util/collection/DiskBasedMap.java | 99 ++++++------ .../util/collection/ExternalSpillableMap.java | 126 ++++++++------- .../util/collection/LazyFileIterable.java | 54 +++---- .../util/collection/converter/Converter.java | 39 +++++ .../converter/HoodieRecordConverter.java | 107 +++++++++++++ .../collection/converter/StringConverter.java | 41 +++++ .../HoodieNotSerializableException.java | 37 +++++ .../common/model/AvroBinaryTestPayload.java | 58 +++++++ .../hoodie/common/util/SchemaTestUtil.java | 54 ++++++- .../common/util/SpillableMapTestUtils.java | 4 +- .../util/collection/TestDiskBasedMap.java | 149 +++++++++++++++--- .../collection/TestExternalSpillableMap.java | 149 +++++++++++++----- .../OverwriteWithLatestAvroPayload.java | 4 + 24 files changed, 981 insertions(+), 278 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java diff --git a/docs/configurations.md b/docs/configurations.md index 2ff6077075ba8..bd4f4a27a2824 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -23,7 +23,7 @@ summary: "Here we list all possible configurations and what they mean" Should HoodieWriteClient autoCommit after insert and upsert. The client can choose to turn off auto-commit and commit on a "defined success condition" - [withAssumeDatePartitioning](#withAssumeDatePartitioning) (assumeDatePartitioning = false)
Should HoodieWriteClient assume the data is partitioned by dates, i.e three levels from base path. This is a stop-gap to support tables created by versions < 0.3.1. Will be removed eventually - + - [withIndexConfig](#withIndexConfig) (HoodieIndexConfig)
Hoodie uses a index to help find the FileID which contains an incoming record key. This is pluggable to have a external index (HBase) or use the default bloom filter stored in the Parquet files - [withIndexType](#withIndexType) (indexType = BLOOM)
@@ -80,8 +80,6 @@ summary: "Here we list all possible configurations and what they mean" Number of max delta commits to keep before triggering an inline compaction - [withCompactionReverseLogReadEnabled](#withCompactionReverseLogReadEnabled) (false)
HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. If this config is set to true, the Reader reads the logfile in reverse direction, from pos=file_length to pos=0 - - [withMaxMemorySizePerCompactionInBytes](#withMaxMemorySizePerCompactionInBytes) (maxMemorySizePerCompactionInBytes = 1GB)
- HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map. - [withMetricsConfig](#withMetricsConfig) (HoodieMetricsConfig)
Hoodie publishes metrics on every commit, clean, rollback etc. @@ -96,6 +94,13 @@ summary: "Here we list all possible configurations and what they mean" - [usePrefix](#usePrefix) ()
Standard prefix for all metrics + - [withMemoryConfig](#withMemoryConfig) (HoodieMemoryConfig)
+ Memory related configs + - [withMaxMemoryFractionPerPartitionMerge](#withMaxMemoryFractionPerPartitionMerge) (maxMemoryFractionPerPartitionMerge = 0.6)
+ This fraction is multiplied with the user memory fraction (1 - spark.memory.fraction) to get a final fraction of heap space to use during merge + - [withMaxMemorySizePerCompactionInBytes](#withMaxMemorySizePerCompactionInBytes) (maxMemorySizePerCompactionInBytes = 1GB)
+ HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map. + - [S3Configs](s3_hoodie.html) (Hoodie S3 Configs)
Configurations required for S3 and Hoodie co-operability. diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index e12ed4b462036..2de48500b6a00 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -31,6 +31,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieMemoryConfig; import com.uber.hoodie.hive.util.SchemaUtil; import java.io.IOException; import java.util.ArrayList; @@ -173,7 +174,7 @@ public String showLogFileRecords( HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema, HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get() .getTimestamp(), - Long.valueOf(HoodieCompactionConfig.DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES), + Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES), Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED)); for (HoodieRecord hoodieRecord : scanner) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index 56257fdc397d3..dbb47e6bd83b4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -99,11 +99,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { // used to merge records written to log file public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName(); - public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class"; - - public static final String MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP = "hoodie.compaction.spill.threshold"; - // Default memory size per compaction, excess spills to disk - public static final String DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES = String.valueOf(1024*1024*1024L); //1GB + public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class"; // used to choose a trade off between IO vs Memory when performing compaction process // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory @@ -212,7 +208,7 @@ public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) { } public Builder withPayloadClass(String payloadClassName) { - props.setProperty(PAYLOAD_CLASS, payloadClassName); + props.setProperty(PAYLOAD_CLASS_PROP, payloadClassName); return this; } @@ -222,12 +218,6 @@ public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) { return this; } - public Builder withMaxMemorySizePerCompactionInBytes(long maxMemorySizePerCompactionInBytes) { - props.setProperty(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, - String.valueOf(maxMemorySizePerCompactionInBytes)); - return this; - } - public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) { props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction)); @@ -277,12 +267,10 @@ public HoodieCompactionConfig build() { CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP), COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY); - setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS), - PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS); + setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP), + PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS); setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP), TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB); - setDefaultOnCondition(props, !props.containsKey(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP), - MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES); setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP), COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED); setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java new file mode 100644 index 0000000000000..65f64e41b3411 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java @@ -0,0 +1,147 @@ +/* + * 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.config; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; +import javax.annotation.concurrent.Immutable; +import org.apache.spark.SparkEnv; +import org.apache.spark.util.Utils; + +/** + * Memory related config + */ +@Immutable +public class HoodieMemoryConfig extends DefaultHoodieConfig { + + // This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use during merge + // This makes it easier to scale this value as one increases the spark.executor.memory + public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction"; + // Default max memory fraction during hash-merge, excess spills to disk + public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6); + public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = "hoodie.memory.compaction.fraction"; + // Default max memory fraction during compaction, excess spills to disk + public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6); + // Default memory size per compaction (used if SparkEnv is absent), excess spills to disk + public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = + 1024 * 1024 * 1024L; // 1GB + // Property to set the max memory for merge + public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size"; + // Property to set the max memory for compaction + public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size"; + + + private HoodieMemoryConfig(Properties props) { + super(props); + } + + public static HoodieMemoryConfig.Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + + private final Properties props = new Properties(); + + public Builder fromFile(File propertiesFile) throws IOException { + FileReader reader = new FileReader(propertiesFile); + try { + this.props.load(reader); + return this; + } finally { + reader.close(); + } + } + + public Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public Builder withMaxMemoryFractionPerPartitionMerge(long maxMemoryFractionPerPartitionMerge) { + props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, + String.valueOf(maxMemoryFractionPerPartitionMerge)); + return this; + } + + public Builder withMaxMemoryFractionPerCompaction(long maxMemoryFractionPerCompaction) { + props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, + String.valueOf(maxMemoryFractionPerCompaction)); + return this; + } + + /** + * Dynamic calculation of max memory to use for for spillable map. user.available.memory = + * spark.executor.memory * (1 - spark.memory.fraction) spillable.available.memory = + * user.available.memory * hoodie.memory.fraction. Anytime the spark.executor.memory or the + * spark.memory.fraction is changed, the memory used for spillable map changes accordingly + */ + private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) { + final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory"; + final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction"; + // This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala#L231} + // so have to re-define this here + final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6"; + // This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/SparkContext.scala#L471} + // so have to re-define this here + final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB + + if (SparkEnv.get() != null) { + // 1 GB is the default conf used by Spark, look at SparkContext.scala + long executorMemoryInBytes = Long.valueOf( + Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP, + DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 + * 1024L); + // 0.6 is the default value used by Spark, + // look at {@link https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} + double memoryFraction = Double + .valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, + DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION)); + double maxMemoryFractionForMerge = Double.valueOf(maxMemoryFraction); + double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction); + long maxMemoryForMerge = (long) Math + .floor(userAvailableMemory * maxMemoryFractionForMerge); + return maxMemoryForMerge; + } else { + return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; + } + } + + public HoodieMemoryConfig build() { + HoodieMemoryConfig config = new HoodieMemoryConfig(props); + setDefaultOnCondition(props, + !props.containsKey(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP), + MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, + DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION); + setDefaultOnCondition(props, + !props.containsKey(MAX_MEMORY_FRACTION_FOR_MERGE_PROP), + MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE); + setDefaultOnCondition(props, + !props.containsKey(MAX_MEMORY_FOR_MERGE_PROP), + MAX_MEMORY_FOR_MERGE_PROP, String.valueOf( + getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP)))); + setDefaultOnCondition(props, + !props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP), + MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf( + getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP)))); + return config; + } + } + +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index d0911dca1ec22..14909bffae38e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -48,7 +48,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism"; private static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes"; - private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4*1024*1024); + private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4 * 1024 * 1024); private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert"; private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false"; private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert"; @@ -108,7 +108,8 @@ public int getUpsertShuffleParallelism() { } public int getWriteBufferLimitBytes() { - return Integer.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES)); + return Integer + .parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES)); } public boolean shouldCombineBeforeInsert() { @@ -217,17 +218,18 @@ public Long getTargetIOPerCompactionInMB() { .parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP)); } - public Long getMaxMemorySizePerCompactionInBytes() { - return Long - .parseLong(props.getProperty(HoodieCompactionConfig.MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP)); - } - public Boolean getCompactionLazyBlockReadEnabled() { - return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP)); + return Boolean + .valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP)); } public Boolean getCompactionReverseLogReadEnabled() { - return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP)); + return Boolean.valueOf( + props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP)); + } + + public String getPayloadClass() { + return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP); } /** @@ -283,7 +285,8 @@ public int getNumBucketsPerPartition() { } public StorageLevel getBloomIndexInputStorageLevel() { - return StorageLevel.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL)); + return StorageLevel + .fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL)); } /** @@ -302,7 +305,8 @@ public int getParquetPageSize() { } public int getLogFileDataBlockMaxSize() { - return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES)); + return Integer + .parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES)); } public int getLogFileMaxSize() { @@ -313,7 +317,6 @@ public double getParquetCompressionRatio() { return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO)); } - /** * metrics properties **/ @@ -342,6 +345,28 @@ public static HoodieWriteConfig.Builder newBuilder() { return new Builder(); } + /** + * memory configs + */ + public Double getMaxMemoryFractionPerPartitionMerge() { + return Double.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP)); + } + + public Double getMaxMemoryFractionPerCompaction() { + return Double + .valueOf( + props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP)); + } + + public Long getMaxMemoryPerPartitionMerge() { + return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP)); + } + + public Long getMaxMemoryPerCompaction() { + return Long + .valueOf( + props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP)); + } public static class Builder { @@ -351,6 +376,7 @@ public static class Builder { private boolean isCompactionConfigSet = false; private boolean isMetricsConfigSet = false; private boolean isAutoCommit = true; + private boolean isMemoryConfigSet = false; public Builder fromFile(File propertiesFile) throws IOException { FileReader reader = new FileReader(propertiesFile); @@ -501,9 +527,11 @@ public HoodieWriteConfig build() { setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE), - HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE); + HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, + DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE); setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE), - HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE); + HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, + DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE); setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM); @@ -516,6 +544,8 @@ public HoodieWriteConfig build() { HoodieCompactionConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isMemoryConfigSet, + HoodieMemoryConfig.newBuilder().fromProperties(props).build()); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 41cf3644ce640..f9795ceae19a1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -19,7 +19,6 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; -import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -65,10 +64,12 @@ public Path makeNewPath(String partitionPath, int taskPartitionId, String fileNa FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)); } - public Path makeTempPath(String partitionPath, int taskPartitionId, String fileName, int stageId, long taskAttemptId) { + public Path makeTempPath(String partitionPath, int taskPartitionId, String fileName, int stageId, + long taskAttemptId) { Path path = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME); return new Path(path.toString(), - FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId)); + FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, + taskAttemptId)); } /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index ce5f1e9e42a4c..c5ee6da8bd772 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -25,17 +25,15 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ReflectionUtils; +import com.uber.hoodie.common.util.collection.ExternalSpillableMap; +import com.uber.hoodie.common.util.collection.converter.StringConverter; +import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.io.storage.HoodieStorageWriter; import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; -import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Optional; - import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -43,6 +41,11 @@ import org.apache.log4j.Logger; import org.apache.spark.TaskContext; +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; + @SuppressWarnings("Duplicates") public class HoodieMergeHandle extends HoodieIOHandle { @@ -145,9 +148,14 @@ private void init(String fileId, String partitionPath) { * @return */ private String init(String fileId, Iterator> newRecordsItr) { - // Load the new records in a map - // TODO (NA) instantiate a ExternalSpillableMap - this.keyToNewRecords = new HashMap<>(); + try { + // Load the new records in a map + logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); + this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), + Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass())); + } catch(IOException io) { + throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); + } String partitionPath = null; while (newRecordsItr.hasNext()) { HoodieRecord record = newRecordsItr.next(); @@ -156,6 +164,15 @@ private String init(String fileId, Iterator> newRecordsItr) { // update the new location of the record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); } + logger.debug("Number of entries in MemoryBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() + + "Total size in bytes of MemoryBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + + "Number of entries in DiskBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + + "Size of file spilled to disk => " + + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); + return partitionPath; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 62a606efa8fd0..094ee0200e129 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -98,11 +98,12 @@ private List compact(HoodieTable hoodieTable, .getTimelineOfActions( Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .filterCompletedInstants().lastInstant().get().getTimestamp(); + log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime, - config.getMaxMemorySizePerCompactionInBytes(), config.getCompactionLazyBlockReadEnabled(), + config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled()); if (!scanner.iterator().hasNext()) { return Lists.newArrayList(); diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 0d29828ab0dbd..7c5a9adb50445 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -135,5 +135,10 @@ 1.16.0 test
+ + com.twitter.common + objectsize + 0.0.12 +
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index c8fe0e9e3c3d4..5dcb2c83cd407 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.table.log; -import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; @@ -29,8 +28,9 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; +import com.uber.hoodie.common.util.collection.converter.StringConverter; +import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; import com.uber.hoodie.exception.HoodieIOException; -import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -48,6 +48,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; @@ -102,8 +103,8 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List(maxMemorySizeInBytes, readerSchema, - payloadClassFQN, Optional.empty()); + this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, Optional.empty(), + new StringConverter(), new HoodieRecordConverter(readerSchema, payloadClassFQN)); // iterate over the paths HoodieLogFormatReader logFormatReaderWrapper = new HoodieLogFormatReader(fs, diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index 4f915897e872f..bdd5e692141f6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -19,11 +19,10 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.collection.DiskBasedMap; +import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieCorruptedDataException; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.RandomAccessFile; @@ -32,28 +31,23 @@ public class SpillableMapUtils { - public static ObjectMapper objectMapper = new ObjectMapper(); /** * Using the schema and payload class, read and convert the bytes on disk to a HoodieRecord + * * @param file - * @param schema - * @param payloadClazz * @param valuePosition * @param valueLength - * @param * @return * @throws IOException */ - public static R readFromDisk(RandomAccessFile file, Schema schema, String payloadClazz, - long valuePosition, int valueLength) throws IOException { - + public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) throws IOException { DiskBasedMap.FileEntry fileEntry = readInternal(file, valuePosition, valueLength); - return (R) convertToHoodieRecordPayload(HoodieAvroUtils.bytesToAvro(fileEntry.getValue(), schema), - payloadClazz); + return fileEntry.getValue(); } /** * |crc|timestamp|sizeOfKey|SizeOfValue|key|value| + * * @param file * @param valuePosition * @param valueLength @@ -66,15 +60,15 @@ private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long v long timestamp = file.readLong(); int keySize = file.readInt(); int valueSize = file.readInt(); - byte [] key = new byte[keySize]; + byte[] key = new byte[keySize]; file.read(key, 0, keySize); - byte [] value = new byte[valueSize]; - if(!(valueSize == valueLength)) { + byte[] value = new byte[valueSize]; + if (!(valueSize == valueLength)) { throw new HoodieCorruptedDataException("unequal size of payload written to external file, data may be corrupted"); } file.read(value, 0, valueSize); long crcOfReadValue = generateChecksum(value); - if(!(crc == crcOfReadValue)) { + if (!(crc == crcOfReadValue)) { throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " + "data may be corrupted"); } @@ -83,7 +77,7 @@ private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long v /** * Write Value and other metadata necessary to disk. Each entry has the following sequence of data - * + *

* |crc|timestamp|sizeOfKey|SizeOfValue|key|value| * * @param outputStream @@ -108,10 +102,11 @@ private static long spill(SizeAwareDataOutputStream outputStream, DiskBasedMap.F /** * Generate a checksum for a given set of bytes + * * @param data * @return */ - public static long generateChecksum(byte [] data) { + public static long generateChecksum(byte[] data) { CRC32 crc = new CRC32(); crc.update(data); return crc.getValue(); @@ -120,20 +115,19 @@ public static long generateChecksum(byte [] data) { /** * Compute a bytes representation of the payload by serializing the contents * This is used to estimate the size of the payload (either in memory or when written to disk) + * * @param * @param value - * @param schema * @return * @throws IOException */ - public static int computePayloadSize(R value, Schema schema) throws IOException { - HoodieRecord payload = (HoodieRecord) value; - byte [] val = HoodieAvroUtils.avroToBytes((GenericRecord) payload.getData().getInsertValue(schema).get()); - return val.length; + public static long computePayloadSize(R value, Converter valueConverter) throws IOException { + return valueConverter.sizeEstimate(value); } /** * Utility method to convert bytes to HoodieRecord using schema and payload class + * * @param rec * @param payloadClazz * @param diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java index 2503c409fb554..27ee0fba2be1f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -16,15 +16,14 @@ package com.uber.hoodie.common.util.collection; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import java.io.File; import java.io.FileOutputStream; @@ -48,13 +47,11 @@ * 1) An in-memory map that tracks the key-> latest ValueMetadata. * 2) Current position in the file * NOTE : Only String.class type supported for Key - * @param - * @param */ -final public class DiskBasedMap implements Map { +final public class DiskBasedMap implements Map { // Stores the key and corresponding value's latest metadata spilled to disk - final private Map inMemoryMetadataOfSpilledData; + final private Map valueMetadataMap; // Read only file access to be able to seek to random positions to readFromDisk values private RandomAccessFile readOnlyFileHandle; // Write only OutputStream to be able to ONLY append to the file @@ -64,16 +61,20 @@ final public class DiskBasedMap implements Map { private FileOutputStream fileOutputStream; // Current position in the file private AtomicLong filePosition; - // Schema used to de-serialize payload written to disk - private Schema schema; - // Class used to de-serialize/realize payload written to disk - private String payloadClazz; // FilePath to store the spilled data private String filePath; // Default file path prefix to put the spillable file private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; + // Key converter to convert key type to bytes + final private Converter keyConverter; + // Value converter to convert value type to bytes + final private Converter valueConverter; + + private static Logger log = LogManager.getLogger(DiskBasedMap.class); + public final class ValueMetadata { + // FilePath to store the spilled data private String filePath; // Size (numberOfBytes) of the value written to disk @@ -108,6 +109,7 @@ public long getTimestamp() { } public static final class FileEntry { + // Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption private Long crc; // Size (numberOfBytes) of the key written to disk @@ -115,13 +117,14 @@ public static final class FileEntry { // Size (numberOfBytes) of the value written to disk private Integer sizeOfValue; // Actual key - private byte [] key; + private byte[] key; // Actual value - private byte [] value; + private byte[] value; // Current timestamp when the value was written to disk private Long timestamp; - public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte [] key, byte [] value, long timestamp) { + public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value, + long timestamp) { this.crc = crc; this.sizeOfKey = sizeOfKey; this.sizeOfValue = sizeOfValue; @@ -155,10 +158,11 @@ public long getTimestamp() { } } - protected DiskBasedMap(Schema schema, String payloadClazz, Optional baseFilePath) throws IOException { - this.inMemoryMetadataOfSpilledData = new HashMap<>(); + protected DiskBasedMap(Optional baseFilePath, + Converter keyConverter, Converter valueConverter) throws IOException { + this.valueMetadataMap = new HashMap<>(); - if(!baseFilePath.isPresent()) { + if (!baseFilePath.isPresent()) { baseFilePath = Optional.of(DEFAULT_BASE_FILE_PATH); } this.filePath = baseFilePath.get() + UUID.randomUUID().toString(); @@ -168,16 +172,18 @@ protected DiskBasedMap(Schema schema, String payloadClazz, Optional base this.fileOutputStream = new FileOutputStream(writeOnlyFileHandle, true); this.writeOnlyFileHandle = new SizeAwareDataOutputStream(fileOutputStream); this.filePosition = new AtomicLong(0L); - this.schema = schema; - this.payloadClazz = payloadClazz; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; } private void initFile(File writeOnlyFileHandle) throws IOException { // delete the file if it exists - if(writeOnlyFileHandle.exists()) { + if (writeOnlyFileHandle.exists()) { writeOnlyFileHandle.delete(); } writeOnlyFileHandle.createNewFile(); + + log.info("Spilling to file location " + writeOnlyFileHandle.getAbsolutePath()); // Open file in readFromDisk-only mode readOnlyFileHandle = new RandomAccessFile(filePath, "r"); readOnlyFileHandle.seek(0); @@ -194,12 +200,12 @@ private void addShutDownHook() { Runtime.getRuntime().addShutdownHook(new Thread() { public void run() { try { - if(writeOnlyFileHandle != null) { + if (writeOnlyFileHandle != null) { writeOnlyFileHandle.flush(); fileOutputStream.getChannel().force(false); writeOnlyFileHandle.close(); } - } catch(Exception e) { + } catch (Exception e) { // fail silently for any sort of exception } } @@ -208,16 +214,14 @@ public void run() { /** * Custom iterator to iterate over values written to disk - * @return */ public Iterator iterator() { return new LazyFileIterable(readOnlyFileHandle, - inMemoryMetadataOfSpilledData, schema, payloadClazz).iterator(); + valueMetadataMap, valueConverter).iterator(); } /** * Number of bytes spilled to disk - * @return */ public long sizeOfFileOnDiskInBytes() { return filePosition.get(); @@ -225,17 +229,17 @@ public long sizeOfFileOnDiskInBytes() { @Override public int size() { - return inMemoryMetadataOfSpilledData.size(); + return valueMetadataMap.size(); } @Override public boolean isEmpty() { - return inMemoryMetadataOfSpilledData.isEmpty(); + return valueMetadataMap.isEmpty(); } @Override public boolean containsKey(Object key) { - return inMemoryMetadataOfSpilledData.containsKey(key); + return valueMetadataMap.containsKey(key); } @Override @@ -245,34 +249,31 @@ public boolean containsValue(Object value) { @Override public R get(Object key) { - ValueMetadata entry = inMemoryMetadataOfSpilledData.get(key); - if(entry == null) { + ValueMetadata entry = valueMetadataMap.get(key); + if (entry == null) { return null; } try { - return SpillableMapUtils.readFromDisk(readOnlyFileHandle, schema, - payloadClazz, entry.getOffsetOfValue(), entry.getSizeOfValue()); - } catch(IOException e) { + return this.valueConverter.getData(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle, + entry.getOffsetOfValue(), entry.getSizeOfValue())); + } catch (IOException e) { throw new HoodieIOException("Unable to readFromDisk Hoodie Record from disk", e); } } @Override public R put(T key, R value) { - //TODO (na) : check value instanceof HoodieRecordPayload, now assume every payload is HoodieRecord - HoodieRecord payload = (HoodieRecord) value; try { - byte [] val = HoodieAvroUtils.avroToBytes((GenericRecord) payload.getData().getInsertValue(this.schema).get()); + byte[] val = this.valueConverter.getBytes(value); Integer valueSize = val.length; Long timestamp = new Date().getTime(); - this.inMemoryMetadataOfSpilledData.put(key, new DiskBasedMap.ValueMetadata(this.filePath, valueSize, - filePosition.get(), timestamp)); - // TODO(na) : Test serializer performance for generic types - String serializedKey = SpillableMapUtils.objectMapper.writeValueAsString(key); + this.valueMetadataMap.put(key, + new DiskBasedMap.ValueMetadata(this.filePath, valueSize, filePosition.get(), timestamp)); + byte[] serializedKey = keyConverter.getBytes(key); filePosition.set(SpillableMapUtils.spillToDisk(writeOnlyFileHandle, new FileEntry(SpillableMapUtils.generateChecksum(val), - serializedKey.getBytes().length, valueSize, serializedKey.getBytes(), val, timestamp))); - } catch(IOException io) { + serializedKey.length, valueSize, serializedKey, val, timestamp))); + } catch (IOException io) { throw new HoodieIOException("Unable to store data in Disk Based map", io); } return value; @@ -281,33 +282,33 @@ public R put(T key, R value) { @Override public R remove(Object key) { R value = get(key); - inMemoryMetadataOfSpilledData.remove(key); + valueMetadataMap.remove(key); return value; } @Override public void putAll(Map m) { - for(Map.Entry entry: m.entrySet()) { + for (Map.Entry entry : m.entrySet()) { put(entry.getKey(), entry.getValue()); } } @Override public void clear() { - inMemoryMetadataOfSpilledData.clear(); + valueMetadataMap.clear(); // close input/output streams try { writeOnlyFileHandle.flush(); writeOnlyFileHandle.close(); new File(filePath).delete(); - } catch(IOException e) { + } catch (IOException e) { throw new HoodieIOException("unable to clear map or delete file on disk", e); } } @Override public Set keySet() { - return inMemoryMetadataOfSpilledData.keySet(); + return valueMetadataMap.keySet(); } @Override @@ -318,7 +319,7 @@ public Collection values() { @Override public Set> entrySet() { Set> entrySet = new HashSet<>(); - for(T key: inMemoryMetadataOfSpilledData.keySet()) { + for (T key : valueMetadataMap.keySet()) { entrySet.add(new AbstractMap.SimpleEntry<>(key, get(key))); } return entrySet; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index 261ca2c31f602..3d390690edc06 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -16,10 +16,10 @@ package com.uber.hoodie.common.util.collection; -import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.exception.HoodieIOException; +import com.twitter.common.objectsize.ObjectSizeCalculator; +import com.uber.hoodie.common.util.collection.converter.Converter; +import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -33,56 +33,54 @@ import java.util.Set; /** - * An external map that spills content to disk when there is insufficient space for it - * to grow. - * - * This map holds 2 types of data structures : - * - * (1) Key-Value pairs in a in-memory map - * (2) Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk - * - * NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed from the in-memory - * key-valueMetadata map but it's values will be lying around in the temp file on disk until the file is cleaned. - * - * The setting of the spill threshold faces the following trade-off: If the spill threshold is - * too high, the in-memory map may occupy more memory than is available, resulting in OOM. - * However, if the spill threshold is too low, we spill frequently and incur unnecessary disk - * writes. - * @param - * @param + * An external map that spills content to disk when there is insufficient space for it to grow.

+ * This map holds 2 types of data structures :

(1) Key-Value pairs in a in-memory map (2) + * Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk + *

NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed + * from the in-memory key-valueMetadata map but it's values will be lying around in the temp file on + * disk until the file is cleaned.

The setting of the spill threshold faces the following + * trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is + * available, resulting in OOM. However, if the spill threshold is too low, we spill frequently and + * incur unnecessary disk writes. */ -public class ExternalSpillableMap implements Map { +public class ExternalSpillableMap implements Map { + // Find the actual estimated payload size after inserting N records + final private static int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100; // maximum space allowed in-memory for this map final private long maxInMemorySizeInBytes; // current space occupied by this map in-memory private Long currentInMemoryMapSize; // Map to store key-values in memory until it hits maxInMemorySizeInBytes - final private Map inMemoryMap; + final private Map inMemoryMap; // Map to store key-valuemetadata important to find the values spilled to disk - final private DiskBasedMap diskBasedMap; - // Schema used to de-serialize and readFromDisk the records written to disk - final private Schema schema; + final private DiskBasedMap diskBasedMap; // An estimate of the size of each payload written to this map private volatile long estimatedPayloadSize = 0; // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and incorrect payload estimation final private Double sizingFactorForInMemoryMap = 0.8; + // Key converter to convert key type to bytes + final private Converter keyConverter; + // Value converter to convert value type to bytes + final private Converter valueConverter; + // Flag to determine whether to stop re-estimating payload size + private boolean shouldEstimatePayloadSize = true; private static Logger log = LogManager.getLogger(ExternalSpillableMap.class); - - public ExternalSpillableMap(Long maxInMemorySizeInBytes, Schema schema, - String payloadClazz, Optional baseFilePath) throws IOException { + public ExternalSpillableMap(Long maxInMemorySizeInBytes, Optional baseFilePath, + Converter keyConverter, Converter valueConverter) throws IOException { this.inMemoryMap = new HashMap<>(); - this.diskBasedMap = new DiskBasedMap<>(schema, payloadClazz, baseFilePath); - this.maxInMemorySizeInBytes = (long) Math.floor(maxInMemorySizeInBytes*sizingFactorForInMemoryMap); - this.schema = schema; + this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter); + this.maxInMemorySizeInBytes = (long) Math + .floor(maxInMemorySizeInBytes * sizingFactorForInMemoryMap); this.currentInMemoryMapSize = 0L; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; } /** * A custom iterator to wrap over iterating in-memory + disk spilled data - * @return */ public Iterator iterator() { return new IteratorWrapper<>(inMemoryMap.values().iterator(), diskBasedMap.iterator()); @@ -90,7 +88,6 @@ public Iterator iterator() { /** * Number of entries in DiskBasedMap - * @return */ public int getDiskBasedMapNumEntries() { return diskBasedMap.size(); @@ -98,7 +95,6 @@ public int getDiskBasedMapNumEntries() { /** * Number of bytes spilled to disk - * @return */ public long getSizeOfFileOnDiskInBytes() { return diskBasedMap.sizeOfFileOnDiskInBytes(); @@ -106,7 +102,6 @@ public long getSizeOfFileOnDiskInBytes() { /** * Number of entries in InMemoryMap - * @return */ public int getInMemoryMapNumEntries() { return inMemoryMap.size(); @@ -114,7 +109,6 @@ public int getInMemoryMapNumEntries() { /** * Approximate memory footprint of the in-memory map - * @return */ public long getCurrentInMemoryMapSize() { return currentInMemoryMapSize; @@ -142,9 +136,9 @@ public boolean containsValue(Object value) { @Override public R get(Object key) { - if(inMemoryMap.containsKey(key)) { + if (inMemoryMap.containsKey(key)) { return inMemoryMap.get(key); - } else if(diskBasedMap.containsKey(key)) { + } else if (diskBasedMap.containsKey(key)) { return diskBasedMap.get(key); } return null; @@ -152,33 +146,43 @@ public R get(Object key) { @Override public R put(T key, R value) { - try { - if (this.currentInMemoryMapSize < maxInMemorySizeInBytes || inMemoryMap.containsKey(key)) { - // Naive approach for now - if (estimatedPayloadSize == 0) { - this.estimatedPayloadSize = SpillableMapUtils.computePayloadSize(value, schema); - log.info("Estimated Payload size => " + estimatedPayloadSize); - } - if(!inMemoryMap.containsKey(key)) { - currentInMemoryMapSize += this.estimatedPayloadSize; - } - inMemoryMap.put(key, value); - } else { - diskBasedMap.put(key, value); + if (this.currentInMemoryMapSize < maxInMemorySizeInBytes || inMemoryMap.containsKey(key)) { + if (shouldEstimatePayloadSize && estimatedPayloadSize == 0) { + // At first, use the sizeEstimate of a record being inserted into the spillable map. + // Note, the converter may over estimate the size of a record in the JVM + this.estimatedPayloadSize = + keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value); + log.info("Estimated Payload size => " + estimatedPayloadSize); + } + else if(shouldEstimatePayloadSize && + inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { + // Re-estimate the size of a record by calculating the size of the entire map containing + // N entries and then dividing by the number of entries present (N). This helps to get a + // correct estimation of the size of each record in the JVM. + long totalMapSize = ObjectSizeCalculator.getObjectSize(inMemoryMap); + this.currentInMemoryMapSize = totalMapSize; + this.estimatedPayloadSize = totalMapSize/inMemoryMap.size(); + shouldEstimatePayloadSize = false; + log.info("New Estimated Payload size => " + this.estimatedPayloadSize); } - return value; - } catch(IOException io) { - throw new HoodieIOException("Unable to estimate size of payload", io); + if (!inMemoryMap.containsKey(key)) { + // TODO : Add support for adjusting payloadSize for updates to the same key + currentInMemoryMapSize += this.estimatedPayloadSize; + } + inMemoryMap.put(key, value); + } else { + diskBasedMap.put(key, value); } + return value; } @Override public R remove(Object key) { // NOTE : diskBasedMap.remove does not delete the data from disk - if(inMemoryMap.containsKey(key)) { + if (inMemoryMap.containsKey(key)) { currentInMemoryMapSize -= estimatedPayloadSize; return inMemoryMap.remove(key); - } else if(diskBasedMap.containsKey(key)) { + } else if (diskBasedMap.containsKey(key)) { return diskBasedMap.remove(key); } return null; @@ -186,7 +190,7 @@ public R remove(Object key) { @Override public void putAll(Map m) { - for(Map.Entry entry: m.entrySet()) { + for (Map.Entry entry : m.entrySet()) { put(entry.getKey(), entry.getValue()); } } @@ -208,7 +212,7 @@ public Set keySet() { @Override public Collection values() { - if(diskBasedMap.isEmpty()) { + if (diskBasedMap.isEmpty()) { return inMemoryMap.values(); } throw new HoodieNotSupportedException("Cannot return all values in memory"); @@ -226,7 +230,6 @@ public Set> entrySet() { * Iterator that wraps iterating over all the values for this map * 1) inMemoryIterator - Iterates over all the data in-memory map * 2) diskLazyFileIterator - Iterates over all the data spilled to disk - * @param */ private class IteratorWrapper implements Iterator { @@ -237,9 +240,10 @@ public IteratorWrapper(Iterator inMemoryIterator, Iterator diskLazyFileIte this.inMemoryIterator = inMemoryIterator; this.diskLazyFileIterator = diskLazyFileIterator; } + @Override public boolean hasNext() { - if(inMemoryIterator.hasNext()) { + if (inMemoryIterator.hasNext()) { return true; } return diskLazyFileIterator.hasNext(); @@ -247,7 +251,7 @@ public boolean hasNext() { @Override public R next() { - if(inMemoryIterator.hasNext()) { + if (inMemoryIterator.hasNext()) { return inMemoryIterator.next(); } return diskLazyFileIterator.next(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java index f91894ecdc193..78eaa3f348c48 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java @@ -17,14 +17,13 @@ package com.uber.hoodie.common.util.collection; import com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.avro.Schema; import java.io.IOException; import java.io.RandomAccessFile; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.Map; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -33,57 +32,52 @@ * Iterable to lazily fetch values spilled to disk. * This class uses RandomAccessFile to randomly access the position of * the latest value for a key spilled to disk and returns the result. - * @param */ -public class LazyFileIterable implements Iterable { +public class LazyFileIterable implements Iterable { // Used to access the value written at a specific position in the file - private RandomAccessFile readOnlyFileHandle; + private final RandomAccessFile readOnlyFileHandle; // Stores the key and corresponding value's latest metadata spilled to disk - private Map inMemoryMetadataOfSpilledData; - // Schema used to de-serialize payload written to disk - private Schema schema; - // Class used to de-serialize/realize payload written to disk - private String payloadClazz; + private final Map inMemoryMetadataOfSpilledData; + private final Converter valueConverter; public LazyFileIterable(RandomAccessFile file, Map map, - Schema schema, String payloadClazz) { + Converter valueConverter) { this.readOnlyFileHandle = file; this.inMemoryMetadataOfSpilledData = map; - this.schema = schema; - this.payloadClazz = payloadClazz; + this.valueConverter = valueConverter; } + @Override - public Iterator iterator() { + public Iterator iterator() { try { - return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData, schema, payloadClazz); - } catch(IOException io) { + return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData, + valueConverter); + } catch (IOException io) { throw new HoodieException("Unable to initialize iterator for file on disk", io); } } /** * Iterator implementation for the iterable defined above. - * @param */ - public class LazyFileIterator implements Iterator { + public class LazyFileIterator implements Iterator { private RandomAccessFile readOnlyFileHandle; - private Schema schema; - private String payloadClazz; private Iterator> metadataIterator; + private final Converter valueConverter; public LazyFileIterator(RandomAccessFile file, Map map, - Schema schema, String payloadClazz) throws IOException { + Converter valueConverter) throws IOException { this.readOnlyFileHandle = file; - this.schema = schema; - this.payloadClazz = payloadClazz; + this.valueConverter = valueConverter; // sort the map in increasing order of offset of value so disk seek is only in one(forward) direction this.metadataIterator = map .entrySet() .stream() - .sorted((Map.Entry o1, Map.Entry o2) -> - o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue())) + .sorted( + (Map.Entry o1, Map.Entry o2) -> + o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue())) .collect(Collectors.toList()).iterator(); } @@ -93,12 +87,12 @@ public boolean hasNext() { } @Override - public T next() { + public R next() { Map.Entry entry = this.metadataIterator.next(); try { - return SpillableMapUtils.readFromDisk(readOnlyFileHandle, schema, - payloadClazz, entry.getValue().getOffsetOfValue(), entry.getValue().getSizeOfValue()); - } catch(IOException e) { + return valueConverter.getData(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle, + entry.getValue().getOffsetOfValue(), entry.getValue().getSizeOfValue())); + } catch (IOException e) { throw new HoodieIOException("Unable to read hoodie record from value spilled to disk", e); } } @@ -109,7 +103,7 @@ public void remove() { } @Override - public void forEachRemaining(Consumer action) { + public void forEachRemaining(Consumer action) { action.accept(next()); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java new file mode 100644 index 0000000000000..b06651973cb5e --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java @@ -0,0 +1,39 @@ +/* + * 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.common.util.collection.converter; + +/** + * A converter interface to getBytes or deserialize a payload. This is used in {@link + * com.uber.hoodie.common.util.collection.ExternalSpillableMap} to spillToDisk + */ +public interface Converter { + + /** + * This method is used to convert a payload to bytes + */ + byte[] getBytes(T t); + + /** + * This method is used to convert the serialized payload (in bytes) to the actual payload instance + */ + T getData(byte[] bytes); + + /** + * This method is used to estimate the size of a payload in memory + */ + long sizeEstimate(T t); +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java new file mode 100644 index 0000000000000..73dc0541ad0c3 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -0,0 +1,107 @@ +/* + * 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.common.util.collection.converter; + +import com.twitter.common.objectsize.ObjectSizeCalculator; +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.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.ReflectionUtils; +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieNotSerializableException; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Optional; +import java.util.stream.Stream; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.lang3.SerializationUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * A default converter implementation for HoodieRecord + */ +public class HoodieRecordConverter implements + Converter> { + + // Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa + private final Schema schema; + // The client implementation of HoodieRecordPayload used to re-create HoodieRecord from bytes + private final String payloadClazz; + + private static Logger log = LogManager.getLogger(HoodieRecordConverter.class); + + public HoodieRecordConverter(Schema schema, String payloadClazz) { + this.schema = schema; + this.payloadClazz = payloadClazz; + } + + @Override + public byte[] getBytes(HoodieRecord hoodieRecord) { + try { + // Need to initialize this to 0 bytes since deletes are handled by putting an empty record in HoodieRecord + byte[] val = new byte[0]; + if (hoodieRecord.getData().getInsertValue(schema).isPresent()) { + val = HoodieAvroUtils + .avroToBytes((GenericRecord) hoodieRecord.getData().getInsertValue(schema).get()); + } + Pair, byte[]> data = + Pair.of(Pair.of(hoodieRecord.getKey().getRecordKey(), + hoodieRecord.getKey().getPartitionPath()), val); + return SerializationUtils.serialize(data); + } catch (IOException io) { + throw new HoodieNotSerializableException("Cannot serialize value to bytes", io); + } + } + + @Override + public HoodieRecord getData(byte[] bytes) { + try { + Pair, byte[]> data = SerializationUtils.deserialize(bytes); + Optional payload = Optional.empty(); + if (data.getValue().length > 0) { + // This can happen if the record is deleted, the payload is optional with 0 bytes + payload = Optional.of(HoodieAvroUtils.bytesToAvro(data.getValue(), schema)); + } + HoodieRecord hoodieRecord = new HoodieRecord<>( + new HoodieKey(data.getKey().getKey(), data.getKey().getValue()), + ReflectionUtils + .loadPayload(payloadClazz, + new Object[]{payload}, Optional.class)); + return hoodieRecord; + } catch (IOException io) { + throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io); + } + } + + @Override + public long sizeEstimate(HoodieRecord hoodieRecord) { + // Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst + // all records in the JVM. Calculate and print the size of the Schema and of the Record to + // note the sizes and differences. A correct estimation in such cases is handled in + /** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/ + long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord); + long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema); + log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); + return sizeOfRecord; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java new file mode 100644 index 0000000000000..ea84a7dd9467a --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java @@ -0,0 +1,41 @@ +/* + * 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.common.util.collection.converter; + +import com.twitter.common.objectsize.ObjectSizeCalculator; +import java.nio.charset.StandardCharsets; + +/** + * A default converter implementation for String type of payload key + */ +public class StringConverter implements Converter { + + @Override + public byte[] getBytes(String s) { + return s.getBytes(StandardCharsets.UTF_8); + } + + @Override + public String getData(byte[] bytes) { + return new String(bytes); + } + + @Override + public long sizeEstimate(String s) { + return ObjectSizeCalculator.getObjectSize(s); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java new file mode 100644 index 0000000000000..17650c79d4b76 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java @@ -0,0 +1,37 @@ +/* + * 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.exception; + +import java.io.IOException; + +public class HoodieNotSerializableException extends HoodieException { + + private IOException ioException; + + public HoodieNotSerializableException(String msg, IOException t) { + super(msg, t); + this.ioException = t; + } + + public HoodieNotSerializableException(String msg) { + super(msg); + } + + public IOException getIOException() { + return ioException; + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java new file mode 100644 index 0000000000000..984fb3a412c2c --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java @@ -0,0 +1,58 @@ +/* + * 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.common.model; + +import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Optional; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +public class AvroBinaryTestPayload implements HoodieRecordPayload { + + private final byte[] recordBytes; + + public AvroBinaryTestPayload(Optional record) { + + try { + if (record.isPresent()) { + recordBytes = HoodieAvroUtils.avroToBytes(record.get()); + } else { + recordBytes = new byte[0]; + } + } catch (IOException io) { + throw new HoodieIOException("unable to convert payload to bytes"); + } + } + + @Override + public HoodieRecordPayload preCombine(HoodieRecordPayload another) { + return this; + } + + @Override + public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + return getInsertValue(schema); + } + + @Override + public Optional getInsertValue(Schema schema) throws IOException { + return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); + } +} \ No newline at end of file diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java index 1de52e8ed593b..a736da8e5f40a 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java @@ -17,9 +17,13 @@ package com.uber.hoodie.common.util; import com.uber.hoodie.avro.MercifulJsonConverter; +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.exception.HoodieIOException; + import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -32,9 +36,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; @@ -109,21 +115,63 @@ public static List generateHoodieTestRecords(int from, int limit) } + public static List generateHoodieTestRecords(int from, int limit, Schema schema) + throws IOException, URISyntaxException { + List records = generateTestRecords(from, limit); + return records.stream() + .map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, schema)) + .map(p -> convertToHoodieRecords(p, + UUID.randomUUID().toString(), "000/00/00")).collect( + Collectors.toList()); + } + + private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) { + return new HoodieRecord<>(new HoodieKey(key, partitionPath), + new HoodieAvroPayload(Optional.of((GenericRecord) iRecord))); + } + public static List updateHoodieTestRecords(List oldRecordKeys, List newRecords, String commitTime) throws IOException, URISyntaxException { return newRecords.stream() .map(p -> { - ((GenericRecord)p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0)); - ((GenericRecord)p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); - ((GenericRecord)p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); + ((GenericRecord) p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0)); + ((GenericRecord) p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); + ((GenericRecord) p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); return p; }).collect( Collectors.toList()); } + public static List generateHoodieTestRecordsWithoutHoodieMetadata(int from, int limit) + throws IOException, URISyntaxException { + + List iRecords = generateTestRecords(from, limit); + return iRecords + .stream() + .map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), + new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList()); + } + + public static List updateHoodieTestRecordsWithoutHoodieMetadata(List oldRecords, Schema schema, + String fieldNameToUpdate, String newValue) + throws IOException, URISyntaxException { + return oldRecords + .stream() + .map(r -> { + try { + GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get(); + rec.put(fieldNameToUpdate, newValue); + return new HoodieRecord<>(r.getKey(), + new HoodieAvroPayload(Optional.of(rec))); + } catch (IOException io) { + throw new HoodieIOException("unable to get data from hoodie record", io); + } + }).collect(Collectors.toList()); + } + public static Schema getEvolvedSchema() throws IOException { return new Schema.Parser() .parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro")); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java index 05be501448d8d..f8360376ef644 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java @@ -36,8 +36,8 @@ public static List upsertRecords(List iRecords, iRecords .stream() .forEach(r -> { - String key = ((GenericRecord)r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - String partitionPath = ((GenericRecord)r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); recordKeys.add(key); records.put(key, new HoodieRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Optional.of((GenericRecord) r)))); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index eb570a6558ced..5eee986186fbf 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -16,45 +16,83 @@ 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.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.model.AvroBinaryTestPayload; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.junit.Test; - +import com.uber.hoodie.common.util.SpillableMapUtils; +import com.uber.hoodie.common.util.collection.converter.StringConverter; +import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; 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 static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; +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.Test; public class TestDiskBasedMap { @Test public void testSimpleInsert() throws IOException, URISyntaxException { - Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); - DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty()); + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); + + DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), + new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + String commitTime = ((GenericRecord) iRecords.get(0)) + .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); // make sure records have spilled to disk assertTrue(records.sizeOfFileOnDiskInBytes() > 0); Iterator> itr = records.iterator(); List oRecords = new ArrayList<>(); - while(itr.hasNext()) { + while (itr.hasNext()) { + HoodieRecord 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<>(Optional.empty(), + new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); + List hoodieRecords = SchemaTestUtil + .generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); + Set 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> itr = records.iterator(); + List oRecords = new ArrayList<>(); + while (itr.hasNext()) { HoodieRecord rec = itr.next(); oRecords.add(rec); assert recordKeys.contains(rec.getRecordKey()); @@ -64,10 +102,14 @@ public void testSimpleInsert() throws IOException, URISyntaxException { @Test public void testSimpleUpsert() throws IOException, URISyntaxException { - Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); - DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty()); + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); + + DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), + new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + String commitTime = ((GenericRecord) iRecords.get(0)) + .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); // perform some inserts List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -77,9 +119,11 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { // generate updates from inserts List updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100), - HoodieActiveTimeline.createNewCommitTime()); - String newCommitTime = ((GenericRecord)updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + SchemaTestUtil + .updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100), + HoodieActiveTimeline.createNewCommitTime()); + String newCommitTime = ((GenericRecord) updatedRecords.get(0)) + .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); // new commit time should be different assertEquals(commitTime, newCommitTime); @@ -92,16 +136,75 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { // Upserted records (on disk) should have the latest commit time Iterator> itr = records.iterator(); - while(itr.hasNext()) { + while (itr.hasNext()) { HoodieRecord 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(); + 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) { + } 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 hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); + + long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), + new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName())); + 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 HoodieRecordConverter(schema, HoodieAvroPayload.class.getName())); + 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 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 HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName())); + 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 HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName())); + assertTrue(payloadSize > 0); + } + + @Test + public void testSizeEstimatorPerformance() throws IOException, URISyntaxException { + // Test sizeEstimatorPerformance with simpleSchema + Schema schema = SchemaTestUtil.getSimpleSchema(); + List hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); + HoodieRecordConverter converter = + new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()); + HoodieRecord record = hoodieRecords.remove(0); + long startTime = System.currentTimeMillis(); + SpillableMapUtils.computePayloadSize(record, converter); + long timeTaken = System.currentTimeMillis() - startTime; + assertTrue(timeTaken < 100); + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index d15caf6457f01..1c56b6cb85e9f 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -16,6 +16,10 @@ package com.uber.hoodie.common.util.collection; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + import com.uber.hoodie.common.model.HoodieAvroPayload; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -24,13 +28,8 @@ import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.junit.FixMethodOrder; -import org.junit.Test; -import org.junit.runners.MethodSorters; - +import com.uber.hoodie.common.util.collection.converter.StringConverter; +import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -39,10 +38,12 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class TestExternalSpillableMap { @@ -51,17 +52,19 @@ public class TestExternalSpillableMap { @Test public void simpleInsertTest() throws IOException, URISyntaxException { + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = new ExternalSpillableMap<> - (16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()), - HoodieAvroPayload.class.getName(), Optional.empty()); //16B + (16L, Optional.empty(), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); assert (recordKeys.size() == 100); Iterator> itr = records.iterator(); List oRecords = new ArrayList<>(); - while(itr.hasNext()) { + while (itr.hasNext()) { HoodieRecord rec = itr.next(); oRecords.add(rec); assert recordKeys.contains(rec.getRecordKey()); @@ -72,16 +75,18 @@ public void simpleInsertTest() throws IOException, URISyntaxException { public void testSimpleUpsert() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); + ExternalSpillableMap> records = new ExternalSpillableMap<> - (16L, schema, - HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B + (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); assert (recordKeys.size() == 100); Iterator> itr = records.iterator(); - while(itr.hasNext()) { + while (itr.hasNext()) { HoodieRecord rec = itr.next(); assert recordKeys.contains(rec.getRecordKey()); } @@ -99,8 +104,8 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { updatedRecords.stream().forEach(record -> { HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); try { - assertEquals(rec.getData().getInsertValue(schema).get(),record); - } catch(IOException io) { + assertEquals(rec.getData().getInsertValue(schema).get(), record); + } catch (IOException io) { throw new UncheckedIOException(io); } }); @@ -109,25 +114,28 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { @Test public void testAllMapOperations() throws IOException, URISyntaxException { + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); + ExternalSpillableMap> records = new ExternalSpillableMap<> - (16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()), - HoodieAvroPayload.class.getName(), Optional.empty()); //16B + (16L, Optional.empty(), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // insert a bunch of records so that values spill to disk too List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); IndexedRecord inMemoryRecord = iRecords.get(0); - String ikey = ((GenericRecord)inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - String iPartitionPath = ((GenericRecord)inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + String ikey = ((GenericRecord) inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String iPartitionPath = ((GenericRecord) inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath), - new HoodieAvroPayload(Optional.of((GenericRecord)inMemoryRecord))); + new HoodieAvroPayload(Optional.of((GenericRecord) inMemoryRecord))); IndexedRecord onDiskRecord = iRecords.get(99); - String dkey = ((GenericRecord)onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - String dPartitionPath = ((GenericRecord)onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + String dkey = ((GenericRecord) onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String dPartitionPath = ((GenericRecord) onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath), - new HoodieAvroPayload(Optional.of((GenericRecord)onDiskRecord))); + new HoodieAvroPayload(Optional.of((GenericRecord) onDiskRecord))); // assert size assert records.size() == 100; // get should return the same HoodieKey and same value @@ -162,16 +170,19 @@ public void testAllMapOperations() throws IOException, URISyntaxException { @Test(expected = IOException.class) public void simpleTestWithException() throws IOException, URISyntaxException { + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); + ExternalSpillableMap> records = new ExternalSpillableMap<> - (16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()), - HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B + (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); assert (recordKeys.size() == 100); Iterator> itr = records.iterator(); - while(itr.hasNext()) { + while (itr.hasNext()) { throw new IOException("Testing failures..."); } } @@ -183,17 +194,19 @@ public void simpleTestWithExceptionValidateFileIsRemoved() throws Exception { } @Test - public void testDataCorrectnessInMapAndDisk() throws IOException, URISyntaxException { + public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException { + + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + String payloadClazz = HoodieAvroPayload.class.getName(); - Schema schema = SchemaTestUtil.getSimpleSchema(); ExternalSpillableMap> records = new ExternalSpillableMap<> - (16L, HoodieAvroUtils.addMetadataFields(schema), - HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B + (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk - while(records.getDiskBasedMapNumEntries() < 1) { + while (records.getDiskBasedMapNumEntries() < 1) { List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); recordKeys.addAll(SpillableMapTestUtils.upsertRecords(iRecords, records)); } @@ -202,7 +215,6 @@ public void testDataCorrectnessInMapAndDisk() throws IOException, URISyntaxExcep String key = recordKeys.get(0); HoodieRecord record = records.get(key); List recordsToUpdate = new ArrayList<>(); - schema = HoodieAvroUtils.addMetadataFields(schema); recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get()); String newCommitTime = HoodieActiveTimeline.createNewCommitTime(); @@ -235,7 +247,72 @@ record = records.get(key); gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); // The record returned for this key should have the updated commitTime assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + } + @Test + public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException { + + Schema schema = SchemaTestUtil.getSimpleSchema(); + String payloadClazz = HoodieAvroPayload.class.getName(); + + ExternalSpillableMap> records = + new ExternalSpillableMap<> + (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B + + List recordKeys = new ArrayList<>(); + // Ensure we spill to disk + while (records.getDiskBasedMapNumEntries() < 1) { + List hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 100); + hoodieRecords.stream().forEach(r -> { + records.put(r.getRecordKey(), r); + recordKeys.add(r.getRecordKey()); + }); + } + + // Get a record from the in-Memory map + String key = recordKeys.get(0); + HoodieRecord record = records.get(key); + // Get the field we want to update + String fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING) + .findAny().get().name(); + // Use a new value to update this field + String newValue = "update1"; + List recordsToUpdate = new ArrayList<>(); + recordsToUpdate.add(record); + + List updatedRecords = + SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue); + + // Upsert this updated record + updatedRecords.stream().forEach(r -> { + records.put(r.getRecordKey(), r); + }); + GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); + // The record returned for this key should have the updated value for the field name + assertEquals(gRecord.get(fieldName).toString(), newValue); + + // Get a record from the disk based map + key = recordKeys.get(recordKeys.size() - 1); + record = records.get(key); + // Get the field we want to update + fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING) + .findAny().get().name(); + // Use a new value to update this field + newValue = "update2"; + recordsToUpdate = new ArrayList<>(); + recordsToUpdate.add(record); + + updatedRecords = + SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue); + + // Upsert this updated record + updatedRecords.stream().forEach(r -> { + records.put(r.getRecordKey(), r); + }); + gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); + // The record returned for this key should have the updated value for the field name + assertEquals(gRecord.get(fieldName).toString(), newValue); } // TODO : come up with a performance eval test for spillableMap diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java index 74424ac36d592..6b12b1f97242b 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java @@ -44,6 +44,10 @@ public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingV super(record, orderingVal); } + public OverwriteWithLatestAvroPayload(Optional record) { + this(record.get(), (record1) -> 0); // natural order + } + @Override public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) { // pick the payload with greatest ordering value From 5c1dd7a02c24895991a3eae030e05bbe98ee730e Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 20 Mar 2018 16:29:20 -0700 Subject: [PATCH 041/374] CodeStyle formatting to conform to basic Checkstyle rules. The code-style rules follow google style with some changes: 1. Increase line length from 100 to 120 2. Disable JavaDoc related checkstyles as this needs more manual work. Both source and test code are checked for code-style --- docs/dev_setup.md | 9 +- .../com/uber/hoodie/cli/HoodiePrompt.java | 6 +- .../uber/hoodie/cli/HoodieSplashScreen.java | 28 +- .../main/java/com/uber/hoodie/cli/Main.java | 3 +- .../cli/commands/ArchivedCommitsCommand.java | 26 +- .../hoodie/cli/commands/CleansCommand.java | 38 +- .../hoodie/cli/commands/CommitsCommand.java | 148 +-- .../hoodie/cli/commands/DatasetsCommand.java | 7 +- .../commands/HDFSParquetImportCommand.java | 51 +- .../cli/commands/HoodieLogFileCommand.java | 80 +- .../cli/commands/HoodieSyncCommand.java | 76 +- .../hoodie/cli/commands/RepairsCommand.java | 40 +- .../cli/commands/SavepointsCommand.java | 48 +- .../uber/hoodie/cli/commands/SparkMain.java | 48 +- .../hoodie/cli/commands/StatsCommand.java | 62 +- .../hoodie/cli/commands/UtilsCommand.java | 4 +- .../com/uber/hoodie/cli/utils/CommitUtil.java | 11 +- .../com/uber/hoodie/cli/utils/HiveUtil.java | 39 +- .../hoodie/cli/utils/InputStreamConsumer.java | 2 +- .../com/uber/hoodie/cli/utils/SparkUtil.java | 12 +- .../com/uber/hoodie/HoodieReadClient.java | 46 +- .../com/uber/hoodie/HoodieWriteClient.java | 364 +++--- .../java/com/uber/hoodie/WriteStatus.java | 20 +- .../hoodie/config/DefaultHoodieConfig.java | 8 +- .../hoodie/config/HoodieCompactionConfig.java | 97 +- .../uber/hoodie/config/HoodieIndexConfig.java | 28 +- .../hoodie/config/HoodieMemoryConfig.java | 26 +- .../hoodie/config/HoodieMetricsConfig.java | 40 +- .../hoodie/config/HoodieStorageConfig.java | 5 +- .../uber/hoodie/config/HoodieWriteConfig.java | 54 +- .../uber/hoodie/func/BufferedIterator.java | 96 +- .../hoodie/func/BulkInsertMapFunction.java | 7 +- .../uber/hoodie/func/LazyInsertIterable.java | 83 +- .../hoodie/func/LazyIterableIterator.java | 4 +- .../com/uber/hoodie/index/HoodieIndex.java | 43 +- .../uber/hoodie/index/InMemoryHashIndex.java | 46 +- .../index/bloom/BloomIndexFileInfo.java | 8 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 208 ++-- .../bloom/HoodieBloomIndexCheckFunction.java | 24 +- .../hoodie/index/bucketed/BucketedIndex.java | 4 +- .../uber/hoodie/index/hbase/HBaseIndex.java | 195 ++-- .../uber/hoodie/io/HoodieAppendHandle.java | 114 +- .../com/uber/hoodie/io/HoodieCleanHelper.java | 62 +- .../hoodie/io/HoodieCommitArchiveLog.java | 49 +- .../uber/hoodie/io/HoodieCreateHandle.java | 31 +- .../com/uber/hoodie/io/HoodieIOHandle.java | 61 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 101 +- .../io/compact/CompactionOperation.java | 4 +- .../hoodie/io/compact/HoodieCompactor.java | 8 +- .../compact/HoodieRealtimeTableCompactor.java | 60 +- .../strategy/BoundedIOCompactionStrategy.java | 9 +- .../compact/strategy/CompactionStrategy.java | 8 +- .../strategy/DayBasedCompactionStrategy.java | 18 +- .../LogFileSizeBasedCompactionStrategy.java | 6 +- .../io/storage/HoodieParquetWriter.java | 42 +- .../storage/HoodieStorageWriterFactory.java | 18 +- .../io/storage/HoodieWrapperFileSystem.java | 177 ++- .../uber/hoodie/metrics/HoodieMetrics.java | 25 +- .../java/com/uber/hoodie/metrics/Metrics.java | 2 +- .../metrics/MetricsGraphiteReporter.java | 19 +- .../metrics/MetricsReporterFactory.java | 3 +- .../hoodie/metrics/MetricsReporterType.java | 3 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 1006 ++++++++--------- .../hoodie/table/HoodieMergeOnReadTable.java | 215 ++-- .../com/uber/hoodie/table/HoodieTable.java | 47 +- .../UserDefinedBulkInsertPartitioner.java | 1 + .../uber/hoodie/table/WorkloadProfile.java | 11 +- .../com/uber/hoodie/table/WorkloadStat.java | 3 +- .../src/test/java/HoodieClientExample.java | 22 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 569 ++++------ .../java/com/uber/hoodie/TestMultiFS.java | 23 +- .../hoodie/common/HoodieClientTestUtils.java | 69 +- .../common/HoodieMergeOnReadTestUtils.java | 13 +- .../common/HoodieTestDataGenerator.java | 171 ++- .../hoodie/common/TestRawTripPayload.java | 72 +- .../hoodie/config/HoodieWriteConfigTest.java | 3 +- .../hoodie/func/TestBufferedIterator.java | 320 +++--- .../hoodie/func/TestUpdateMapFunction.java | 43 +- .../com/uber/hoodie/index/TestHbaseIndex.java | 96 +- .../uber/hoodie/index/TestHoodieIndex.java | 12 +- .../index/bloom/TestHoodieBloomIndex.java | 253 ++--- .../hoodie/io/TestHoodieCommitArchiveLog.java | 66 +- .../uber/hoodie/io/TestHoodieCompactor.java | 55 +- .../TestHoodieCompactionStrategy.java | 74 +- .../io/strategy/TestHoodieDataFile.java | 9 +- .../hoodie/io/strategy/TestHoodieLogFile.java | 8 +- .../hoodie/metrics/TestHoodieMetrics.java | 3 +- .../hoodie/table/TestCopyOnWriteTable.java | 204 ++-- .../hoodie/table/TestMergeOnReadTable.java | 197 ++-- .../hoodie/avro/HoodieAvroWriteSupport.java | 6 +- .../hoodie/avro/MercifulJsonConverter.java | 15 +- .../uber/hoodie/common/HoodieCleanStat.java | 5 +- .../common/model/HoodieCommitMetadata.java | 12 +- .../hoodie/common/model/HoodieFileGroup.java | 11 +- .../uber/hoodie/common/model/HoodieKey.java | 6 +- .../hoodie/common/model/HoodieLogFile.java | 10 +- .../common/model/HoodiePartitionMetadata.java | 5 +- .../hoodie/common/model/HoodieRecord.java | 8 +- .../common/model/HoodieRecordLocation.java | 4 +- .../common/model/HoodieRecordPayload.java | 4 +- .../hoodie/common/model/HoodieTableType.java | 10 +- .../hoodie/common/model/HoodieWriteStat.java | 3 +- .../storage/SizeAwareDataInputStream.java | 70 +- .../common/table/HoodieTableConfig.java | 2 +- .../common/table/HoodieTableMetaClient.java | 2 +- .../log/HoodieCompactedLogRecordScanner.java | 109 +- .../common/table/log/HoodieLogFileReader.java | 106 +- .../common/table/log/HoodieLogFormat.java | 68 +- .../table/log/HoodieLogFormatReader.java | 24 +- ...rsion.java => HoodieLogFormatVersion.java} | 45 +- .../table/log/HoodieLogFormatWriter.java | 56 +- .../table/log/block/HoodieAvroDataBlock.java | 14 +- .../log/block/HoodieAvroDataBlockVersion.java | 37 + .../table/log/block/HoodieCommandBlock.java | 7 +- .../log/block/HoodieCommandBlockVersion.java | 28 + .../table/log/block/HoodieCorruptBlock.java | 6 +- .../table/log/block/HoodieDeleteBlock.java | 14 +- .../log/block/HoodieDeleteBlockVersion.java | 28 + .../table/log/block/HoodieLogBlock.java | 57 +- .../log/block/HoodieLogBlockVersion.java | 33 + .../table/log/block/LogBlockVersion.java | 79 -- .../table/timeline/HoodieActiveTimeline.java | 32 +- .../timeline/HoodieArchivedTimeline.java | 8 +- .../table/timeline/HoodieDefaultTimeline.java | 12 +- .../common/table/timeline/HoodieInstant.java | 31 +- .../table/view/HoodieTableFileSystemView.java | 358 +++--- .../com/uber/hoodie/common/util/FSUtils.java | 12 +- .../hoodie/common/util/HoodieAvroUtils.java | 4 +- .../uber/hoodie/common/util/ParquetUtils.java | 8 +- .../hoodie/common/util/ReflectionUtils.java | 3 +- .../hoodie/common/util/SpillableMapUtils.java | 58 +- .../common/util/collection/DiskBasedMap.java | 206 ++-- .../util/collection/ExternalSpillableMap.java | 62 +- .../util/collection/LazyFileIterable.java | 6 +- .../converter/HoodieRecordConverter.java | 5 - .../io/storage/SizeAwareDataOutputStream.java | 5 +- .../HoodieCorruptedDataException.java | 3 +- .../uber/hoodie/common/TestBloomFilter.java | 4 +- .../common/minicluster/HdfsTestService.java | 33 +- .../minicluster/ZookeeperTestService.java | 10 +- .../common/model/AvroBinaryTestPayload.java | 3 +- .../hoodie/common/model/HoodieTestUtils.java | 133 +-- .../common/model/TestHoodieWriteStat.java | 8 +- .../table/HoodieTableMetaClientTest.java | 37 +- .../common/table/log/HoodieLogFormatTest.java | 544 ++++----- .../string/HoodieActiveTimelineTest.java | 58 +- .../table/string/MockHoodieTimeline.java | 9 +- .../view/HoodieTableFileSystemViewTest.java | 251 ++-- .../hoodie/common/util/SchemaTestUtil.java | 50 +- .../common/util/SpillableMapTestUtils.java | 7 +- .../uber/hoodie/common/util/TestFSUtils.java | 9 +- .../hoodie/common/util/TestParquetUtils.java | 17 +- .../util/collection/TestDiskBasedMap.java | 21 +- .../collection/TestExternalSpillableMap.java | 45 +- .../uber/hoodie/hadoop/HoodieHiveUtil.java | 3 +- .../uber/hoodie/hadoop/HoodieInputFormat.java | 61 +- .../hadoop/HoodieROTablePathFilter.java | 32 +- .../realtime/HoodieRealtimeInputFormat.java | 38 +- .../realtime/HoodieRealtimeRecordReader.java | 70 +- .../uber/hoodie/hadoop/AnnotationTest.java | 1 - .../hoodie/hadoop/HoodieInputFormatTest.java | 63 +- .../hoodie/hadoop/InputFormatTestUtil.java | 31 +- .../hadoop/TestHoodieROTablePathFilter.java | 1 + .../HoodieRealtimeRecordReaderTest.java | 37 +- .../com/uber/hoodie/hive/HiveSyncConfig.java | 14 +- .../com/uber/hoodie/hive/HiveSyncTool.java | 35 +- .../uber/hoodie/hive/HoodieHiveClient.java | 147 ++- .../hoodie/hive/PartitionValueExtractor.java | 5 +- ...lashEncodedDayPartitionValueExtractor.java | 2 +- .../hoodie/hive/util/ColumnNameXLator.java | 4 +- .../com/uber/hoodie/hive/util/SchemaUtil.java | 88 +- .../uber/hoodie/hive/HiveSyncToolTest.java | 160 ++- .../java/com/uber/hoodie/hive/TestUtil.java | 82 +- .../hoodie/hive/util/HiveTestService.java | 42 +- .../java/com/uber/hoodie/BaseAvroPayload.java | 1 - .../java/com/uber/hoodie/DataSourceUtils.java | 47 +- .../uber/hoodie/HoodieDataSourceHelpers.java | 5 +- .../java/com/uber/hoodie/KeyGenerator.java | 3 +- .../OverwriteWithLatestAvroPayload.java | 3 +- .../com/uber/hoodie/AvroConversionUtils.scala | 9 +- .../src/test/java/DataSourceTestUtils.java | 10 +- hoodie-spark/src/test/java/HoodieJavaApp.java | 45 +- .../hoodie/utilities/HDFSParquetImporter.java | 102 +- .../utilities/HiveIncrementalPuller.java | 64 +- .../utilities/HoodieSnapshotCopier.java | 69 +- .../uber/hoodie/utilities/UtilHelpers.java | 9 +- .../deltastreamer/HoodieDeltaStreamer.java | 145 ++- .../keygen/TimestampBasedKeyGenerator.java | 15 +- .../schema/FilebasedSchemaProvider.java | 16 +- .../hoodie/utilities/sources/DFSSource.java | 24 +- .../utilities/sources/HiveIncrPullSource.java | 12 +- .../hoodie/utilities/sources/KafkaSource.java | 39 +- .../uber/hoodie/utilities/sources/Source.java | 3 +- .../utilities/TestHDFSParquetImporter.java | 24 +- .../utilities/TestHoodieSnapshotCopier.java | 6 +- pom.xml | 35 + style/checkstyle-suppressions.xml | 12 + style/checkstyle.xml | 246 ++++ style/eclipse-java-google-style.xml | 337 ++++++ style/intellij-java-google-style.xml | 598 ++++++++++ 200 files changed, 6247 insertions(+), 6013 deletions(-) rename hoodie-common/src/main/java/com/uber/hoodie/common/table/log/{LogFormatVersion.java => HoodieLogFormatVersion.java} (68%) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java create mode 100644 style/checkstyle-suppressions.xml create mode 100644 style/checkstyle.xml create mode 100644 style/eclipse-java-google-style.xml create mode 100644 style/intellij-java-google-style.xml diff --git a/docs/dev_setup.md b/docs/dev_setup.md index 2fc34677f881a..1bdeec793de2c 100644 --- a/docs/dev_setup.md +++ b/docs/dev_setup.md @@ -7,8 +7,7 @@ permalink: dev_setup.html ### Code Style - We have embraced the [Google Java code style](https://google.github.io/styleguide/javaguide.html). Please setup your IDE accordingly with style files from [here](https://github.com/google/styleguide/blob/gh-pages/intellij-java-google-style.xml) - Also recommend setting up the [Save Action Plugin](https://plugins.jetbrains.com/plugin/7642-save-actions) to auto format & organize imports on save. - - - + We have embraced the code style largely based on [google format](https://google.github.io/styleguide/javaguide.html). + Please setup your IDE with style files from [here](../style/) + We also recommend setting up the [Save Action Plugin](https://plugins.jetbrains.com/plugin/7642-save-actions) to auto format & organize imports on save. + The Maven Compilation life-cycle will fail if there are checkstyle violations. diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java index 2839cac99d5ac..937fd064bf066 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java @@ -35,10 +35,10 @@ public String getPrompt() { case DATASET: return "hoodie:" + tableName + "->"; case SYNC: - return "hoodie:" + tableName + " <==> " - + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; + return "hoodie:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; + default: + return "hoodie:" + tableName + "->"; } - return "hoodie:" + tableName + "->"; } return "hoodie->"; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java index d6a16891a3030..6bbe2e218fa89 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java @@ -24,22 +24,18 @@ @Component @Order(Ordered.HIGHEST_PRECEDENCE) -public class HoodieSplashScreen - extends DefaultBannerProvider { - - private static String screen = - "============================================" + OsUtils.LINE_SEPARATOR + - "* *" + OsUtils.LINE_SEPARATOR + - "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR + - "* | | | | | (_) *" + OsUtils.LINE_SEPARATOR + - "* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR + - "* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" + - OsUtils.LINE_SEPARATOR + - "* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR + - "* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" + - OsUtils.LINE_SEPARATOR + - "* *" + OsUtils.LINE_SEPARATOR + - "============================================" + OsUtils.LINE_SEPARATOR; +public class HoodieSplashScreen extends DefaultBannerProvider { + + private static String screen = "============================================" + OsUtils.LINE_SEPARATOR + + "* *" + OsUtils.LINE_SEPARATOR + + "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR + + "* | | | | | (_) *" + OsUtils.LINE_SEPARATOR + + "* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR + + "* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" + OsUtils.LINE_SEPARATOR + + "* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR + + "* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" + OsUtils.LINE_SEPARATOR + + "* *" + OsUtils.LINE_SEPARATOR + + "============================================" + OsUtils.LINE_SEPARATOR; public String getBanner() { return screen; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java index c0d7924ffa665..65f98f00f6025 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java @@ -22,8 +22,7 @@ public class Main { /** - * Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging - * inside an IDE + * Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE */ public static void main(String[] args) throws IOException { Bootstrap.main(args); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 50fb6a56561cb..99c79863b8f84 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -47,13 +47,11 @@ public boolean isShowArchivedCommitAvailable() { } @CliCommand(value = "show archived commits", help = "Read commits from archived files and show details") - public String showCommits( - @CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") - final Integer limit) throws IOException { + public String showCommits(@CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) + throws IOException { - System.out - .println("===============> Showing only " + limit + " archived commits <==============="); + System.out.println("===============> Showing only " + limit + " archived commits <==============="); String basePath = HoodieCLI.tableMetadata.getBasePath(); FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf) .globStatus(new Path(basePath + "/.hoodie/.commits_.archive*")); @@ -61,8 +59,7 @@ public String showCommits( int commits = 0; for (FileStatus fs : fsStatuses) { //read the archived file - HoodieLogFormat.Reader reader = HoodieLogFormat - .newReader(FSUtils.getFs(basePath, HoodieCLI.conf), + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf), new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema()); List readRecords = new ArrayList<>(); @@ -71,20 +68,19 @@ public String showCommits( HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); List records = blk.getRecords(); readRecords.addAll(records); - if(commits == limit) { + if (commits == limit) { break; } commits++; } - List readCommits = readRecords.stream().map(r -> (GenericRecord) r) - .map(r -> readCommit(r)).collect(Collectors.toList()); + List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r)) + .collect(Collectors.toList()); allCommits.addAll(readCommits); - if(commits == limit) { + if (commits == limit) { break; } } - return HoodiePrintHelper.print( - new String[]{"CommitTime", "CommitType", "CommitDetails"}, + return HoodiePrintHelper.print(new String[] {"CommitTime", "CommitType", "CommitDetails"}, allCommits.toArray(new String[allCommits.size()][])); } @@ -122,6 +118,8 @@ private String[] readCommit(GenericRecord record) { commitDetails.add(record.get("hoodieSavePointMetadata").toString()); break; } + default: + return commitDetails.toArray(new String[commitDetails.size()]); } } catch (Exception e) { e.printStackTrace(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java index 38638866b647d..aa1792ecb36d6 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.cli.commands; import com.uber.hoodie.avro.model.HoodieCleanMetadata; @@ -63,51 +64,46 @@ public String showCleans() throws IOException { Collections.reverse(cleans); for (int i = 0; i < cleans.size(); i++) { HoodieInstant clean = cleans.get(i); - HoodieCleanMetadata cleanMetadata = - AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); - rows[i] = new String[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), - String.valueOf(cleanMetadata.getTotalFilesDeleted()), - String.valueOf(cleanMetadata.getTimeTakenInMillis())}; + HoodieCleanMetadata cleanMetadata = AvroUtils + .deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); + rows[i] = new String[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), + String.valueOf(cleanMetadata.getTotalFilesDeleted()), String.valueOf(cleanMetadata.getTimeTakenInMillis())}; } - return HoodiePrintHelper.print( - new String[]{"CleanTime", "EarliestCommandRetained", "Total Files Deleted", - "Total Time Taken"}, rows); + return HoodiePrintHelper + .print(new String[] {"CleanTime", "EarliestCommandRetained", "Total Files Deleted", "Total Time Taken"}, + rows); } @CliCommand(value = "cleans refresh", help = "Refresh the commits") public String refreshCleans() throws IOException { - HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } @CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean") - public String showCleanPartitions( - @CliOption(key = {"clean"}, help = "clean to show") - final String commitTime) throws Exception { + public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to show") final String commitTime) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants(); - HoodieInstant cleanInstant = - new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime); + HoodieInstant cleanInstant = new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime); if (!timeline.containsInstant(cleanInstant)) { return "Clean " + commitTime + " not found in metadata " + timeline; } - HoodieCleanMetadata cleanMetadata = - AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get()); + HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata( + timeline.getInstantDetails(cleanInstant).get()); List rows = new ArrayList<>(); - for (Map.Entry entry : cleanMetadata - .getPartitionMetadata().entrySet()) { + for (Map.Entry entry : cleanMetadata.getPartitionMetadata().entrySet()) { String path = entry.getKey(); HoodieCleanPartitionMetadata stats = entry.getValue(); String policy = stats.getPolicy(); String totalSuccessDeletedFiles = String.valueOf(stats.getSuccessDeleteFiles().size()); String totalFailedDeletedFiles = String.valueOf(stats.getFailedDeleteFiles().size()); - rows.add(new String[]{path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles}); + rows.add(new String[] {path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles}); } return HoodiePrintHelper.print( - new String[]{"Partition Path", "Cleaning policy", "Total Files Successfully Deleted", + new String[] {"Partition Path", "Cleaning policy", "Total Files Successfully Deleted", "Total Failed Deletions"}, rows.toArray(new String[rows.size()][])); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index d7b6594d138c4..0f5f87b0966fb 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -64,21 +64,18 @@ public boolean isCommitShowAvailable() { } @CliCommand(value = "commits show", help = "Show the commits") - public String showCommits( - @CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") - final Integer limit) throws IOException { + public String showCommits(@CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) + throws IOException { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); List commits = timeline.getInstants().collect(Collectors.toList()); String[][] rows = new String[commits.size()][]; Collections.reverse(commits); for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get()); - rows[i] = new String[]{commit.getTimestamp(), + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get()); + rows[i] = new String[] {commit.getTimestamp(), NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()), String.valueOf(commitMetadata.fetchTotalFilesInsert()), String.valueOf(commitMetadata.fetchTotalFilesUpdated()), @@ -88,39 +85,32 @@ public String showCommits( String.valueOf(commitMetadata.fetchTotalWriteErrors())}; } return HoodiePrintHelper.print( - new String[]{"CommitTime", "Total Written (B)", "Total Files Added", - "Total Files Updated", "Total Partitions Written", "Total Records Written", - "Total Update Records Written", "Total Errors"}, rows); + new String[] {"CommitTime", "Total Written (B)", "Total Files Added", "Total Files Updated", + "Total Partitions Written", "Total Records Written", "Total Update Records Written", "Total Errors"}, rows); } @CliCommand(value = "commits refresh", help = "Refresh the commits") public String refreshCommits() throws IOException { - HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } @CliCommand(value = "commit rollback", help = "Rollback a commit") - public String rollbackCommit( - @CliOption(key = {"commit"}, help = "Commit to rollback") - final String commitTime, - @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") - final String sparkPropertiesPath) throws Exception { + public String rollbackCommit(@CliOption(key = {"commit"}, help = "Commit to rollback") final String commitTime, + @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), - commitTime, - HoodieCLI.tableMetadata.getBasePath()); + sparkLauncher + .addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), commitTime, HoodieCLI.tableMetadata.getBasePath()); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); @@ -133,23 +123,18 @@ public String rollbackCommit( } @CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit") - public String showCommitPartitions( - @CliOption(key = {"commit"}, help = "Commit to show") - final String commitTime) throws Exception { + public String showCommitPartitions(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } - HoodieCommitMetadata meta = - HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); List rows = new ArrayList(); - for (Map.Entry> entry : meta.getPartitionToWriteStats() - .entrySet()) { + for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); List stats = entry.getValue(); long totalFilesAdded = 0; @@ -169,50 +154,40 @@ public String showCommitPartitions( totalBytesWritten += stat.getTotalWriteBytes(); totalWriteErrors += stat.getTotalWriteErrors(); } - rows.add(new String[]{path, String.valueOf(totalFilesAdded), - String.valueOf(totalFilesUpdated), String.valueOf(totalRecordsInserted), - String.valueOf(totalRecordsUpdated), - NumericUtils.humanReadableByteCount(totalBytesWritten), - String.valueOf(totalWriteErrors)}); + rows.add(new String[] {path, String.valueOf(totalFilesAdded), String.valueOf(totalFilesUpdated), + String.valueOf(totalRecordsInserted), String.valueOf(totalRecordsUpdated), + NumericUtils.humanReadableByteCount(totalBytesWritten), String.valueOf(totalWriteErrors)}); } return HoodiePrintHelper.print( - new String[]{"Partition Path", "Total Files Added", "Total Files Updated", - "Total Records Inserted", "Total Records Updated", "Total Bytes Written", - "Total Errors"}, rows.toArray(new String[rows.size()][])); + new String[] {"Partition Path", "Total Files Added", "Total Files Updated", "Total Records Inserted", + "Total Records Updated", "Total Bytes Written", "Total Errors"}, rows.toArray(new String[rows.size()][])); } @CliCommand(value = "commit showfiles", help = "Show file level details of a commit") - public String showCommitFiles( - @CliOption(key = {"commit"}, help = "Commit to show") - final String commitTime) throws Exception { + public String showCommitFiles(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } - HoodieCommitMetadata meta = - HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); List rows = new ArrayList(); - for (Map.Entry> entry : meta.getPartitionToWriteStats() - .entrySet()) { + for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); List stats = entry.getValue(); for (HoodieWriteStat stat : stats) { - rows.add(new String[]{path, stat.getFileId(), stat.getPrevCommit(), - String.valueOf(stat.getNumUpdateWrites()), String.valueOf(stat.getNumWrites()), - String.valueOf(stat.getTotalWriteBytes()), + rows.add(new String[] {path, stat.getFileId(), stat.getPrevCommit(), String.valueOf(stat.getNumUpdateWrites()), + String.valueOf(stat.getNumWrites()), String.valueOf(stat.getTotalWriteBytes()), String.valueOf(stat.getTotalWriteErrors())}); } } return HoodiePrintHelper.print( - new String[]{"Partition Path", "File ID", "Previous Commit", "Total Records Updated", - "Total Records Written", "Total Bytes Written", "Total Errors"}, - rows.toArray(new String[rows.size()][])); + new String[] {"Partition Path", "File ID", "Previous Commit", "Total Records Updated", "Total Records Written", + "Total Bytes Written", "Total Errors"}, rows.toArray(new String[rows.size()][])); } @CliAvailabilityIndicator({"commits compare"}) @@ -221,38 +196,30 @@ public boolean isCompareCommitsAvailable() { } @CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset") - public String compareCommits( - @CliOption(key = {"path"}, help = "Path of the dataset to compare to") - final String path) throws Exception { + public String compareCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) + throws Exception { HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path); - HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTableMetaClient source = HoodieCLI.tableMetadata; - HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); String targetLatestCommit = - targetTimeline.getInstants().iterator().hasNext() ? "0" - : targetTimeline.lastInstant().get().getTimestamp(); + targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); String sourceLatestCommit = - sourceTimeline.getInstants().iterator().hasNext() ? "0" - : sourceTimeline.lastInstant().get().getTimestamp(); + sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp(); - if (sourceLatestCommit != null && - HoodieTimeline - .compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) { + if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, + HoodieTimeline.GREATER)) { // source is behind the target - List commitsToCatchup = - targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - return "Source " + source.getTableConfig().getTableName() + " is behind by " - + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; + List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) + .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size() + + " commits. Commits to catch up - " + commitsToCatchup; } else { - List commitsToCatchup = - sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - return "Source " + source.getTableConfig().getTableName() + " is ahead by " - + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; + List commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) + .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + return "Source " + source.getTableConfig().getTableName() + " is ahead by " + commitsToCatchup.size() + + " commits. Commits to catch up - " + commitsToCatchup; } } @@ -262,13 +229,12 @@ public boolean isSyncCommitsAvailable() { } @CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset") - public String syncCommits( - @CliOption(key = {"path"}, help = "Path of the dataset to compare to") - final String path) throws Exception { + public String syncCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) + throws Exception { HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieCLI.state = HoodieCLI.CLIState.SYNC; - return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() - + " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); + return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " and " + + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java index de59d06694d86..1bcdfa5bc7574 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -29,13 +29,12 @@ public class DatasetsCommand implements CommandMarker { @CliCommand(value = "connect", help = "Connect to a hoodie dataset") public String connect( - @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") - final String path) throws IOException { + @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path) + throws IOException { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path)); HoodieCLI.state = HoodieCLI.CLIState.DATASET; - return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() - + " loaded"; + return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java index c9d2f98b5dd41..c3298c9ae4999 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java @@ -37,44 +37,33 @@ public class HDFSParquetImportCommand implements CommandMarker { @CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset") public String convert( - @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") - final String srcPath, - @CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") - final String srcType, - @CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") - final String targetPath, - @CliOption(key = "tableName", mandatory = true, help = "Table name") - final String tableName, - @CliOption(key = "tableType", mandatory = true, help = "Table type") - final String tableType, - @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") - final String rowKeyField, - @CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name") - final String partitionPathField, - @CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert") - final String parallelism, - @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") - final String schemaFilePath, - @CliOption(key = "format", mandatory = true, help = "Format for the input data") - final String format, - @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") - final String sparkMemory, - @CliOption(key = "retry", mandatory = true, help = "Number of retries") - final String retry) - throws Exception { + @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath, + @CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") final String srcType, + @CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String + targetPath, + @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, + @CliOption(key = "tableType", mandatory = true, help = "Table type") final String tableType, + @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField, + @CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name") final String + partitionPathField, + @CliOption(key = { + "parallelism"}, mandatory = true, help = "Parallelism for hoodie insert") final String parallelism, + @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String + schemaFilePath, + @CliOption(key = "format", mandatory = true, help = "Format for the input data") final String format, + @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, + @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception { validate(format, srcType); boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); - String sparkPropertiesPath = Utils - .getDefaultPropertiesFile( - scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName, - tableType, rowKeyField, partitionPathField, parallelism, schemaFilePath, sparkMemory, - retry); + sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName, tableType, rowKeyField, + partitionPathField, parallelism, schemaFilePath, sparkMemory, retry); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 2de48500b6a00..9d9790bba8e74 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -64,25 +64,25 @@ public boolean isShowArchivedLogFileAvailable() { @CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files") public String showLogFileCommits( - @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") - final String logFilePathPattern) throws IOException { + @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") final + String logFilePathPattern) + throws IOException { FileSystem fs = HoodieCLI.tableMetadata.getFs(); List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) .map(status -> status.getPath().toString()).collect(Collectors.toList()); - Map, Map>, Integer>>> commitCountAndMetadata = Maps - .newHashMap(); + Map, Map>, Integer>>> + commitCountAndMetadata = Maps.newHashMap(); int totalEntries = 0; int numCorruptBlocks = 0; for (String logFilePath : logFilePaths) { - FileStatus[] fsStatus = fs.listStatus( - new Path(logFilePath)); - Schema writerSchema = new AvroSchemaConverter() - .convert(SchemaUtil - .readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath))); - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, - new HoodieLogFile(fsStatus[0].getPath()), writerSchema); + FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath)); + Schema writerSchema = new AvroSchemaConverter().convert( + SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath))); + HoodieLogFormat.Reader reader = HoodieLogFormat + .newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema); // read the avro blocks while (reader.hasNext()) { @@ -104,15 +104,14 @@ public String showLogFileCommits( } } if (commitCountAndMetadata.containsKey(instantTime)) { - commitCountAndMetadata.get(instantTime) - .add(new Tuple3<>(n.getBlockType(), - new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); + commitCountAndMetadata.get(instantTime).add( + new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); totalEntries++; } else { - List, Map>, Integer>> list - = new ArrayList<>(); - list.add(new Tuple3<>(n.getBlockType(), - new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); + List, Map>, + Integer>> list = new ArrayList<>(); + list.add( + new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); commitCountAndMetadata.put(instantTime, list); totalEntries++; } @@ -121,11 +120,12 @@ public String showLogFileCommits( String[][] rows = new String[totalEntries + 1][]; int i = 0; ObjectMapper objectMapper = new ObjectMapper(); - for (Map.Entry, Map>, Integer>>> entry : commitCountAndMetadata - .entrySet()) { + for (Map.Entry, Map>, Integer>>> entry + : commitCountAndMetadata.entrySet()) { String instantTime = entry.getKey().toString(); - for (Tuple3, Map>, Integer> tuple3 : entry - .getValue()) { + for (Tuple3, + Map>, Integer> tuple3 : entry.getValue()) { String[] output = new String[5]; output[0] = instantTime; output[1] = String.valueOf(tuple3._3()); @@ -136,24 +136,21 @@ public String showLogFileCommits( i++; } } - return HoodiePrintHelper.print( - new String[]{"InstantTime", "RecordCount", "BlockType", "HeaderMetadata", "FooterMetadata"}, - rows); + return HoodiePrintHelper + .print(new String[] {"InstantTime", "RecordCount", "BlockType", "HeaderMetadata", "FooterMetadata"}, + rows); } @CliCommand(value = "show logfile records", help = "Read records from log files") - public String showLogFileRecords( - @CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") - final Integer limit, + public String showLogFileRecords(@CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit, @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified paths for the log files") final String logFilePathPattern, @CliOption(key = "mergeRecords", mandatory = false, help = "If the records in the log files should be merged", - unspecifiedDefaultValue = "false") - final Boolean shouldMerge) throws IOException { + unspecifiedDefaultValue = "false") final Boolean shouldMerge) + throws IOException { - System.out - .println("===============> Showing only " + limit + " records <==============="); + System.out.println("===============> Showing only " + limit + " records <==============="); FileSystem fs = HoodieCLI.tableMetadata.getFs(); List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) @@ -162,9 +159,8 @@ public String showLogFileRecords( // TODO : readerSchema can change across blocks/log files, fix this inside Scanner AvroSchemaConverter converter = new AvroSchemaConverter(); // get schema from last log file - Schema readerSchema = converter - .convert(SchemaUtil - .readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))); + Schema readerSchema = converter.convert( + SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))); List allRecords = new ArrayList<>(); @@ -186,11 +182,10 @@ public String showLogFileRecords( } } else { for (String logFile : logFilePaths) { - Schema writerSchema = new AvroSchemaConverter() - .convert(SchemaUtil - .readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile))); - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, - new HoodieLogFile(new Path(logFile)), writerSchema); + Schema writerSchema = new AvroSchemaConverter().convert( + SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile))); + HoodieLogFormat.Reader reader = HoodieLogFormat + .newReader(fs, new HoodieLogFile(new Path(logFile)), writerSchema); // read the avro blocks while (reader.hasNext()) { HoodieLogBlock n = reader.next(); @@ -216,7 +211,6 @@ public String showLogFileRecords( rows[i] = data; i++; } - return HoodiePrintHelper.print( - new String[]{"Records"}, rows); + return HoodiePrintHelper.print(new String[] {"Records"}, rows); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java index dd9560a4a0f8e..0640c9416d9d4 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java @@ -40,26 +40,22 @@ public boolean isSyncVerificationAvailable() { @CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records") public String validateSync( - @CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") - final String mode, + @CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") final String mode, + @CliOption(key = {"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") final String srcDb, @CliOption(key = { - "sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") - final String srcDb, - @CliOption(key = { - "targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") - final String tgtDb, + "targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") final String tgtDb, @CliOption(key = { "partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate") final int partitionCount, @CliOption(key = { - "hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") - final String hiveServerUrl, + "hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") final String hiveServerUrl, @CliOption(key = { - "hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") - final String hiveUser, + "hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") final + String hiveUser, @CliOption(key = { - "hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") - final String hivePass) throws Exception { + "hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") final + String hivePass) + throws Exception { HoodieTableMetaClient target = HoodieCLI.syncTableMetadata; HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline(); HoodieTableMetaClient source = HoodieCLI.tableMetadata; @@ -70,52 +66,42 @@ public String validateSync( sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass); targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass); } else if ("latestPartitions".equals(mode)) { - sourceCount = HiveUtil - .countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass); - targetCount = HiveUtil - .countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass); + sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass); + targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass); } String targetLatestCommit = - targetTimeline.getInstants().iterator().hasNext() ? "0" - : targetTimeline.lastInstant().get().getTimestamp(); + targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); String sourceLatestCommit = - sourceTimeline.getInstants().iterator().hasNext() ? "0" - : sourceTimeline.lastInstant().get().getTimestamp(); + sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp(); - if (sourceLatestCommit != null && HoodieTimeline - .compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) { + if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, + HoodieTimeline.GREATER)) { // source is behind the target - List commitsToCatchup = - targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants() - .collect(Collectors.toList()); + List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) + .getInstants().collect(Collectors.toList()); if (commitsToCatchup.isEmpty()) { - return "Count difference now is (count(" + target.getTableConfig().getTableName() - + ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount - - sourceCount); + return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count(" + + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount); } else { long newInserts = CommitUtil.countNewRecords(target, - commitsToCatchup.stream().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList())); - return "Count difference now is (count(" + target.getTableConfig().getTableName() - + ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount - - sourceCount) + ". Catch up count is " + newInserts; + commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList())); + return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count(" + + source.getTableConfig().getTableName() + + ") == " + (targetCount - sourceCount) + ". Catch up count is " + newInserts; } } else { - List commitsToCatchup = - sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants() - .collect(Collectors.toList()); + List commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) + .getInstants().collect(Collectors.toList()); if (commitsToCatchup.isEmpty()) { - return "Count difference now is (count(" + source.getTableConfig().getTableName() - + ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount - - targetCount); + return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count(" + + target.getTableConfig().getTableName() + ") == " + (sourceCount - targetCount); } else { long newInserts = CommitUtil.countNewRecords(source, - commitsToCatchup.stream().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList())); - return "Count difference now is (count(" + source.getTableConfig().getTableName() - + ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount - - targetCount) + ". Catch up count is " + newInserts; + commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList())); + return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count(" + + target.getTableConfig().getTableName() + + ") == " + (sourceCount - targetCount) + ". Catch up count is " + newInserts; } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java index 1db6075e15e23..bf6c828885b6f 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java @@ -45,20 +45,20 @@ public boolean isRepairAddPartitionMetaAvailable() { return HoodieCLI.tableMetadata != null; } - @CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with") - public String deduplicate( + @CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce " + + "repaired files to replace with") + public String deduplicate(@CliOption(key = { + "duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true) final String + duplicatedPartitionPath, @CliOption(key = { - "duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true) - final String duplicatedPartitionPath, + "repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true) final String + repairedOutputPath, @CliOption(key = { - "repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true) - final String repairedOutputPath, - @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path", mandatory = true) - final String sparkPropertiesPath) throws Exception { + "sparkProperties"}, help = "Spark Properites File Path", mandatory = true) final String sparkPropertiesPath) + throws Exception { SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher - .addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, - repairedOutputPath, HoodieCLI.tableMetadata.getBasePath()); + sparkLauncher.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, repairedOutputPath, + HoodieCLI.tableMetadata.getBasePath()); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); @@ -71,14 +71,12 @@ public String deduplicate( @CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present") - public String addPartitionMeta( - @CliOption(key = {"dryrun"}, - help = "Should we actually add or just print what would be done", - unspecifiedDefaultValue = "true") + public String addPartitionMeta(@CliOption(key = { + "dryrun"}, help = "Should we actually add or just print what would be done", unspecifiedDefaultValue = "true") final boolean dryRun) throws IOException { - String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline() - .lastInstant().get().getTimestamp(); + String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get() + .getTimestamp(); List partitionPaths = FSUtils.getAllFoldersThreeLevelsDown(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); Path basePath = new Path(HoodieCLI.tableMetadata.getBasePath()); @@ -94,10 +92,7 @@ public String addPartitionMeta( if (!HoodiePartitionMetadata.hasPartitionMetadata(HoodieCLI.fs, partitionPath)) { row[1] = "No"; if (!dryRun) { - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata( - HoodieCLI.fs, - latestCommit, - basePath, + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath); partitionMetadata.trySave(0); } @@ -105,7 +100,6 @@ public String addPartitionMeta( rows[ind++] = row; } - return HoodiePrintHelper.print( - new String[]{"Partition Path", "Metadata Present?", "Action"}, rows); + return HoodiePrintHelper.print(new String[] {"Partition Path", "Metadata Present?", "Action"}, rows); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java index db267d6a656e5..0bbfa3d1a28ac 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.cli.commands; import com.uber.hoodie.HoodieWriteClient; @@ -60,8 +61,8 @@ public boolean isCreateSavepointAvailable() { @CliAvailabilityIndicator({"savepoint rollback"}) public boolean isRollbackToSavepointAvailable() { - return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline() - .getSavePointTimeline().filterCompletedInstants().empty(); + return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline() + .filterCompletedInstants().empty(); } @CliCommand(value = "savepoints show", help = "Show the savepoints") @@ -73,23 +74,19 @@ public String showSavepoints() throws IOException { Collections.reverse(commits); for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); - rows[i] = new String[]{commit.getTimestamp()}; + rows[i] = new String[] {commit.getTimestamp()}; } - return HoodiePrintHelper.print(new String[]{"SavepointTime"}, rows); + return HoodiePrintHelper.print(new String[] {"SavepointTime"}, rows); } @CliCommand(value = "savepoint create", help = "Savepoint a commit") - public String savepoint( - @CliOption(key = {"commit"}, help = "Commit to savepoint") - final String commitTime, - @CliOption(key = {"user"}, help = "User who is creating the savepoint") - final String user, - @CliOption(key = {"comments"}, help = "Comments for creating the savepoint") - final String comments) throws Exception { + public String savepoint(@CliOption(key = {"commit"}, help = "Commit to savepoint") final String commitTime, + @CliOption(key = {"user"}, help = "User who is creating the savepoint") final String user, + @CliOption(key = {"comments"}, help = "Comments for creating the savepoint") final String comments) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants(); - HoodieInstant - commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; @@ -106,22 +103,19 @@ public String savepoint( @CliCommand(value = "savepoint rollback", help = "Savepoint a commit") public String rollbackToSavepoint( - @CliOption(key = {"savepoint"}, help = "Savepoint to rollback") - final String commitTime, - @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") - final String sparkPropertiesPath) throws Exception { + @CliOption(key = {"savepoint"}, help = "Savepoint to rollback") final String commitTime, + @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants(); - HoodieInstant - commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(), - commitTime, + sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(), commitTime, HoodieCLI.tableMetadata.getBasePath()); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); @@ -137,18 +131,14 @@ public String rollbackToSavepoint( @CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") public String refreshMetaClient() throws IOException { - HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } - private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) - throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); return new HoodieWriteClient(jsc, config, false); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index 4f8f2f856b306..32e2a0091e683 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -30,18 +30,14 @@ public class SparkMain { - protected final static Logger LOG = Logger.getLogger(SparkMain.class); + protected static final Logger LOG = Logger.getLogger(SparkMain.class); /** * Commands */ enum SparkCommand { - ROLLBACK, - DEDUPLICATE, - ROLLBACK_TO_SAVEPOINT, - SAVEPOINT, - IMPORT + ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT } public static void main(String[] args) throws Exception { @@ -67,18 +63,19 @@ public static void main(String[] args) throws Exception { break; case IMPORT: assert (args.length == 11); - returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6], - Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], - Integer.parseInt(args[10])); + returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6], Integer.parseInt(args[7]), + args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10])); + break; + default: break; } System.exit(returnCode); } - private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath, - String tableName, String tableType, String rowKey, String partitionKey, int parallelism, - String schemaFile, String sparkMaster, String sparkMemory, int retry) throws Exception { + private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath, String tableName, + String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster, + String sparkMemory, int retry) throws Exception { HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); cfg.srcPath = srcPath; cfg.targetPath = targetPath; @@ -92,19 +89,15 @@ private static int dataImport(JavaSparkContext jsc, String srcPath, String targe return new HDFSParquetImporter(cfg).dataImport(jsc, retry); } - private static int deduplicatePartitionPath(JavaSparkContext jsc, - String duplicatedPartitionPath, - String repairedOutputPath, - String basePath) - throws Exception { - DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, - new SQLContext(jsc), FSUtils.getFs(basePath, jsc.hadoopConfiguration())); + private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath, + String repairedOutputPath, String basePath) throws Exception { + DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), + FSUtils.getFs(basePath, jsc.hadoopConfiguration())); job.fixDuplicates(true); return 0; } - private static int rollback(JavaSparkContext jsc, String commitTime, String basePath) - throws Exception { + private static int rollback(JavaSparkContext jsc, String commitTime, String basePath) throws Exception { HoodieWriteClient client = createHoodieClient(jsc, basePath); if (client.rollback(commitTime)) { LOG.info(String.format("The commit \"%s\" rolled back.", commitTime)); @@ -115,9 +108,7 @@ private static int rollback(JavaSparkContext jsc, String commitTime, String base } } - private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, - String basePath) - throws Exception { + private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception { HoodieWriteClient client = createHoodieClient(jsc, basePath); if (client.rollbackToSavepoint(savepointTime)) { LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime)); @@ -128,12 +119,9 @@ private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTim } } - private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) - throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); return new HoodieWriteClient(jsc, config); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java index cb61eef0be9b8..2cdf826e8f408 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java @@ -16,7 +16,6 @@ package com.uber.hoodie.cli.commands; - import com.codahale.metrics.Histogram; import com.codahale.metrics.Snapshot; import com.codahale.metrics.UniformReservoir; @@ -44,12 +43,15 @@ @Component public class StatsCommand implements CommandMarker { + private static final int MAX_FILES = 1000000; + @CliAvailabilityIndicator({"stats wa"}) public boolean isWriteAmpAvailable() { return HoodieCLI.tableMetadata != null; } - @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many records were actually written") + @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many " + + "records were actually written") public String writeAmplificationStats() throws IOException { long totalRecordsUpserted = 0; long totalRecordsWritten = 0; @@ -60,18 +62,13 @@ public String writeAmplificationStats() throws IOException { String[][] rows = new String[new Long(timeline.countInstants()).intValue() + 1][]; int i = 0; DecimalFormat df = new DecimalFormat("#.00"); - for (HoodieInstant commitTime : timeline.getInstants().collect( - Collectors.toList())) { + for (HoodieInstant commitTime : timeline.getInstants().collect(Collectors.toList())) { String waf = "0"; - HoodieCommitMetadata commit = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(commitTime).get()); + HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get()); if (commit.fetchTotalUpdateRecordsWritten() > 0) { - waf = df.format( - (float) commit.fetchTotalRecordsWritten() / commit - .fetchTotalUpdateRecordsWritten()); + waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten()); } - rows[i++] = new String[]{commitTime.getTimestamp(), - String.valueOf(commit.fetchTotalUpdateRecordsWritten()), + rows[i++] = new String[] {commitTime.getTimestamp(), String.valueOf(commit.fetchTotalUpdateRecordsWritten()), String.valueOf(commit.fetchTotalRecordsWritten()), waf}; totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten(); totalRecordsWritten += commit.fetchTotalRecordsWritten(); @@ -80,43 +77,32 @@ public String writeAmplificationStats() throws IOException { if (totalRecordsUpserted > 0) { waf = df.format((float) totalRecordsWritten / totalRecordsUpserted); } - rows[i] = new String[]{"Total", String.valueOf(totalRecordsUpserted), - String.valueOf(totalRecordsWritten), waf}; - return HoodiePrintHelper.print( - new String[]{"CommitTime", "Total Upserted", "Total Written", - "Write Amplifiation Factor"}, rows); + rows[i] = new String[] {"Total", String.valueOf(totalRecordsUpserted), String.valueOf(totalRecordsWritten), waf}; + return HoodiePrintHelper + .print(new String[] {"CommitTime", "Total Upserted", "Total Written", "Write Amplifiation Factor"}, + rows); } private String[] printFileSizeHistogram(String commitTime, Snapshot s) { - return new String[]{ - commitTime, - NumericUtils.humanReadableByteCount(s.getMin()), - NumericUtils.humanReadableByteCount(s.getValue(0.1)), - NumericUtils.humanReadableByteCount(s.getMedian()), - NumericUtils.humanReadableByteCount(s.getMean()), - NumericUtils.humanReadableByteCount(s.get95thPercentile()), - NumericUtils.humanReadableByteCount(s.getMax()), - String.valueOf(s.size()), - NumericUtils.humanReadableByteCount(s.getStdDev()) - }; + return new String[] {commitTime, NumericUtils.humanReadableByteCount(s.getMin()), + NumericUtils.humanReadableByteCount(s.getValue(0.1)), NumericUtils.humanReadableByteCount(s.getMedian()), + NumericUtils.humanReadableByteCount(s.getMean()), NumericUtils.humanReadableByteCount(s.get95thPercentile()), + NumericUtils.humanReadableByteCount(s.getMax()), String.valueOf(s.size()), + NumericUtils.humanReadableByteCount(s.getStdDev())}; } @CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files") - public String fileSizeStats( - @CliOption(key = { - "partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") - final String globRegex) throws IOException { + public String fileSizeStats(@CliOption(key = { + "partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") final + String globRegex) throws IOException { FileSystem fs = HoodieCLI.fs; - String globPath = String.format("%s/%s/*", - HoodieCLI.tableMetadata.getBasePath(), - globRegex); + String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex); FileStatus[] statuses = fs.globStatus(new Path(globPath)); // max, min, #small files < 10MB, 50th, avg, 95th - final int MAX_FILES = 1000000; Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES)); HashMap commitHistoMap = new HashMap(); for (FileStatus fileStatus : statuses) { @@ -138,8 +124,8 @@ public String fileSizeStats( Snapshot s = globalHistogram.getSnapshot(); rows[ind++] = printFileSizeHistogram("ALL", s); - return HoodiePrintHelper.print( - new String[]{"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", - "StdDev"}, rows); + return HoodiePrintHelper + .print(new String[] {"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", "StdDev"}, + rows); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java index 3733a8c0c66f3..676b6ecc60bd2 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java @@ -25,9 +25,7 @@ public class UtilsCommand implements CommandMarker { @CliCommand(value = "utils loadClass", help = "Load a class") - public String loadClass( - @CliOption(key = {"class"}, help = "Check mode") final String clazz - ) throws Exception { + public String loadClass(@CliOption(key = {"class"}, help = "Check mode") final String clazz) throws Exception { Class klass = Class.forName(clazz); return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm(); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java index 71ed5aca6b141..55c2471a9ae76 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java @@ -25,15 +25,12 @@ public class CommitUtil { - public static long countNewRecords(HoodieTableMetaClient target, List commitsToCatchup) - throws IOException { + public static long countNewRecords(HoodieTableMetaClient target, List commitsToCatchup) throws IOException { long totalNew = 0; - HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants(); for (String commit : commitsToCatchup) { - HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline - .getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)) - .get()); + HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get()); totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten(); } return totalNew; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java index 52ec668c2eafb..7e396434996d5 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java @@ -27,7 +27,7 @@ public class HiveUtil { - private static String driverName = "org.apache.hive.jdbc.HiveDriver"; + private static final String driverName = "org.apache.hive.jdbc.HiveDriver"; static { try { @@ -39,8 +39,7 @@ public class HiveUtil { private static Connection connection; - private static Connection getConnection(String jdbcUrl, String user, String pass) - throws SQLException { + private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException { DataSource ds = getDatasource(jdbcUrl, user, pass); return ds.getConnection(); } @@ -54,8 +53,8 @@ private static DataSource getDatasource(String jdbcUrl, String user, String pass return ds; } - public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, - String user, String pass) throws SQLException { + public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass) + throws SQLException { Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); ResultSet rs = null; Statement stmt = conn.createStatement(); @@ -64,15 +63,13 @@ public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, St stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); stmt.execute("set hive.stats.autogather=false"); rs = stmt.executeQuery( - "select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source - .getTableConfig() - .getTableName()); + "select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + + source.getTableConfig().getTableName()); long count = -1; if (rs.next()) { count = rs.getLong("cnt"); } - System.out - .println("Total records in " + source.getTableConfig().getTableName() + " is " + count); + System.out.println("Total records in " + source.getTableConfig().getTableName() + " is " + count); return count; } finally { if (rs != null) { @@ -84,22 +81,19 @@ public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, St } } - public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, - int partitions, String user, String pass) throws SQLException { + public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, int partitions, + String user, String pass) throws SQLException { DateTime dateTime = DateTime.now(); - String endDateStr = - dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + - String.format("%02d", dateTime.getDayOfMonth()); + String endDateStr = dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + + String.format("%02d", dateTime.getDayOfMonth()); dateTime = dateTime.minusDays(partitions); - String startDateStr = - dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + - String.format("%02d", dateTime.getDayOfMonth()); + String startDateStr = dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + + String.format("%02d", dateTime.getDayOfMonth()); System.out.println("Start date " + startDateStr + " and end date " + endDateStr); return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass); } - private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, - String startDateStr, + private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, String startDateStr, String endDateStr, String user, String pass) throws SQLException { Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); ResultSet rs = null; @@ -109,9 +103,8 @@ private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, S stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); stmt.execute("set hive.stats.autogather=false"); rs = stmt.executeQuery( - "select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig() - .getTableName() + " where datestr>'" + startDateStr + "' and datestr<='" - + endDateStr + "'"); + "select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig().getTableName() + + " where datestr>'" + startDateStr + "' and datestr<='" + endDateStr + "'"); if (rs.next()) { return rs.getLong("cnt"); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java index 4d926cea35d64..894bcfae44354 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java @@ -24,7 +24,7 @@ public class InputStreamConsumer extends Thread { - protected final static Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName()); + protected static final Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName()); private InputStream is; public InputStreamConsumer(InputStream is) { diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java index d85626d358bf6..74fce58f56836 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java @@ -35,13 +35,10 @@ public class SparkUtil { * TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro */ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException { - String currentJar = new File( - SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath()) + String currentJar = new File(SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath()) .getAbsolutePath(); - SparkLauncher sparkLauncher = - new SparkLauncher().setAppResource(currentJar) - .setMainClass(SparkMain.class.getName()) - .setPropertiesFile(propertiesFile); + SparkLauncher sparkLauncher = new SparkLauncher().setAppResource(currentJar).setMainClass(SparkMain.class.getName()) + .setPropertiesFile(propertiesFile); File libDirectory = new File(new File(currentJar).getParent(), "lib"); for (String library : libDirectory.list()) { sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath()); @@ -60,8 +57,7 @@ public static JavaSparkContext initJavaSparkConf(String name) { // Configure hadoop conf 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", - "org.apache.hadoop.io.compress.GzipCodec"); + sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec"); sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); sparkConf = HoodieWriteClient.registerClasses(sparkConf); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 82bb9218f89b3..9a2bed3abfe10 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -50,17 +50,17 @@ */ public class HoodieReadClient implements Serializable { - private static Logger logger = LogManager.getLogger(HoodieReadClient.class); + private static final Logger logger = LogManager.getLogger(HoodieReadClient.class); - private transient final JavaSparkContext jsc; + private final transient JavaSparkContext jsc; - private transient final FileSystem fs; + private final transient FileSystem fs; /** * TODO: We need to persist the index type into hoodie.properties and be able to access the index * just with a simple basepath pointing to the dataset. Until, then just always assume a * BloomIndex */ - private transient final HoodieIndex index; + private final transient HoodieIndex index; private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; private transient Optional sqlContextOpt; @@ -69,8 +69,7 @@ public class HoodieReadClient implements Serializ * @param basePath path to Hoodie dataset */ public HoodieReadClient(JavaSparkContext jsc, String basePath) { - this(jsc, HoodieWriteConfig.newBuilder() - .withPath(basePath) + this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath) // by default we use HoodieBloomIndex .withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) @@ -78,7 +77,6 @@ public HoodieReadClient(JavaSparkContext jsc, String basePath) { } /** - * * @param jsc * @param basePath * @param sqlContext @@ -96,9 +94,9 @@ public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { this.jsc = jsc; this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); // Create a Hoodie table which encapsulated the commits and files visible - this.hoodieTable = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), - clientConfig); + this.hoodieTable = HoodieTable + .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), + clientConfig); this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); this.index = HoodieIndex.createIndex(clientConfig, jsc); this.sqlContextOpt = Optional.absent(); @@ -126,33 +124,27 @@ private void assertSqlContext() { * * @return a dataframe */ - public Dataset read(JavaRDD hoodieKeys, int parallelism) - throws Exception { + public Dataset read(JavaRDD hoodieKeys, int parallelism) throws Exception { assertSqlContext(); - JavaPairRDD> keyToFileRDD = - index.fetchRecordLocation(hoodieKeys, hoodieTable); - List paths = keyToFileRDD - .filter(keyFileTuple -> keyFileTuple._2().isPresent()) - .map(keyFileTuple -> keyFileTuple._2().get()) - .collect(); + JavaPairRDD> keyToFileRDD = index + .fetchRecordLocation(hoodieKeys, hoodieTable); + List paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) + .map(keyFileTuple -> keyFileTuple._2().get()).collect(); // record locations might be same for multiple keys, so need a unique list Set uniquePaths = new HashSet<>(paths); Dataset originalDF = sqlContextOpt.get().read() .parquet(uniquePaths.toArray(new String[uniquePaths.size()])); StructType schema = originalDF.schema(); - JavaPairRDD keyRowRDD = originalDF.javaRDD() - .mapToPair(row -> { - HoodieKey key = new HoodieKey( - row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), - row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); - return new Tuple2<>(key, row); - }); + JavaPairRDD keyRowRDD = originalDF.javaRDD().mapToPair(row -> { + HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), + row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); + return new Tuple2<>(key, row); + }); // Now, we need to further filter out, for only rows that match the supplied hoodie keys - JavaRDD rowRDD = keyRowRDD.join(keyToFileRDD, parallelism) - .map(tuple -> tuple._2()._1()); + JavaRDD rowRDD = keyRowRDD.join(keyToFileRDD, parallelism).map(tuple -> tuple._2()._1()); return sqlContextOpt.get().createDataFrame(rowRDD, schema); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index bb5613f2a61b9..93dd4c2924cf4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -81,18 +81,18 @@ /** * Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient * mutations on a HDFS dataset [upsert()] - * + *

* Note that, at any given time, there can only be one Spark job performing these operatons on a * Hoodie dataset. */ public class HoodieWriteClient implements Serializable { private static Logger logger = LogManager.getLogger(HoodieWriteClient.class); - private transient final FileSystem fs; - private transient final JavaSparkContext jsc; + private final transient FileSystem fs; + private final transient JavaSparkContext jsc; private final HoodieWriteConfig config; - private transient final HoodieMetrics metrics; - private transient final HoodieIndex index; + private final transient HoodieMetrics metrics; + private final transient HoodieIndex index; private transient Timer.Context writeContext = null; /** @@ -100,8 +100,7 @@ public class HoodieWriteClient implements Seriali * @param clientConfig * @throws Exception */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) - throws Exception { + public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception { this(jsc, clientConfig, false); } @@ -129,6 +128,12 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, } } + public static SparkConf registerClasses(SparkConf conf) { + conf.registerKryoClasses( + new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); + return conf; + } + /** * Filter out HoodieRecords that already exists in the output folder. This is useful in * deduplication. @@ -139,8 +144,7 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, public JavaRDD> filterExists(JavaRDD> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); @@ -153,9 +157,8 @@ public JavaRDD upsert(JavaRDD> records, final Strin HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeUpsert(), records, - config.getUpsertShuffleParallelism()); + JavaRDD> dedupedRecords = combineOnCondition( + config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism()); // perform index loop up to get existing location of records JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, table); @@ -170,12 +173,12 @@ public JavaRDD upsert(JavaRDD> records, final Strin /** * Upserts the given prepared records into the Hoodie table, at the supplied commitTime. - * + *

* This implementation requires that the input records are already tagged, and de-duped if * needed. * * @param preppedRecords Prepared HoodieRecords to upsert - * @param commitTime Commit Time handle + * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, @@ -187,19 +190,19 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe if (e instanceof HoodieUpsertException) { throw (HoodieUpsertException) e; } - throw new HoodieUpsertException("Failed to upsert prepared records for commit time " + - commitTime, e); + throw new HoodieUpsertException( + "Failed to upsert prepared records for commit time " + commitTime, e); } } /** * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal * writes. - * + *

* This implementation skips the index check and is able to leverage benefits such as small file * handling/blocking alignment, as with upsert(), by profiling the workload * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ @@ -207,9 +210,8 @@ public JavaRDD insert(JavaRDD> records, final Strin HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeInsert(), records, - config.getInsertShuffleParallelism()); + JavaRDD> dedupedRecords = combineOnCondition( + config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism()); return upsertRecordsInternal(dedupedRecords, commitTime, table, false); } catch (Throwable e) { @@ -222,13 +224,13 @@ public JavaRDD insert(JavaRDD> records, final Strin /** * Inserts the given prepared records into the Hoodie table, at the supplied commitTime. - * + *

* This implementation skips the index check, skips de-duping and is able to leverage benefits * such as small file handling/blocking alignment, as with insert(), by profiling the workload. * The prepared HoodieRecords should be de-duped if needed. * * @param preppedRecords HoodieRecords to insert - * @param commitTime Commit Time handle + * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, @@ -240,8 +242,8 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe if (e instanceof HoodieInsertException) { throw e; } - throw new HoodieInsertException("Failed to insert prepared records for commit time " + - commitTime, e); + throw new HoodieInsertException( + "Failed to insert prepared records for commit time " + commitTime, e); } } @@ -249,12 +251,12 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to * Hoodie). - * + *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)} * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ @@ -267,36 +269,33 @@ public JavaRDD bulkInsert(JavaRDD> records, * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to * Hoodie). - * + *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own * partitioner. If specified then it will be used for repartitioning records. See {@link * UserDefinedBulkInsertPartitioner}. * - * @param records HoodieRecords to insert - * @param commitTime Commit Time handle + * @param records HoodieRecords to insert + * @param commitTime Commit Time handle * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. + * before they are inserted into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ - public JavaRDD bulkInsert(JavaRDD> records, - final String commitTime, + public JavaRDD bulkInsert(JavaRDD> records, final String commitTime, Option bulkInsertPartitioner) { HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeInsert(), records, - config.getInsertShuffleParallelism()); + JavaRDD> dedupedRecords = combineOnCondition( + config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism()); return bulkInsertInternal(dedupedRecords, commitTime, table, bulkInsertPartitioner); } catch (Throwable e) { if (e instanceof HoodieInsertException) { throw e; } - throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, - e); + throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e); } } @@ -304,22 +303,21 @@ public JavaRDD bulkInsert(JavaRDD> records, * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to * Hoodie). The input records should contain no duplicates if needed. - * + *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own * partitioner. If specified then it will be used for repartitioning records. See {@link * UserDefinedBulkInsertPartitioner}. * - * @param preppedRecords HoodieRecords to insert - * @param commitTime Commit Time handle + * @param preppedRecords HoodieRecords to insert + * @param commitTime Commit Time handle * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. + * before they are inserted into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, - final String commitTime, - Option bulkInsertPartitioner) { + final String commitTime, Option bulkInsertPartitioner) { HoodieTable table = getTableAndInitCtx(); try { return bulkInsertInternal(preppedRecords, commitTime, table, bulkInsertPartitioner); @@ -327,35 +325,29 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr if (e instanceof HoodieInsertException) { throw e; } - throw new HoodieInsertException("Failed to bulk insert prepared records for commit time " + - commitTime, e); + throw new HoodieInsertException( + "Failed to bulk insert prepared records for commit time " + commitTime, e); } } - private JavaRDD bulkInsertInternal( - JavaRDD> dedupedRecords, - String commitTime, - HoodieTable table, + private JavaRDD bulkInsertInternal(JavaRDD> dedupedRecords, + String commitTime, HoodieTable table, Option bulkInsertPartitioner) { final JavaRDD> repartitionedRecords; if (bulkInsertPartitioner.isDefined()) { - repartitionedRecords = - bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, - config.getBulkInsertShuffleParallelism()); + repartitionedRecords = bulkInsertPartitioner.get() + .repartitionRecords(dedupedRecords, config.getBulkInsertShuffleParallelism()); } else { // Now, sort the records and line them up nicely for loading. - repartitionedRecords = dedupedRecords - .sortBy(record -> { - // Let's use "partitionPath + key" as the sort key. Spark, will ensure - // the records split evenly across RDD partitions, such that small partitions fit - // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions - return String - .format("%s+%s", record.getPartitionPath(), record.getRecordKey()); - }, true, config.getBulkInsertShuffleParallelism()); + repartitionedRecords = dedupedRecords.sortBy(record -> { + // Let's use "partitionPath + key" as the sort key. Spark, will ensure + // the records split evenly across RDD partitions, such that small partitions fit + // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions + return String.format("%s+%s", record.getPartitionPath(), record.getRecordKey()); + }, true, config.getBulkInsertShuffleParallelism()); } JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), - true) + .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), true) .flatMap(writeStatuses -> writeStatuses.iterator()); return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); @@ -375,8 +367,7 @@ private void commitOnAutoCommit(String commitTime, JavaRDD resultRD } private JavaRDD> combineOnCondition(boolean condition, - JavaRDD> records, - int parallelism) { + JavaRDD> records, int parallelism) { if (condition) { return deduplicateRecords(records, parallelism); } @@ -390,8 +381,8 @@ private JavaRDD> combineOnCondition(boolean condition, * files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata * file instead of using HoodieCommitMetadata */ - private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, - HoodieTable table, String commitTime) throws HoodieCommitException { + private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable table, + String commitTime) throws HoodieCommitException { try { HoodieCommitMetadata metadata = new HoodieCommitMetadata(); profile.getPartitionPaths().stream().forEach(path -> { @@ -416,9 +407,7 @@ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, } private JavaRDD upsertRecordsInternal(JavaRDD> preppedRecords, - String commitTime, - HoodieTable hoodieTable, - final boolean isUpsert) { + String commitTime, HoodieTable hoodieTable, final boolean isUpsert) { // Cache the tagged records, so we don't end up computing both // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling @@ -441,20 +430,16 @@ private JavaRDD upsertRecordsInternal(JavaRDD> prep JavaRDD writeStatusRDD = partitionedRecords .mapPartitionsWithIndex((partition, recordItr) -> { if (isUpsert) { - return hoodieTable - .handleUpsertPartition(commitTime, partition, recordItr, partitioner); + return hoodieTable.handleUpsertPartition(commitTime, partition, recordItr, partitioner); } else { - return hoodieTable - .handleInsertPartition(commitTime, partition, recordItr, partitioner); + return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner); } - }, true) - .flatMap(writeStatuses -> writeStatuses.iterator()); + }, true).flatMap(writeStatuses -> writeStatuses.iterator()); return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime); } - private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, - WorkloadProfile profile) { + private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) { if (isUpsert) { return table.getUpsertPartitioner(profile); } else { @@ -474,13 +459,9 @@ private JavaRDD updateIndexAndCommitIfNeeded(JavaRDD w private JavaRDD> partition(JavaRDD> dedupedRecords, Partitioner partitioner) { - return dedupedRecords - .mapToPair(record -> - new Tuple2<>( - new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), - record)) - .partitionBy(partitioner) - .map(tuple -> tuple._2()); + return dedupedRecords.mapToPair(record -> new Tuple2<>( + new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record)) + .partitionBy(partitioner).map(tuple -> tuple._2()); } /** @@ -493,12 +474,10 @@ public boolean commit(String commitTime, JavaRDD writeStatuses) { /** * Commit changes performed at the given commitTime marker */ - public boolean commit(String commitTime, - JavaRDD writeStatuses, + public boolean commit(String commitTime, JavaRDD writeStatuses, Optional> extraMetadata) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType()); } @@ -508,15 +487,13 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - List> stats = writeStatuses - .mapToPair((PairFunction) writeStatus -> - new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) - .collect(); + List> stats = writeStatuses.mapToPair( + (PairFunction) writeStatus -> new Tuple2<>( + writeStatus.getPartitionPath(), writeStatus.getStat())).collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); for (Tuple2 stat : stats) { @@ -529,10 +506,9 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, if (finalizeCtx != null && result.isPresent()) { Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); durationInMs.ifPresent(duration -> { - logger.info("Finalize write elapsed time (milliseconds): " + duration); - metrics.updateFinalizeWriteMetrics(duration, result.get()); - } - ); + logger.info("Finalize write elapsed time (milliseconds): " + duration); + metrics.updateFinalizeWriteMetrics(duration, result.get()); + }); } // add in extra metadata @@ -541,8 +517,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, } try { - activeTimeline.saveAsComplete( - new HoodieInstant(true, actionType, commitTime), + activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, commitTime), Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); // Save was a success // Do a inline compaction if enabled @@ -566,9 +541,9 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, } if (writeContext != null) { long durationInMs = metrics.getDurationInMs(writeContext.stop()); - metrics.updateCommitMetrics( - HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs, - metadata); + metrics + .updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), + durationInMs, metadata); writeContext = null; } logger.info("Committed " + commitTime); @@ -587,20 +562,19 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be * rolledback or archived. - * + *

* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be * manually created and deleted. - * + *

* Savepoint should be on a commit that could not have been cleaned. * - * @param user - User creating the savepoint + * @param user - User creating the savepoint * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); if (table.getCompletedCommitTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -614,21 +588,20 @@ public boolean savepoint(String user, String comment) { * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be * rolledback or archived. - * + *

* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be * manually created and deleted. - * + *

* Savepoint should be on a commit that could not have been cleaned. * * @param commitTime - commit that should be savepointed - * @param user - User creating the savepoint - * @param comment - Comment for the savepoint + * @param user - User creating the savepoint + * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String commitTime, String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, @@ -646,8 +619,7 @@ public boolean savepoint(String commitTime, String user, String comment) { table.getActiveTimeline().getInstantDetails(cleanInstant.get()).get()); lastCommitRetained = cleanMetadata.getEarliestCommitToRetain(); } else { - lastCommitRetained = - table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); + lastCommitRetained = table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); } // Cannot allow savepoint time on a commit that could have been cleaned @@ -656,25 +628,24 @@ public boolean savepoint(String commitTime, String user, String comment) { "Could not savepoint commit " + commitTime + " as this is beyond the lookup window " + lastCommitRetained); - Map> latestFilesMap = jsc.parallelize( - FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), + Map> latestFilesMap = jsc.parallelize(FSUtils + .getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) .mapToPair((PairFunction>) partitionPath -> { // Scan all partitions files with this commit time logger.info("Collecting latest files in partition path " + partitionPath); TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); - List latestFiles = - view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) - .map(HoodieDataFile::getFileName).collect(Collectors.toList()); + List latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) + .map(HoodieDataFile::getFileName).collect(Collectors.toList()); return new Tuple2<>(partitionPath, latestFiles); }).collectAsMap(); - HoodieSavepointMetadata metadata = - AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap); + HoodieSavepointMetadata metadata = AvroUtils + .convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime), - AvroUtils.serializeSavepointMetadata(metadata)); + table.getActiveTimeline() + .saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime), + AvroUtils.serializeSavepointMetadata(metadata)); logger.info("Savepoint " + commitTime + " created"); return true; } catch (IOException e) { @@ -691,22 +662,20 @@ public boolean savepoint(String commitTime, String user, String comment) { */ public void deleteSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieInstant savePoint = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); - boolean isSavepointPresent = - table.getCompletedSavepointTimeline().containsInstant(savePoint); + HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime); + boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { logger.warn("No savepoint present " + savepointTime); return; } activeTimeline.revertToInflight(savePoint); - activeTimeline.deleteInflight( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); + activeTimeline + .deleteInflight(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); logger.info("Savepoint " + savepointTime + " deleted"); } @@ -719,30 +688,27 @@ public void deleteSavepoint(String savepointTime) { */ public boolean rollbackToSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline commitTimeline = table.getCommitsTimeline(); - HoodieInstant savePoint = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); - boolean isSavepointPresent = - table.getCompletedSavepointTimeline().containsInstant(savePoint); + HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime); + boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { throw new HoodieRollbackException("No savepoint for commitTime " + savepointTime); } - List commitsToRollback = - commitTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List commitsToRollback = commitTimeline + .findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); logger.info("Rolling back commits " + commitsToRollback); rollback(commitsToRollback); // Make sure the rollback was successful - Optional lastInstant = - activeTimeline.reload().getCommitsTimeline().filterCompletedInstants() - .lastInstant(); + Optional lastInstant = activeTimeline.reload().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); Preconditions.checkArgument(lastInstant.isPresent()); Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), savepointTime + "is not the last commit after rolling back " + commitsToRollback @@ -771,16 +737,14 @@ private void rollback(List commits) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); // Check if any of the commits is a savepoint - do not allow rollback on those commits - List savepoints = - table.getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); + List savepoints = table.getCompletedSavepointTimeline().getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); commits.forEach(s -> { if (savepoints.contains(s)) { throw new HoodieRollbackException( @@ -800,16 +764,15 @@ private void rollback(List commits) { String lastCommit = commits.get(commits.size() - 1); if (!commitTimeline.empty() && !commitTimeline .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { - throw new HoodieRollbackException("Found commits after time :" + lastCommit + - ", please rollback greater commits first"); + throw new HoodieRollbackException( + "Found commits after time :" + lastCommit + ", please rollback greater commits first"); } List inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) { - throw new HoodieRollbackException( - "Found in-flight commits after time :" + lastCommit + - ", please rollback greater commits first"); + throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit + + ", please rollback greater commits first"); } List stats = table.rollback(jsc, commits); @@ -817,8 +780,7 @@ private void rollback(List commits) { // cleanup index entries commits.stream().forEach(s -> { if (!index.rollbackCommit(s)) { - throw new HoodieRollbackException( - "Rollback index changes failed, for time :" + s); + throw new HoodieRollbackException("Rollback index changes failed, for time :" + s); } }); logger.info("Index rolled back for commits " + commits); @@ -826,13 +788,12 @@ private void rollback(List commits) { Optional durationInMs = Optional.empty(); if (context != null) { durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); - Long numFilesDeleted = stats.stream() - .mapToLong(stat -> stat.getSuccessDeleteFiles().size()) + Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()) .sum(); metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); } - HoodieRollbackMetadata rollbackMetadata = - AvroUtils.convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); + HoodieRollbackMetadata rollbackMetadata = AvroUtils + .convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); table.getActiveTimeline().saveAsComplete( new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime), AvroUtils.serializeRollbackMetadata(rollbackMetadata)); @@ -846,8 +807,8 @@ private void rollback(List commits) { table.getActiveTimeline().getRollbackTimeline().getInstants()); } } catch (IOException e) { - throw new HoodieRollbackException("Failed to rollback " + - config.getBasePath() + " commits " + commits, e); + throw new HoodieRollbackException( + "Failed to rollback " + config.getBasePath() + " commits " + commits, e); } } @@ -880,8 +841,7 @@ private void clean(String startCleanTime) throws HoodieIOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), - true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List cleanStats = table.clean(jsc); if (cleanStats.isEmpty()) { @@ -896,15 +856,15 @@ private void clean(String startCleanTime) throws HoodieIOException { } // Create the metadata and save it - HoodieCleanMetadata metadata = - AvroUtils.convertCleanMetadata(startCleanTime, durationInMs, cleanStats); + HoodieCleanMetadata metadata = AvroUtils + .convertCleanMetadata(startCleanTime, durationInMs, cleanStats); logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"); - metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), - metadata.getTotalFilesDeleted()); + metrics + .updateCleanMetrics(durationInMs.orElseGet(() -> -1L), metadata.getTotalFilesDeleted()); - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime), - AvroUtils.serializeCleanMetadata(metadata)); + table.getActiveTimeline() + .saveAsComplete(new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime), + AvroUtils.serializeCleanMetadata(metadata)); logger.info("Marked clean started on " + startCleanTime + " as complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) { @@ -930,12 +890,10 @@ public String startCommit() { public void startCommitWithTime(String commitTime) { logger.info("Generate a new commit time " + commitTime); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = table.getCommitActionType(); - activeTimeline.createInflight( - new HoodieInstant(true, commitActionType, commitTime)); + activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); } /** @@ -948,17 +906,16 @@ public String startCompaction() { return commitTime; } - /** Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, - * we need to explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction + /** + * Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, we need to + * explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction */ public void startCompactionWithTime(String commitTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = HoodieTimeline.COMMIT_ACTION; - activeTimeline.createInflight( - new HoodieInstant(true, commitActionType, commitTime)); + String commitActionType = HoodieTimeline.COMMIT_ACTION; + activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); } /** @@ -968,8 +925,7 @@ public void startCompactionWithTime(String commitTime) { public JavaRDD compact(String commitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); JavaRDD statuses = table.compact(jsc, commitTime); // Trigger the insert and collect statuses statuses = statuses.persist(config.getWriteStatusStorageLevel()); @@ -980,9 +936,6 @@ public JavaRDD compact(String commitTime) throws IOException { /** * Commit a compaction operation - * @param commitTime - * @param writeStatuses - * @param extraMetadata */ public void commitCompaction(String commitTime, JavaRDD writeStatuses, Optional> extraMetadata) { @@ -992,8 +945,6 @@ public void commitCompaction(String commitTime, JavaRDD writeStatus /** * Commit a compaction operation - * @param commitTime - * @param writeStatuses */ public void commitCompaction(String commitTime, JavaRDD writeStatuses) { String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION; @@ -1006,8 +957,8 @@ public void commitCompaction(String commitTime, JavaRDD writeStatus */ private void forceCompact(String compactionCommitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); JavaRDD compactedStatuses = table.compact(jsc, compactionCommitTime); if (!compactedStatuses.isEmpty()) { @@ -1029,8 +980,7 @@ private String forceCompact() throws IOException { } private void commitForceCompaction(JavaRDD writeStatuses, - HoodieTableMetaClient metaClient, - String compactionCommitTime) { + HoodieTableMetaClient metaClient, String compactionCommitTime) { List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) .collect(); @@ -1054,12 +1004,6 @@ private void commitForceCompaction(JavaRDD writeStatuses, } } - public static SparkConf registerClasses(SparkConf conf) { - conf.registerKryoClasses( - new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); - return conf; - } - /** * Deduplicate Hoodie records, using the given deduplication funciton. */ @@ -1074,13 +1018,13 @@ JavaRDD> deduplicateRecords(JavaRDD> records, return new Tuple2<>(key, record); }) .reduceByKey((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect everything + @SuppressWarnings("unchecked") T reducedData = (T) rec1.getData() + .preCombine(rec2.getData()); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything // so pick it from one of the records. return new HoodieRecord(rec1.getKey(), reducedData); - }, parallelism) - .map(recordTuple -> recordTuple._2()); + }, parallelism).map(recordTuple -> recordTuple._2()); } /** @@ -1088,8 +1032,7 @@ JavaRDD> deduplicateRecords(JavaRDD> records, */ private void rollbackInflightCommits() { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); @@ -1103,7 +1046,6 @@ private HoodieTable getTableAndInitCtx() { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible return HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java index 7854e128e10b0..c28db2567b8c0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java @@ -53,10 +53,10 @@ public class WriteStatus implements Serializable { * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * objects are collected in Spark Driver. * - * @param record deflated {@code HoodieRecord} containing information that uniquely identifies - * it. + * @param record deflated {@code HoodieRecord} containing information that uniquely identifies + * it. * @param optionalRecordMetadata optional metadata related to data contained in {@link - * HoodieRecord} before deflation. + * HoodieRecord} before deflation. */ public void markSuccess(HoodieRecord record, Optional> optionalRecordMetadata) { @@ -69,10 +69,10 @@ public void markSuccess(HoodieRecord record, * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * objects are collected in Spark Driver. * - * @param record deflated {@code HoodieRecord} containing information that uniquely identifies - * it. + * @param record deflated {@code HoodieRecord} containing information that uniquely identifies + * it. * @param optionalRecordMetadata optional metadata related to data contained in {@link - * HoodieRecord} before deflation. + * HoodieRecord} before deflation. */ public void markFailure(HoodieRecord record, Throwable t, Optional> optionalRecordMetadata) { @@ -106,14 +106,14 @@ public boolean hasGlobalError() { return globalError != null; } - public void setGlobalError(Throwable t) { - this.globalError = t; - } - public Throwable getGlobalError() { return this.globalError; } + public void setGlobalError(Throwable t) { + this.globalError = t; + } + public List getWrittenRecords() { return writtenRecords; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java index 948a1e00e0e3f..515434a27fc12 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java @@ -30,10 +30,6 @@ public DefaultHoodieConfig(Properties props) { this.props = props; } - public Properties getProps() { - return props; - } - public static void setDefaultOnCondition(Properties props, boolean condition, String propName, String defaultValue) { if (condition) { @@ -48,4 +44,8 @@ public static void setDefaultOnCondition(Properties props, boolean condition, } } + public Properties getProps() { + return props; + } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index dbb47e6bd83b4..3d10d1b73e292 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -34,81 +34,76 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy"; - private static final String DEFAULT_CLEANER_POLICY = - HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name(); - public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic"; - private static final String DEFAULT_AUTO_CLEAN = "true"; - // Turn on inline compaction - after fw delta commits a inline compaction will be run public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline"; - private static final String DEFAULT_INLINE_COMPACT = "false"; - // Run a compaction every N delta commits - public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits"; - private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10"; - + public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = + "hoodie.compact.inline.max" + ".delta.commits"; public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = - "hoodie.cleaner.fileversions.retained"; - private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3"; - + "hoodie.cleaner.fileversions" + ".retained"; public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained"; - private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24"; - public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits"; - private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128); public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits"; - private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96); // Upsert uses this file size to compact new data onto existing files.. public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit"; // Turned off by default public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0); - /** * Configs related to specific table types **/ // Number of inserts, that will be put each partition/bucket for writing - public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size"; + public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = + "hoodie.copyonwrite.insert" + ".split.size"; // The rationale to pick the insert parallelism is the following. Writing out 100MB files, // with atleast 1kb records, means 100K records per file. we just overprovision to 500K public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000); - - // Config to control whether we control insert split sizes automatically based on average record sizes - public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split"; + // Config to control whether we control insert split sizes automatically based on average + // record sizes + public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = + "hoodie.copyonwrite.insert" + ".auto.split"; // its off by default public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false); - - - // This value is used as a guessimate for the record size, if we can't determine this from previous commits - public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate"; - // Used to determine how much more can be packed into a small file, before it exceeds the size limit. + // This value is used as a guessimate for the record size, if we can't determine this from + // previous commits + public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = + "hoodie.copyonwrite" + ".record.size.estimate"; + // Used to determine how much more can be packed into a small file, before it exceeds the size + // limit. public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String .valueOf(1024); - public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism"; public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200); - public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io"; // 500GB of target IO per compaction (both read and write) public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024); - public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy"; // 200GB of target IO per compaction public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class .getName(); - // used to merge records written to log file public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName(); public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class"; // used to choose a trade off between IO vs Memory when performing compaction process - // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory - public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read"; + // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file + // size + small memory + public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = + "hoodie.compaction.lazy" + ".block.read"; public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false"; - // used to choose whether to enable reverse log reading (reverse log traversal) - public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read"; + public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = + "hoodie.compaction" + ".reverse.log.read"; public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false"; + private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS + .name(); + private static final String DEFAULT_AUTO_CLEAN = "true"; + private static final String DEFAULT_INLINE_COMPACT = "false"; + private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10"; + private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3"; + private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24"; + private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128); + private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96); private HoodieCompactionConfig(Properties props) { super(props); @@ -159,8 +154,7 @@ public Builder withCleanerPolicy(HoodieCleaningPolicy policy) { } public Builder retainFileVersions(int fileVersionsRetained) { - props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, - String.valueOf(fileVersionsRetained)); + props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained)); return this; } @@ -238,22 +232,22 @@ public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogR public HoodieCompactionConfig build() { HoodieCompactionConfig config = new HoodieCompactionConfig(props); - setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), - AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN); - setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), - INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT); + setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, + DEFAULT_AUTO_CLEAN); + setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP, + DEFAULT_INLINE_COMPACT); setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP), INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS); - setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), - CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY); + setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP, + DEFAULT_CLEANER_POLICY); setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP), CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED); setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED); - setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP), - MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP); - setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP), - MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP); + setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP), MAX_COMMITS_TO_KEEP, + DEFAULT_MAX_COMMITS_TO_KEEP); + setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP), MIN_COMMITS_TO_KEEP, + DEFAULT_MIN_COMMITS_TO_KEEP); setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES), PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES); setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE), @@ -263,8 +257,8 @@ public HoodieCompactionConfig build() { setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE), COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE); - setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), - CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM, + DEFAULT_CLEANER_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP), COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY); setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP), @@ -277,9 +271,8 @@ public HoodieCompactionConfig build() { COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED); HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP)); - Preconditions.checkArgument( - Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer - .parseInt(props.getProperty(MIN_COMMITS_TO_KEEP))); + Preconditions.checkArgument(Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer + .parseInt(props.getProperty(MIN_COMMITS_TO_KEEP))); return config; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java index 9a9687a3dcb66..5a24cc12f2a98 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java @@ -40,23 +40,25 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism"; // Disable explicit bloom index parallelism setting by default - hoodie auto computes public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0"; - public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges"; + public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = + "hoodie.bloom.index.prune.by" + ".ranges"; public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true"; public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching"; public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true"; - public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level"; + public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = + "hoodie.bloom.index.input.storage" + ".level"; public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; // ***** HBase Index Configs ***** - public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; - public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; - public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; - public final static String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; - public final static String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; - public final static String DEFAULT_HBASE_BATCH_SIZE = "100"; + public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; + public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; + public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; + public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; + public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; + public static final String DEFAULT_HBASE_BATCH_SIZE = "100"; // ***** Bucketed Index Configs ***** - public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets"; + public static final String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets"; private HoodieIndexConfig(Properties props) { super(props); @@ -152,12 +154,12 @@ public Builder withBloomIndexInputStorageLevel(String level) { public HoodieIndexConfig build() { HoodieIndexConfig config = new HoodieIndexConfig(props); - setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), - INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE); + setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, + DEFAULT_INDEX_TYPE); setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES), BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES); - setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), - BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP); + setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP, + DEFAULT_BLOOM_FILTER_FPP); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP), BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java index 65f64e41b3411..aa5f28cc74229 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java @@ -30,8 +30,8 @@ @Immutable public class HoodieMemoryConfig extends DefaultHoodieConfig { - // This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use during merge - // This makes it easier to scale this value as one increases the spark.executor.memory + // This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use + // during merge. This makes it easier to scale this value as one increases the spark.executor.memory public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction"; // Default max memory fraction during hash-merge, excess spills to disk public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6); @@ -87,19 +87,21 @@ public Builder withMaxMemoryFractionPerCompaction(long maxMemoryFractionPerCompa } /** - * Dynamic calculation of max memory to use for for spillable map. user.available.memory = - * spark.executor.memory * (1 - spark.memory.fraction) spillable.available.memory = - * user.available.memory * hoodie.memory.fraction. Anytime the spark.executor.memory or the - * spark.memory.fraction is changed, the memory used for spillable map changes accordingly + * Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory * + * (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime + * the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes + * accordingly */ private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) { final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory"; final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction"; - // This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala#L231} - // so have to re-define this here + // This is hard-coded in spark code {@link + // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ + // spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6"; - // This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/SparkContext.scala#L471} - // so have to re-define this here + // This is hard-coded in spark code {@link + // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ + // spark/SparkContext.scala#L471} so have to re-define this here final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB if (SparkEnv.get() != null) { @@ -109,7 +111,8 @@ private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) { DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L); // 0.6 is the default value used by Spark, - // look at {@link https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} + // look at {@link + // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} double memoryFraction = Double .valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION)); @@ -143,5 +146,4 @@ public HoodieMemoryConfig build() { return config; } } - } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java index f3ef7107753b1..3b06e61bdf765 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java @@ -29,22 +29,22 @@ @Immutable public class HoodieMetricsConfig extends DefaultHoodieConfig { - public final static String METRIC_PREFIX = "hoodie.metrics"; - public final static String METRICS_ON = METRIC_PREFIX + ".on"; - public final static boolean DEFAULT_METRICS_ON = false; - public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type"; - public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = - MetricsReporterType.GRAPHITE; + public static final String METRIC_PREFIX = "hoodie.metrics"; + public static final String METRICS_ON = METRIC_PREFIX + ".on"; + public static final boolean DEFAULT_METRICS_ON = false; + public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type"; + public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType + .GRAPHITE; // Graphite - public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite"; - public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host"; - public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost"; + public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite"; + public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host"; + public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost"; - public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port"; - public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756; + public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port"; + public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756; - public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix"; + public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix"; private HoodieMetricsConfig(Properties props) { super(props); @@ -103,14 +103,14 @@ public HoodieMetricsConfig build() { HoodieMetricsConfig config = new HoodieMetricsConfig(props); setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON)); - setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), - METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name()); - setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), - GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST); - setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), - GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); - setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), - GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); + setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE, + DEFAULT_METRICS_REPORTER_TYPE.name()); + setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST, + DEFAULT_GRAPHITE_SERVER_HOST); + setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT, + String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); + setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT, + String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java index 0eb0a5c0eb45f..b72e6f7434530 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java @@ -36,10 +36,11 @@ public class HoodieStorageConfig extends DefaultHoodieConfig { public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024); // used to size log files public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size"; - public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024*1024*1024); // 1 GB + public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String + .valueOf(1024 * 1024 * 1024); // 1 GB // used to size data blocks in log file public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size"; - public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256*1024*1024); // 256 MB + public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio"; // Default compression ratio for parquet public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 14909bffae38e..683ec952d7f3d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -16,7 +16,6 @@ package com.uber.hoodie.config; - import com.google.common.base.Preconditions; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieCleaningPolicy; @@ -24,15 +23,14 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.metrics.MetricsReporterType; -import org.apache.spark.storage.StorageLevel; - -import javax.annotation.concurrent.Immutable; import java.io.File; import java.io.FileReader; import java.io.IOException; import java.io.InputStream; import java.util.Map; import java.util.Properties; +import javax.annotation.concurrent.Immutable; +import org.apache.spark.storage.StorageLevel; /** * Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient} @@ -40,9 +38,9 @@ @Immutable public class HoodieWriteConfig extends DefaultHoodieConfig { + public static final String TABLE_NAME = "hoodie.table.name"; private static final String BASE_PATH_PROP = "hoodie.base.path"; private static final String AVRO_SCHEMA = "hoodie.avro.schema"; - public static final String TABLE_NAME = "hoodie.table.name"; private static final String DEFAULT_PARALLELISM = "200"; private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism"; private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; @@ -57,13 +55,16 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit"; private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true"; - private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning"; + private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = + "hoodie.assume.date" + ".partitioning"; private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class"; private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); - private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "hoodie.copyonwrite.use.temp.folder.for.create"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = + "hoodie.copyonwrite.use" + ".temp.folder.for.create"; private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "false"; - private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "hoodie.copyonwrite.use.temp.folder.for.merge"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = + "hoodie.copyonwrite.use" + ".temp.folder.for.merge"; private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "false"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; @@ -72,6 +73,10 @@ private HoodieWriteConfig(Properties props) { super(props); } + public static HoodieWriteConfig.Builder newBuilder() { + return new Builder(); + } + /** * base properties **/ @@ -137,8 +142,8 @@ public boolean shouldUseTempFolderForCopyOnWriteForMerge() { } public boolean shouldUseTempFolderForCopyOnWrite() { - return shouldUseTempFolderForCopyOnWriteForCreate() || - shouldUseTempFolderForCopyOnWriteForMerge(); + return shouldUseTempFolderForCopyOnWriteForCreate() + || shouldUseTempFolderForCopyOnWriteForMerge(); } public int getFinalizeWriteParallelism() { @@ -154,8 +159,8 @@ public HoodieCleaningPolicy getCleanerPolicy() { } public int getCleanerFileVersionsRetained() { - return Integer.parseInt( - props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP)); + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP)); } public int getCleanerCommitsRetained() { @@ -177,8 +182,8 @@ public int getParquetSmallFileLimit() { } public int getCopyOnWriteInsertSplitSize() { - return Integer.parseInt( - props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE)); + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE)); } public int getCopyOnWriteRecordSizeEstimate() { @@ -204,8 +209,8 @@ public boolean isInlineCompaction() { } public int getInlineCompactDeltaCommitMax() { - return Integer.parseInt( - props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP)); + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP)); } public CompactionStrategy getCompactionStrategy() { @@ -340,11 +345,7 @@ public int getGraphiteServerPort() { public String getGraphiteMetricPrefix() { return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX); } - - public static HoodieWriteConfig.Builder newBuilder() { - return new Builder(); - } - + /** * memory configs */ @@ -486,15 +487,15 @@ public Builder withWriteStatusClass(Class writeStatusClas public Builder withUseTempFolderCopyOnWriteForCreate( boolean shouldUseTempFolderCopyOnWriteForCreate) { - props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, String.valueOf - (shouldUseTempFolderCopyOnWriteForCreate)); + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, + String.valueOf(shouldUseTempFolderCopyOnWriteForCreate)); return this; } public Builder withUseTempFolderCopyOnWriteForMerge( boolean shouldUseTempFolderCopyOnWriteForMerge) { - props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, String.valueOf - (shouldUseTempFolderCopyOnWriteForMerge)); + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, + String.valueOf(shouldUseTempFolderCopyOnWriteForMerge)); return this; } @@ -510,8 +511,7 @@ public HoodieWriteConfig build() { setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), - BULKINSERT_PARALLELISM, - DEFAULT_PARALLELISM); + BULKINSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java index 08cea4dc71dbe..e69c09b1ac784 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java @@ -21,12 +21,6 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.exception.HoodieException; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.util.SizeEstimator; - import java.util.Iterator; import java.util.Optional; import java.util.concurrent.LinkedBlockingQueue; @@ -35,53 +29,63 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.util.SizeEstimator; /** - * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It internally samples - * every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in buffer accordingly. This is done to - * ensure that we don't OOM. + * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It + * internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in + * buffer accordingly. This is done to ensure that we don't OOM. */ -public class BufferedIterator> - implements Iterator> { +public class BufferedIterator> implements + Iterator> { - private static Logger logger = LogManager.getLogger(BufferedIterator.class); // interval used for polling records in the queue. public static final int RECORD_POLL_INTERVAL_SEC = 5; // rate used for sampling records to determine avg record size in bytes. public static final int RECORD_SAMPLING_RATE = 64; // maximum records that will be cached private static final int RECORD_CACHING_LIMIT = 128 * 1024; - // It indicates number of records to cache. We will be using sampled record's average size to determine how many + private static Logger logger = LogManager.getLogger(BufferedIterator.class); + // It indicates number of records to cache. We will be using sampled record's average size to + // determine how many // records we should cache and will change (increase/decrease) permits accordingly. @VisibleForTesting public final Semaphore rateLimiter = new Semaphore(1); // used for sampling records with "RECORD_SAMPLING_RATE" frequency. public final AtomicLong samplingRecordCounter = new AtomicLong(-1); - // indicates rate limit (number of records to cache). it is updated whenever there is a change in avg record size. - @VisibleForTesting - public int currentRateLimit = 1; // internal buffer to cache buffered records. - private final LinkedBlockingQueue>> buffer = new LinkedBlockingQueue<>(); + private final LinkedBlockingQueue>> buffer = new + LinkedBlockingQueue<>(); // maximum amount of memory to be used for buffering records. private final long bufferMemoryLimit; - // indicates avg record size in bytes. It is updated whenever a new record is sampled. - @VisibleForTesting - public long avgRecordSizeInBytes = 0; - // indicates number of samples collected so far. - private long numSamples = 0; // original iterator from where records are read for buffering. private final Iterator inputIterator; - // it holds the root cause of the exception in case either buffering records (reading from inputIterator) fails or + // it holds the root cause of the exception in case either buffering records (reading from + // inputIterator) fails or // thread reading records from buffer fails. private final AtomicReference hasFailed = new AtomicReference(null); // used for indicating that all the records from buffer are read successfully. private final AtomicBoolean isDone = new AtomicBoolean(false); - // next record to be read from buffer. - private BufferedIteratorPayload nextRecord; // schema used for fetching insertValue from HoodieRecord. private final Schema schema; + // indicates rate limit (number of records to cache). it is updated whenever there is a change + // in avg record size. + @VisibleForTesting + public int currentRateLimit = 1; + // indicates avg record size in bytes. It is updated whenever a new record is sampled. + @VisibleForTesting + public long avgRecordSizeInBytes = 0; + // indicates number of samples collected so far. + private long numSamples = 0; + // next record to be read from buffer. + private BufferedIteratorPayload nextRecord; - public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, final Schema schema) { + public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, + final Schema schema) { this.inputIterator = iterator; this.bufferMemoryLimit = bufferMemoryLimit; this.schema = schema; @@ -92,23 +96,28 @@ public int size() { return this.buffer.size(); } - // It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in bytes. It is used - // for determining how many maximum records to buffer. Based on change in avg size it may increase or decrease + // It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in + // bytes. It is used + // for determining how many maximum records to buffer. Based on change in avg size it may + // increase or decrease // available permits. private void adjustBufferSizeIfNeeded(final T record) throws InterruptedException { if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { return; } final long recordSizeInBytes = SizeEstimator.estimate(record); - final long newAvgRecordSizeInBytes = - Math.max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); - final int newRateLimit = - (int) Math.min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes)); -// System.out.println("recordSizeInBytes:" + recordSizeInBytes + ":newAvgRecordSizeInBytes:" + newAvgRecordSizeInBytes -// + ":newRateLimit:" + newRateLimit + ":currentRateLimit:" + currentRateLimit + ":numSamples:" + numSamples -// + ":avgRecordSizeInBytes:" + avgRecordSizeInBytes); - - // If there is any change in number of records to cache then we will either release (if it increased) or acquire + final long newAvgRecordSizeInBytes = Math + .max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); + final int newRateLimit = (int) Math + .min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes)); + // System.out.println("recordSizeInBytes:" + recordSizeInBytes + ":newAvgRecordSizeInBytes:" + + // newAvgRecordSizeInBytes + // + ":newRateLimit:" + newRateLimit + ":currentRateLimit:" + currentRateLimit + + // ":numSamples:" + numSamples + // + ":avgRecordSizeInBytes:" + avgRecordSizeInBytes); + + // If there is any change in number of records to cache then we will either release (if it + // increased) or acquire // (if it decreased) to adjust rate limiting to newly computed value. if (newRateLimit > currentRateLimit) { rateLimiter.release(newRateLimit - currentRateLimit); @@ -120,12 +129,14 @@ private void adjustBufferSizeIfNeeded(final T record) throws InterruptedExceptio numSamples++; } - // inserts record into internal buffer. It also fetches insert value from the record to offload computation work on to + // inserts record into internal buffer. It also fetches insert value from the record to offload + // computation work on to // buffering thread. private void insertRecord(T t) throws Exception { rateLimiter.acquire(); adjustBufferSizeIfNeeded(t); - // We are retrieving insert value in the record buffering thread to offload computation around schema validation + // We are retrieving insert value in the record buffering thread to offload computation + // around schema validation // and record creation to it. final BufferedIteratorPayload payload = new BufferedIteratorPayload<>(t, this.schema); buffer.put(Optional.of(payload)); @@ -198,12 +209,15 @@ private void throwExceptionIfFailed() { public void markAsFailed(Exception e) { this.hasFailed.set(e); - // release the permits so that if the buffering thread is waiting for permits then it will get it. + // release the permits so that if the buffering thread is waiting for permits then it will + // get it. this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); } - // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread. + // Used for caching HoodieRecord along with insertValue. We need this to offload computation + // work to buffering thread. static class BufferedIteratorPayload { + public T record; public Optional insertValue; // It caches the exception seen while fetching insert value. diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java index 8d305d21450b8..c8a39b6857cd5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java @@ -29,8 +29,8 @@ /** * Map function that handles a sorted stream of HoodieRecords */ -public class BulkInsertMapFunction - implements Function2>, Iterator>> { +public class BulkInsertMapFunction implements + Function2>, Iterator>> { private String commitTime; private HoodieWriteConfig config; @@ -45,8 +45,7 @@ public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config, @Override public Iterator> call(Integer partition, - Iterator> sortedRecordItr) - throws Exception { + Iterator> sortedRecordItr) throws Exception { return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index 2406762a99302..6e1ae02c4370e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -24,11 +24,6 @@ import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; - import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -38,6 +33,10 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -68,35 +67,37 @@ protected void start() { @Override protected List computeNext() { - // Need to set current spark thread's TaskContext into newly launched thread so that new thread can access + // Need to set current spark thread's TaskContext into newly launched thread so that new + // thread can access // TaskContext properties. final TaskContext sparkThreadTaskContext = TaskContext.get(); // Executor service used for launching writer thread. final ExecutorService writerService = Executors.newFixedThreadPool(1); try { - // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - final BufferedIterator> bufferedIterator = - new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(), - HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)); - Future> writerResult = - writerService.submit( - () -> { - logger.info("starting hoodie writer thread"); - // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext - // properties. - TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); - List statuses = new LinkedList<>(); - try { - statuses.addAll(handleWrite(bufferedIterator)); - logger.info("hoodie write is done; notifying reader thread"); - return statuses; - } catch (Exception e) { - logger.error("error writing hoodie records", e); - bufferedIterator.markAsFailed(e); - throw e; - } - }); - // Buffering records into internal buffer. This can throw exception either if reading records from spark fails or + // Used for buffering records which is controlled by + // HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. + final BufferedIterator> bufferedIterator = new BufferedIterator<>(inputItr, + hoodieConfig.getWriteBufferLimitBytes(), + HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)); + Future> writerResult = writerService.submit(() -> { + logger.info("starting hoodie writer thread"); + // Passing parent thread's TaskContext to newly launched thread for it to access original + // TaskContext + // properties. + TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + List statuses = new LinkedList<>(); + try { + statuses.addAll(handleWrite(bufferedIterator)); + logger.info("hoodie write is done; notifying reader thread"); + return statuses; + } catch (Exception e) { + logger.error("error writing hoodie records", e); + bufferedIterator.markAsFailed(e); + throw e; + } + }); + // Buffering records into internal buffer. This can throw exception either if reading + // records from spark fails or // if writing buffered records into parquet file fails. bufferedIterator.startBuffering(); logger.info("waiting for hoodie write to finish"); @@ -110,28 +111,27 @@ protected List computeNext() { } } - private List handleWrite(final BufferedIterator> bufferedIterator) { + private List handleWrite( + final BufferedIterator> bufferedIterator) { List statuses = new ArrayList<>(); while (bufferedIterator.hasNext()) { - final BufferedIterator.BufferedIteratorPayload> payload = bufferedIterator.next(); + final BufferedIterator.BufferedIteratorPayload> payload = bufferedIterator + .next(); // clean up any partial failures if (!partitionsCleaned.contains(payload.record.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with // the same data again. Thus, before we open any files under a given partition, we // first delete any files in the same partitionPath written by same Spark partition - HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, - commitTime, - payload.record.getPartitionPath(), - TaskContext.getPartitionId(), - hoodieTable); + HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, + payload.record.getPartitionPath(), TaskContext.getPartitionId(), hoodieTable); partitionsCleaned.add(payload.record.getPartitionPath()); } // lazily initialize the handle, for the first time if (handle == null) { - handle = - new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath()); + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, + payload.record.getPartitionPath()); } if (handle.canWrite(payload.record)) { @@ -141,9 +141,10 @@ private List handleWrite(final BufferedIterator> // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle - handle = - new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath()); - handle.write(payload.record, payload.insertValue, payload.exception); // we should be able to write 1 payload. + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, + payload.record.getPartitionPath()); + handle.write(payload.record, payload.insertValue, + payload.exception); // we should be able to write 1 payload. } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java index 2720e001e0347..4062a7deebf13 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java @@ -23,9 +23,9 @@ * inputItr classes in order to simplify the implementation of lazy iterators for mapPartitions use * cases. Note [SPARK-3369], which gives the reasons for backwards compatibility with regard to the * iterable API despite Spark's single pass nature. - * + *

* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input) - * + *

* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next() * to obtain them - Assumes hasNext() gets called atleast once. - Concrete Implementation is * responsible for calling inputIterator.next() and doing the processing in computeNext() diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java index 642cb7d9bbb20..5733237b44c79 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java @@ -37,22 +37,30 @@ */ public abstract class HoodieIndex implements Serializable { - protected transient JavaSparkContext jsc = null; - - public enum IndexType { - HBASE, - INMEMORY, - BLOOM, - BUCKETED - } - protected final HoodieWriteConfig config; + protected transient JavaSparkContext jsc = null; protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) { this.config = config; this.jsc = jsc; } + public static HoodieIndex createIndex(HoodieWriteConfig config, + JavaSparkContext jsc) throws HoodieIndexException { + switch (config.getIndexType()) { + case HBASE: + return new HBaseIndex<>(config, jsc); + case INMEMORY: + return new InMemoryHashIndex<>(config, jsc); + case BLOOM: + return new HoodieBloomIndex<>(config, jsc); + case BUCKETED: + return new BucketedIndex<>(config, jsc); + default: + throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); + } + } + /** * Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]] * If the optional FullFilePath value is not present, then the key is not found. If the @@ -71,7 +79,7 @@ public abstract JavaRDD> tagLocation(JavaRDD> re /** * Extracts the location of written records, and updates the index. - * + *

* TODO(vc): We may need to propagate the record as well in a WriteStatus class */ public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, @@ -107,18 +115,7 @@ public abstract JavaRDD updateLocation(JavaRDD writeSt public abstract boolean isImplicitWithStorage(); - public static HoodieIndex createIndex( - HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException { - switch (config.getIndexType()) { - case HBASE: - return new HBaseIndex<>(config, jsc); - case INMEMORY: - return new InMemoryHashIndex<>(config, jsc); - case BLOOM: - return new HoodieBloomIndex<>(config, jsc); - case BUCKETED: - return new BucketedIndex<>(config, jsc); - } - throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); + public enum IndexType { + HBASE, INMEMORY, BLOOM, BUCKETED } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java index 422d319837eea..6338e60bb1a14 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java @@ -49,32 +49,11 @@ public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) { } @Override - public JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, final HoodieTable table) { + public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, + final HoodieTable table) { throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); } - /** - * Function that tags each HoodieRecord with an existing location, if known. - */ - class LocationTagFunction - implements Function2>, Iterator>> { - - @Override - public Iterator> call(Integer partitionNum, - Iterator> hoodieRecordIterator) { - List> taggedRecords = new ArrayList<>(); - while (hoodieRecordIterator.hasNext()) { - HoodieRecord rec = hoodieRecordIterator.next(); - if (recordLocationMap.containsKey(rec.getKey())) { - rec.setCurrentLocation(recordLocationMap.get(rec.getKey())); - } - taggedRecords.add(rec); - } - return taggedRecords.iterator(); - } - } - @Override public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieTable hoodieTable) { @@ -132,4 +111,25 @@ public boolean canIndexLogFiles() { public boolean isImplicitWithStorage() { return false; } + + /** + * Function that tags each HoodieRecord with an existing location, if known. + */ + class LocationTagFunction implements + Function2>, Iterator>> { + + @Override + public Iterator> call(Integer partitionNum, + Iterator> hoodieRecordIterator) { + List> taggedRecords = new ArrayList<>(); + while (hoodieRecordIterator.hasNext()) { + HoodieRecord rec = hoodieRecordIterator.next(); + if (recordLocationMap.containsKey(rec.getKey())) { + rec.setCurrentLocation(recordLocationMap.get(rec.getKey())); + } + taggedRecords.add(rec); + } + return taggedRecords.iterator(); + } + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java index 0f0fb990873ee..6d398171b4cb5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java @@ -64,8 +64,7 @@ public boolean hasKeyRanges() { * Does the given key fall within the range (inclusive) */ public boolean isKeyInRange(String recordKey) { - return minRecordKey.compareTo(recordKey) <= 0 && - maxRecordKey.compareTo(recordKey) >= 0; + return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0; } @Override @@ -78,9 +77,8 @@ public boolean equals(Object o) { } BloomIndexFileInfo that = (BloomIndexFileInfo) o; - return Objects.equal(that.fileName, fileName) && - Objects.equal(that.minRecordKey, minRecordKey) && - Objects.equal(that.maxRecordKey, maxRecordKey); + return Objects.equal(that.fileName, fileName) && Objects.equal(that.minRecordKey, minRecordKey) + && Objects.equal(that.maxRecordKey, maxRecordKey); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 73a07fc4cd554..2b756c591840a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -56,12 +56,11 @@ */ public class HoodieBloomIndex extends HoodieIndex { - private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class); - // we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476) private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024; // this is how much a triplet of (partitionPath, fileId, recordKey) costs. private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300; + private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class); private static int MAX_ITEMS_PER_SHUFFLE_PARTITION = SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET; @@ -108,32 +107,31 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, return taggedRecordRDD; } - public JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, final HoodieTable table) { - JavaPairRDD partitionRecordKeyPairRDD = - hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); + public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, + final HoodieTable table) { + JavaPairRDD partitionRecordKeyPairRDD = hoodieKeys + .mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); // Lookup indexes for all the partition/recordkey pair - JavaPairRDD rowKeyFilenamePairRDD = - lookupIndex(partitionRecordKeyPairRDD, table); - - JavaPairRDD rowKeyHoodieKeyPairRDD = - hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); - - return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD) - .mapToPair(keyPathTuple -> { - Optional recordLocationPath; - if (keyPathTuple._2._2.isPresent()) { - String fileName = keyPathTuple._2._2.get(); - String partitionPath = keyPathTuple._2._1.getPartitionPath(); - recordLocationPath = Optional.of(new Path( - new Path(table.getMetaClient().getBasePath(), partitionPath), - fileName).toUri().getPath()); - } else { - recordLocationPath = Optional.absent(); - } - return new Tuple2<>(keyPathTuple._2._1, recordLocationPath); - }); + JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, + table); + + JavaPairRDD rowKeyHoodieKeyPairRDD = hoodieKeys + .mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); + + return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(keyPathTuple -> { + Optional recordLocationPath; + if (keyPathTuple._2._2.isPresent()) { + String fileName = keyPathTuple._2._2.get(); + String partitionPath = keyPathTuple._2._1.getPartitionPath(); + recordLocationPath = Optional + .of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName) + .toUri().getPath()); + } else { + recordLocationPath = Optional.absent(); + } + return new Tuple2<>(keyPathTuple._2._1, recordLocationPath); + }); } /** @@ -152,21 +150,21 @@ private JavaPairRDD lookupIndex( final Map> partitionToFileInfo = fileInfoList.stream() .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); - // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it. + // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, + // that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo, - partitionRecordKeyPairRDD, - parallelism); + partitionRecordKeyPairRDD, parallelism); } /** * The index lookup can be skewed in three dimensions : #files, #partitions, #records - * + *

* To be able to smoothly handle skews, we need to compute how to split each partitions into * subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to < * 2GB. - * + *

* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified * as a NON-zero number, then that is used explicitly. */ @@ -184,7 +182,8 @@ private int autoComputeParallelism(final Map recordsPerPartition, // records for a partition. Map filesPerPartition = partitionToFileInfo.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size()))); - long totalFiles = 0, totalRecords = 0; + long totalFiles = 0; + long totalRecords = 0; for (String partitionPath : recordsPerPartition.keySet()) { long numRecords = recordsPerPartition.get(partitionPath); long numFiles = @@ -210,22 +209,22 @@ private int autoComputeParallelism(final Map recordsPerPartition, /** * Its crucial to pick the right parallelism. - * + *

* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism : * typically number of input file splits - * + *

* We pick the max such that, we are always safe, but go higher if say a there are a lot of input * files. (otherwise, we will fallback to number of partitions in input and end up with slow * performance) */ private int determineParallelism(int inputParallelism, int totalSubPartitions) { - // If bloom index parallelism is set, use it to to check against the input parallelism and take the max + // If bloom index parallelism is set, use it to to check against the input parallelism and + // take the max int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); int joinParallelism = Math.max(totalSubPartitions, indexParallelism); - logger.info("InputParallelism: ${" + inputParallelism + "}, " + - "IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " + - "TotalSubParts: ${" + totalSubPartitions + "}, " + - "Join Parallelism set to : " + joinParallelism); + logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config + .getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, " + + "Join Parallelism set to : " + joinParallelism); return joinParallelism; } @@ -237,36 +236,31 @@ List> loadInvolvedFiles(List partitio final HoodieTable hoodieTable) { // Obtain the latest data files from all the partitions. List> dataFilesList = jsc - .parallelize(partitions, Math.max(partitions.size(), 1)) - .flatMapToPair(partitionPath -> { - java.util.Optional latestCommitTime = - hoodieTable.getCommitsTimeline().filterCompletedInstants().lastInstant(); + .parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> { + java.util.Optional latestCommitTime = hoodieTable.getCommitsTimeline() + .filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { - filteredFiles = - hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath, - latestCommitTime.get().getTimestamp()) - .map(f -> new Tuple2<>(partitionPath, f)) - .collect(toList()); + filteredFiles = hoodieTable.getROFileSystemView() + .getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) + .map(f -> new Tuple2<>(partitionPath, f)).collect(toList()); } return filteredFiles.iterator(); }).collect(); if (config.getBloomIndexPruneByRanges()) { // also obtain file ranges, if range pruning is enabled - return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)) - .mapToPair(ft -> { - try { - String[] minMaxKeys = ParquetUtils - .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), - ft._2().getFileStatus().getPath()); - return new Tuple2<>(ft._1(), - new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); - } catch (MetadataNotFoundException me) { - logger.warn("Unable to find range metadata in file :" + ft._2()); - return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())); - } - }).collect(); + return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)).mapToPair(ft -> { + try { + String[] minMaxKeys = ParquetUtils + .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), ft._2().getFileStatus().getPath()); + return new Tuple2<>(ft._1(), + new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); + } catch (MetadataNotFoundException me) { + logger.warn("Unable to find range metadata in file :" + ft._2()); + return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())); + } + }).collect(); } else { return dataFilesList.stream() .map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()))) @@ -320,62 +314,60 @@ private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recor * (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range * pruning. */ - // sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey + // sub-partition to ensure the records can be looked up against files & also prune + // file<=>record comparisons based on recordKey // ranges in the index info. @VisibleForTesting JavaPairRDD> explodeRecordRDDWithFileComparisons( final Map> partitionToFileIndexInfo, JavaPairRDD partitionRecordKeyPairRDD) { - return partitionRecordKeyPairRDD - .map(partitionRecordKeyPair -> { - String recordKey = partitionRecordKeyPair._2(); - String partitionPath = partitionRecordKeyPair._1(); - - List indexInfos = partitionToFileIndexInfo.get(partitionPath); - List>> recordComparisons = new ArrayList<>(); - if (indexInfos - != null) { // could be null, if there are no files in a given partition yet. - // for each candidate file in partition, that needs to be compared. - for (BloomIndexFileInfo indexInfo : indexInfos) { - if (shouldCompareWithFile(indexInfo, recordKey)) { - recordComparisons.add( - new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey), - new Tuple2<>(indexInfo.getFileName(), - new HoodieKey(recordKey, partitionPath)))); - } - } + return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> { + String recordKey = partitionRecordKeyPair._2(); + String partitionPath = partitionRecordKeyPair._1(); + + List indexInfos = partitionToFileIndexInfo.get(partitionPath); + List>> recordComparisons = new ArrayList<>(); + if (indexInfos != null) { // could be null, if there are no files in a given partition yet. + // for each candidate file in partition, that needs to be compared. + for (BloomIndexFileInfo indexInfo : indexInfos) { + if (shouldCompareWithFile(indexInfo, recordKey)) { + recordComparisons.add( + new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey), + new Tuple2<>(indexInfo.getFileName(), + new HoodieKey(recordKey, partitionPath)))); } - return recordComparisons; - }) - .flatMapToPair(t -> t.iterator()); + } + } + return recordComparisons; + }).flatMapToPair(t -> t.iterator()); } /** * Find out pair. All workload grouped by file-level. - * + *

* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such * that each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load * rowKeys, (3) Tag rowKey - * + *

* Make sure the parallelism is atleast the groupby parallelism for tagging location */ @VisibleForTesting JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTable, final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD, - int totalSubpartitions) { + JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions) { int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions); - JavaPairRDD> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons( - partitionToFileIndexInfo, partitionRecordKeyPairRDD) - // sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly - .sortByKey(true, joinParallelism); + JavaPairRDD> fileSortedTripletRDD = + explodeRecordRDDWithFileComparisons( + partitionToFileIndexInfo, partitionRecordKeyPairRDD) + // sort further based on filename, such that all checking for the file can happen within + // a single partition, on-the-fly + .sortByKey(true, joinParallelism); - return fileSortedTripletRDD - .mapPartitionsWithIndex( - new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) + return fileSortedTripletRDD.mapPartitionsWithIndex( + new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) .flatMap(indexLookupResults -> indexLookupResults.iterator()) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { @@ -391,25 +383,23 @@ JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTab * Tag the back to the original HoodieRecord RDD. */ private JavaRDD> tagLocationBacktoRecords( - JavaPairRDD rowKeyFilenamePairRDD, - JavaRDD> recordRDD) { + JavaPairRDD rowKeyFilenamePairRDD, JavaRDD> recordRDD) { JavaPairRDD> rowKeyRecordPairRDD = recordRDD .mapToPair(record -> new Tuple2<>(record.getRecordKey(), record)); - // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join. - return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map( - v1 -> { - HoodieRecord record = v1._1(); - if (v1._2().isPresent()) { - String filename = v1._2().get(); - if (filename != null && !filename.isEmpty()) { - record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename), - FSUtils.getFileId(filename))); - } - } - return record; + // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), + // so we do left outer join. + return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(v1 -> { + HoodieRecord record = v1._1(); + if (v1._2().isPresent()) { + String filename = v1._2().get(); + if (filename != null && !filename.isEmpty()) { + record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename), + FSUtils.getFileId(filename))); } - ); + } + return record; + }); } @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index cf5a12007de4e..074ec56dae863 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -41,7 +41,8 @@ * actual files */ public class HoodieBloomIndexCheckFunction implements - Function2>>, Iterator>> { + Function2>>, + Iterator>> { private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class); @@ -58,8 +59,7 @@ public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) { * Given a list of row keys and one file, return only row keys existing in that file. */ public static List checkCandidatesAgainstFile(Configuration configuration, - List candidateRecordKeys, - Path filePath) throws HoodieIndexException { + List candidateRecordKeys, Path filePath) throws HoodieIndexException { List foundRecordKeys = new ArrayList<>(); try { // Load all rowKeys from the file, to double-confirm @@ -86,6 +86,13 @@ public static List checkCandidatesAgainstFile(Configuration configuratio return foundRecordKeys; } + @Override + public Iterator> call(Integer partition, + Iterator>> fileParitionRecordKeyTripletItr) + throws Exception { + return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); + } + class LazyKeyCheckIterator extends LazyIterableIterator>, List> { @@ -143,7 +150,8 @@ protected List computeNext() { // if continue on current file) if (fileName.equals(currentFile)) { - // check record key against bloom filter of current file & add to possible keys if needed + // check record key against bloom filter of current file & add to possible keys if + // needed if (bloomFilter.mightContain(recordKey)) { if (logger.isDebugEnabled()) { logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName); @@ -201,12 +209,4 @@ protected List computeNext() { protected void end() { } } - - - @Override - public Iterator> call(Integer partition, - Iterator>> fileParitionRecordKeyTripletItr) - throws Exception { - return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); - } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java index a361a7b06627f..a8e3921730c66 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java @@ -38,9 +38,9 @@ /** * An `stateless` index implementation that will using a deterministic mapping function to determine * the fileID for a given record. - * + *

* Pros: - Fast - * + *

* Cons : - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune * this) - Could increase write amplification on copy-on-write storage since inserts always rewrite * files - Not global. diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java index e2542925c87fb..4928d144055db 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java @@ -27,12 +27,16 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -51,23 +55,18 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function2; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; - /** * Hoodie Index implementation backed by HBase */ public class HBaseIndex extends HoodieIndex { - private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s"); - private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts"); - private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name"); - private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path"); - private static Logger logger = LogManager.getLogger(HBaseIndex.class); + private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s"); + private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts"); + private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name"); + private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path"); + private static Logger logger = LogManager.getLogger(HBaseIndex.class); + private static Connection hbaseConnection = null; private final String tableName; public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { @@ -77,14 +76,12 @@ public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { } @Override - public JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, HoodieTable table) { + public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, + HoodieTable table) { //TODO : Change/Remove filterExists in HoodieReadClient() and revisit throw new UnsupportedOperationException("HBase index does not implement check exist"); } - private static Connection hbaseConnection = null; - private Connection getHBaseConnection() { Configuration hbaseConfig = HBaseConfiguration.create(); String quorum = config.getHbaseZkQuorum(); @@ -100,15 +97,15 @@ private Connection getHBaseConnection() { } /** - * Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is closed when - * JVM exits + * Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is + * closed when JVM exits */ private void addShutDownHook() { Runtime.getRuntime().addShutdownHook(new Thread() { public void run() { try { hbaseConnection.close(); - } catch(Exception e) { + } catch (Exception e) { // fail silently for any sort of exception } } @@ -126,101 +123,103 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); // Check if the last commit ts for this row is 1) present in the timeline or // 2) is less than the first commit ts in the timeline - return !commitTimeline.empty() && (commitTimeline.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) || - HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), - commitTs, HoodieTimeline.GREATER)); + return !commitTimeline.empty() && (commitTimeline + .containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) + || HoodieTimeline + .compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs, + HoodieTimeline.GREATER)); } /** * Function that tags each HoodieRecord with an existing location, if known. */ private Function2>, Iterator>> - locationTagFunction(HoodieTable hoodieTable) { + locationTagFunction(HoodieTable hoodieTable) { return (Function2>, Iterator>>) (partitionNum, hoodieRecordIterator) -> { - Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize(); + Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize(); - // Grab the global HBase connection - synchronized (HBaseIndex.class) { - if (hbaseConnection == null || hbaseConnection.isClosed()) { - hbaseConnection = getHBaseConnection(); - } - } - List> taggedRecords = new ArrayList<>(); - HTable hTable = null; - try { - hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName)); - List statements = new ArrayList<>(); - List currentBatchOfRecords = new LinkedList<>(); - // Do the tagging. - while (hoodieRecordIterator.hasNext()) { - HoodieRecord rec = hoodieRecordIterator.next(); - statements.add(generateStatement(rec.getRecordKey())); - currentBatchOfRecords.add(rec); - // iterator till we reach batch size - if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) { - // get results for batch from Hbase - Result[] results = hTable.get(statements); - // clear statements to be GC'd - statements.clear(); - for (Result result : results) { - // first, attempt to grab location from HBase - HoodieRecord currentRecord = currentBatchOfRecords.remove(0); - if (result.getRow() != null) { - String keyFromResult = Bytes.toString(result.getRow()); - String commitTs = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)); - String fileId = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)); - String partitionPath = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); + // Grab the global HBase connection + synchronized (HBaseIndex.class) { + if (hbaseConnection == null || hbaseConnection.isClosed()) { + hbaseConnection = getHBaseConnection(); + } + } + List> taggedRecords = new ArrayList<>(); + HTable hTable = null; + try { + hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName)); + List statements = new ArrayList<>(); + List currentBatchOfRecords = new LinkedList<>(); + // Do the tagging. + while (hoodieRecordIterator.hasNext()) { + HoodieRecord rec = hoodieRecordIterator.next(); + statements.add(generateStatement(rec.getRecordKey())); + currentBatchOfRecords.add(rec); + // iterator till we reach batch size + if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) { + // get results for batch from Hbase + Result[] results = hTable.get(statements); + // clear statements to be GC'd + statements.clear(); + for (Result result : results) { + // first, attempt to grab location from HBase + HoodieRecord currentRecord = currentBatchOfRecords.remove(0); + if (result.getRow() != null) { + String keyFromResult = Bytes.toString(result.getRow()); + String commitTs = Bytes + .toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)); + String fileId = Bytes + .toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)); + String partitionPath = Bytes + .toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - if (checkIfValidCommit(hoodieTable, commitTs)) { - currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), - partitionPath), currentRecord.getData()); - currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); - taggedRecords.add(currentRecord); - // the key from Result and the key being processed should be same - assert (currentRecord.getRecordKey().contentEquals(keyFromResult)); - } else { //if commit is invalid, treat this as a new taggedRecord - taggedRecords.add(currentRecord); + if (checkIfValidCommit(hoodieTable, commitTs)) { + currentRecord = new HoodieRecord( + new HoodieKey(currentRecord.getRecordKey(), partitionPath), + currentRecord.getData()); + currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); + taggedRecords.add(currentRecord); + // the key from Result and the key being processed should be same + assert (currentRecord.getRecordKey().contentEquals(keyFromResult)); + } else { //if commit is invalid, treat this as a new taggedRecord + taggedRecords.add(currentRecord); + } + } else { + taggedRecords.add(currentRecord); + } } - } else { - taggedRecords.add(currentRecord); } } - } - } - } catch (IOException e) { - throw new HoodieIndexException( - "Failed to Tag indexed locations because of exception with HBase Client", e); - } finally { - if (hTable != null) { - try { - hTable.close(); } catch (IOException e) { - // Ignore - } - } + throw new HoodieIndexException( + "Failed to Tag indexed locations because of exception with HBase Client", e); + } finally { + if (hTable != null) { + try { + hTable.close(); + } catch (IOException e) { + // Ignore + } + } - } - return taggedRecords.iterator(); - }; + } + return taggedRecords.iterator(); + }; } @Override public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) { + HoodieTable hoodieTable) { return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true); } - private Function2, Iterator> updateLocationFunction() { - - return (Function2, Iterator>) (partition, statusIterator) -> { - + private Function2, Iterator> + updateLocationFunction() { + return (Function2, Iterator>) (partition, + statusIterator) -> { Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); List writeStatusList = new ArrayList<>(); @@ -292,16 +291,13 @@ private Function2, Iterator> updateL /** * Helper method to facilitate performing puts and deletes in Hbase - * @param hTable - * @param puts - * @param deletes - * @throws IOException */ - private void doPutsAndDeletes(HTable hTable, List puts, List deletes) throws IOException { - if(puts.size() > 0) { + private void doPutsAndDeletes(HTable hTable, List puts, List deletes) + throws IOException { + if (puts.size() > 0) { hTable.put(puts); } - if(deletes.size() > 0) { + if (deletes.size() > 0) { hTable.delete(deletes); } hTable.flushCommits(); @@ -311,7 +307,7 @@ private void doPutsAndDeletes(HTable hTable, List puts, List delete @Override public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) { + HoodieTable hoodieTable) { return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); } @@ -323,7 +319,6 @@ public boolean rollbackCommit(String commitTime) { /** * Only looks up by recordKey - * @return */ @Override public boolean isGlobal() { @@ -332,7 +327,6 @@ public boolean isGlobal() { /** * Mapping is available in HBase already. - * @return */ @Override public boolean canIndexLogFiles() { @@ -341,7 +335,6 @@ public boolean canIndexLogFiles() { /** * Index needs to be explicitly updated after storage write. - * @return */ @Override public boolean isImplicitWithStorage() { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 4e62818b16d38..82d5a9d0b74f5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -37,14 +37,6 @@ import com.uber.hoodie.exception.HoodieAppendException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.table.HoodieTable; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; -import org.apache.spark.util.SizeEstimator; - import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; @@ -53,6 +45,13 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.util.SizeEstimator; /** * IO Operation to append data onto an existing file. @@ -61,14 +60,13 @@ public class HoodieAppendHandle extends HoodieIOH private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); private static AtomicLong recordIndex = new AtomicLong(1); - - private TableFileSystemView.RealtimeView fileSystemView; private final WriteStatus writeStatus; private final String fileId; - private String partitionPath; - private Iterator> recordItr; List recordList = new ArrayList<>(); List keysToDelete = new ArrayList<>(); + private TableFileSystemView.RealtimeView fileSystemView; + private String partitionPath; + private Iterator> recordItr; private long recordsWritten = 0; private long recordsDeleted = 0; private long averageRecordSize = 0; @@ -76,11 +74,8 @@ public class HoodieAppendHandle extends HoodieIOH private Writer writer; private boolean doInit = true; - public HoodieAppendHandle(HoodieWriteConfig config, - String commitTime, - HoodieTable hoodieTable, - String fileId, - Iterator> recordItr) { + public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String fileId, Iterator> recordItr) { super(config, commitTime, hoodieTable); WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieDeltaWriteStat()); @@ -93,42 +88,40 @@ public HoodieAppendHandle(HoodieWriteConfig config, private void init(String partitionPath) { // extract some information from the first record - FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)) - .findFirst().get(); - // HACK(vc) This also assumes a base file. It will break, if appending without one. - String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); - String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); - writeStatus.getStat().setPrevCommit(baseCommitTime); - writeStatus.setFileId(fileId); - writeStatus.setPartitionPath(partitionPath); - writeStatus.getStat().setFileId(fileId); - this.partitionPath = partitionPath; - - try { - this.writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) - .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles() - .map(logFile -> logFile.getLogVersion()) - .max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) - .withSizeThreshold(config.getLogFileMaxSize()) - .withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - this.currentLogFile = writer.getLogFile(); - ((HoodieDeltaWriteStat) writeStatus.getStat()) - .setLogVersion(currentLogFile.getLogVersion()); - ((HoodieDeltaWriteStat) writeStatus.getStat()) - .setLogOffset(writer.getCurrentSize()); - } catch (Exception e) { - logger.error("Error in update task at commit " + commitTime, e); - writeStatus.setGlobalError(e); - throw new HoodieUpsertException( - "Failed to initialize HoodieUpdateHandle for FileId: " + fileId - + " on commit " + commitTime + " on HDFS path " + hoodieTable - .getMetaClient().getBasePath() + partitionPath, e); - } - Path path = new Path(partitionPath, - FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); - writeStatus.getStat().setPath(path.toString()); + FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath) + .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst() + .get(); + // HACK(vc) This also assumes a base file. It will break, if appending without one. + String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); + String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); + writeStatus.getStat().setPrevCommit(baseCommitTime); + writeStatus.setFileId(fileId); + writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setFileId(fileId); + this.partitionPath = partitionPath; + + try { + this.writer = HoodieLogFormat.newWriterBuilder() + .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) + .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion( + fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion()) + .max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) + .withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + this.currentLogFile = writer.getLogFile(); + ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); + ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); + } catch (Exception e) { + logger.error("Error in update task at commit " + commitTime, e); + writeStatus.setGlobalError(e); + throw new HoodieUpsertException( + "Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " + + commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + + partitionPath, e); + } + Path path = new Path(partitionPath, + FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); + writeStatus.getStat().setPath(path.toString()); } private Optional getIndexedRecord(HoodieRecord hoodieRecord) { @@ -150,7 +143,8 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { } writeStatus.markSuccess(hoodieRecord, recordMetadata); - // deflate record payload after recording success. This will help users access payload as a part of marking + // deflate record payload after recording success. This will help users access payload as a + // part of marking // record successful. hoodieRecord.deflate(); return avroRecord; @@ -165,7 +159,8 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { // to make sure we don't append records with older (shorter) schema than already appended public void doAppend() { - int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0; + int maxBlockSize = config.getLogFileDataBlockMaxSize(); + int numberOfRecords = 0; Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -173,16 +168,17 @@ public void doAppend() { HoodieRecord record = recordItr.next(); // update the new location of the record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); - if(doInit) { + if (doInit) { init(record.getPartitionPath()); averageRecordSize = SizeEstimator.estimate(record); doInit = false; } // Append if max number of records reached to achieve block size - if(numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { - // Recompute averageRecordSize before writing a new block and update existing value with avg of new and old + if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { + // Recompute averageRecordSize before writing a new block and update existing value with + // avg of new and old logger.info("AvgRecordSize => " + averageRecordSize); - averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2; + averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2; doAppend(header); numberOfRecords = 0; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index 85028ba292fcb..c748dfe849ab7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -68,9 +68,8 @@ private List getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException { logger.info("Cleaning " + partitionPath + ", retaining latest " + config .getCleanerFileVersionsRetained() + " file versions. "); - List fileGroups = - fileSystemView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fileSystemView.getAllFileGroups(partitionPath) + .collect(Collectors.toList()); List deletePaths = new ArrayList<>(); // Collect all the datafiles savepointed by all the savepoints List savepointedFiles = hoodieTable.getSavepoints().stream() @@ -94,11 +93,9 @@ private List getFilesToCleanKeepingLatestVersions(String partitionPath) FileSlice nextSlice = fileSliceIterator.next(); HoodieDataFile dataFile = nextSlice.getDataFile().get(); deletePaths.add(dataFile.getFileStatus().getPath().toString()); - if (hoodieTable.getMetaClient().getTableType() - == HoodieTableType.MERGE_ON_READ) { + if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { // If merge on read, then clean the log files for the commits as well - deletePaths.addAll(nextSlice.getLogFiles() - .map(file -> file.getPath().toString()) + deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString()) .collect(Collectors.toList())); } } @@ -121,8 +118,8 @@ private List getFilesToCleanKeepingLatestVersions(String partitionPath) private List getFilesToCleanKeepingLatestCommits(String partitionPath) throws IOException { int commitsRetained = config.getCleanerCommitsRetained(); - logger.info( - "Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. "); + logger + .info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. "); List deletePaths = new ArrayList<>(); // Collect all the datafiles savepointed by all the savepoints @@ -132,15 +129,14 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) // determine if we have enough commits, to start cleaning. if (commitTimeline.countInstants() > commitsRetained) { HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get(); - List fileGroups = - fileSystemView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fileSystemView.getAllFileGroups(partitionPath) + .collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { List fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList()); HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get(); String lastVersion = dataFile.getCommitTime(); - String lastVersionBeforeEarliestCommitToRetain = - getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain); + String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList, + earliestCommitToRetain); // Ensure there are more than 1 version of the file (we only clean old files from updates) // i.e always spare the last commit. @@ -151,28 +147,26 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) // do not clean up a savepoint data file continue; } - // Dont delete the latest commit and also the last commit before the earliest commit we are retaining - // The window of commit retain == max query run time. So a query could be running which still + // Dont delete the latest commit and also the last commit before the earliest commit we + // are retaining + // The window of commit retain == max query run time. So a query could be running which + // still // uses this file. - if (fileCommitTime.equals(lastVersion) || ( - lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime - .equals(lastVersionBeforeEarliestCommitToRetain))) { + if (fileCommitTime.equals(lastVersion) || (lastVersionBeforeEarliestCommitToRetain != null + && fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) { // move on to the next file continue; } // Always keep the last commit - if (HoodieTimeline.compareTimestamps( - earliestCommitToRetain.getTimestamp(), - fileCommitTime, - HoodieTimeline.GREATER)) { + if (HoodieTimeline + .compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, + HoodieTimeline.GREATER)) { // this is a commit, that should be cleaned. deletePaths.add(aFile.getFileStatus().getPath().toString()); - if (hoodieTable.getMetaClient().getTableType() - == HoodieTableType.MERGE_ON_READ) { + if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { // If merge on read, then clean the log files for the commits as well - deletePaths.addAll(aSlice.getLogFiles() - .map(file -> file.getPath().toString()) + deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString()) .collect(Collectors.toList())); } } @@ -190,9 +184,10 @@ private String getLatestVersionBeforeCommit(List fileSliceList, HoodieInstant commitTime) { for (FileSlice file : fileSliceList) { String fileCommitTime = file.getDataFile().get().getCommitTime(); - if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, - HoodieTimeline.GREATER)) { - // fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want + if (HoodieTimeline + .compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) { + // fileList is sorted on the reverse, so the first commit we find <= commitTime is the + // one we want return fileCommitTime; } } @@ -213,8 +208,7 @@ public List getDeletePaths(String partitionPath) throws IOException { } else { throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name()); } - logger.info( - deletePaths.size() + " patterns used to delete in partition path:" + partitionPath); + logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath); return deletePaths; } @@ -227,8 +221,8 @@ public Optional getEarliestCommitToRetain() { int commitsRetained = config.getCleanerCommitsRetained(); if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS && commitTimeline.countInstants() > commitsRetained) { - earliestCommitToRetain = - commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained); + earliestCommitToRetain = commitTimeline + .nthInstant(commitTimeline.countInstants() - commitsRetained); } return earliestCommitToRetain; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index d0454e3d238b6..10d9f94cc0019 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -40,13 +40,6 @@ import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileStream; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -54,6 +47,11 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** * Archiver to bound the growth of .commit files @@ -76,11 +74,9 @@ public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient me private HoodieLogFormat.Writer openWriter() { try { if (this.writer == null) { - return HoodieLogFormat.newWriterBuilder() - .onParentPath(archiveFilePath.getParent()) + return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent()) .withFileId(archiveFilePath.getName()) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) - .withFs(metaClient.getFs()) + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs()) .overBaseCommit("").build(); } else { return this.writer; @@ -136,21 +132,19 @@ private Stream getInstantsToArchive() { .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)) .filterCompletedInstants(); Stream instants = cleanAndRollbackTimeline.getInstants() - .collect(Collectors.groupingBy(s -> s.getAction())) - .entrySet() - .stream() - .map(i -> { + .collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> { if (i.getValue().size() > maxCommitsToKeep) { return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep); } else { return new ArrayList(); } - }) - .flatMap(i -> i.stream()); + }).flatMap(i -> i.stream()); - //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats + //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify + // with logic above to avoid Stream.concats HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); - // We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present. + // We cannot have any holes in the commit timeline. We cannot archive any commits which are + // made after the first savepoint present. Optional firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) { // Actually do the commits @@ -169,16 +163,14 @@ private boolean deleteArchivedInstants(List archivedInstants) { log.info("Deleting instants " + archivedInstants); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { - Path commitFile = - new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); + Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { if (metaClient.getFs().exists(commitFile)) { success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { - throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, - e); + throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); } } return success; @@ -186,8 +178,8 @@ private boolean deleteArchivedInstants(List archivedInstants) { public void archive(List instants) throws HoodieCommitException { try { - HoodieTimeline commitTimeline = - metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline() + .filterCompletedInstants(); Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); log.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); @@ -247,6 +239,8 @@ private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, archivedMetaWrapper.setActionType(ActionType.commit.name()); break; } + default: + throw new UnsupportedOperationException("Action not fully supported yet"); } return archivedMetaWrapper; } @@ -256,9 +250,8 @@ private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter( ObjectMapper mapper = new ObjectMapper(); //Need this to ignore other public get() methods mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = - mapper.convertValue(hoodieCommitMetadata, - com.uber.hoodie.avro.model.HoodieCommitMetadata.class); + com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper + .convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class); return avroMetaData; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 9c7fac34ceead..f16b2d21169b7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -49,8 +49,8 @@ public class HoodieCreateHandle extends HoodieIOH private long recordsWritten = 0; private long recordsDeleted = 0; - public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, - HoodieTable hoodieTable, String partitionPath) { + public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath) { super(config, commitTime, hoodieTable); this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName()); status.setFileId(UUID.randomUUID().toString()); @@ -64,14 +64,11 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, } try { - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, - commitTime, - new Path(config.getBasePath()), - new Path(config.getBasePath(), partitionPath)); + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, + new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath)); partitionMetadata.trySave(TaskContext.getPartitionId()); - this.storageWriter = - HoodieStorageWriterFactory - .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); + this.storageWriter = HoodieStorageWriterFactory + .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); } catch (IOException e) { throw new HoodieInsertException( "Failed to initialize HoodieStorageWriter for path " + getStorageWriterPath(), e); @@ -81,13 +78,12 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, /** * Determines whether we can accept the incoming records, into the current file, depending on - * + *

* - Whether it belongs to the same partitionPath as existing records - Whether the current file * written bytes lt max file size */ public boolean canWrite(HoodieRecord record) { - return storageWriter.canWrite() && record.getPartitionPath() - .equals(status.getPartitionPath()); + return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath()); } /** @@ -111,7 +107,8 @@ public void write(HoodieRecord record, Optional insertValue, recordsDeleted++; } status.markSuccess(record, recordMetadata); - // deflate record payload after recording success. This will help users access payload as a part of marking + // deflate record payload after recording success. This will help users access payload as a + // part of marking // record successful. record.deflate(); } catch (Throwable t) { @@ -126,9 +123,8 @@ public void write(HoodieRecord record, Optional insertValue, * Performs actions to durably, persist the current changes and returns a WriteStatus object */ public WriteStatus close() { - logger.info( - "Closing the file " + status.getFileId() + " as we are done with all the records " - + recordsWritten); + logger.info("Closing the file " + status.getFileId() + " as we are done with all the records " + + recordsWritten); try { storageWriter.close(); @@ -144,8 +140,7 @@ public WriteStatus close() { return status; } catch (IOException e) { - throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, - e); + throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index f9795ceae19a1..a0032de2b8a62 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -39,11 +39,10 @@ public abstract class HoodieIOHandle { protected final HoodieWriteConfig config; protected final FileSystem fs; protected final HoodieTable hoodieTable; - protected HoodieTimeline hoodieTimeline; protected final Schema schema; + protected HoodieTimeline hoodieTimeline; - public HoodieIOHandle(HoodieWriteConfig config, String commitTime, - HoodieTable hoodieTable) { + public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { this.commitTime = commitTime; this.config = config; this.fs = hoodieTable.getMetaClient().getFs(); @@ -52,6 +51,32 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, this.schema = createHoodieWriteSchema(config); } + /** + * Deletes any new tmp files written during the current commit, into the partition + */ + public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, String commitTime, + String partitionPath, int taskPartitionId, HoodieTable hoodieTable) { + FileSystem fs = hoodieTable.getMetaClient().getFs(); + try { + FileStatus[] prevFailedFiles = fs.globStatus(new Path(String + .format("%s/%s/%s", config.getBasePath(), partitionPath, + FSUtils.maskWithoutFileId(commitTime, taskPartitionId)))); + if (prevFailedFiles != null) { + logger.info( + "Deleting " + prevFailedFiles.length + " files generated by previous failed attempts."); + for (FileStatus status : prevFailedFiles) { + fs.delete(status.getPath(), false); + } + } + } catch (IOException e) { + throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime, e); + } + } + + public static Schema createHoodieWriteSchema(HoodieWriteConfig config) { + return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + } + public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) { Path path = new Path(config.getBasePath(), partitionPath); try { @@ -72,37 +97,7 @@ public Path makeTempPath(String partitionPath, int taskPartitionId, String fileN taskAttemptId)); } - /** - * Deletes any new tmp files written during the current commit, into the partition - */ - public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, - String commitTime, - String partitionPath, - int taskPartitionId, - HoodieTable hoodieTable) { - FileSystem fs = hoodieTable.getMetaClient().getFs(); - try { - FileStatus[] prevFailedFiles = fs.globStatus(new Path(String - .format("%s/%s/%s", config.getBasePath(), partitionPath, - FSUtils.maskWithoutFileId(commitTime, taskPartitionId)))); - if (prevFailedFiles != null) { - logger.info("Deleting " + prevFailedFiles.length - + " files generated by previous failed attempts."); - for (FileStatus status : prevFailedFiles) { - fs.delete(status.getPath(), false); - } - } - } catch (IOException e) { - throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime, - e); - } - } - public Schema getSchema() { return schema; } - - public static Schema createHoodieWriteSchema(HoodieWriteConfig config) { - return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); - } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index c5ee6da8bd772..dc2930aae477e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -26,14 +26,18 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.io.storage.HoodieStorageWriter; import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -41,11 +45,6 @@ import org.apache.log4j.Logger; import org.apache.spark.TaskContext; -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; -import java.util.Optional; - @SuppressWarnings("Duplicates") public class HoodieMergeHandle extends HoodieIOHandle { @@ -62,59 +61,46 @@ public class HoodieMergeHandle extends HoodieIOHa private long recordsDeleted = 0; private long updatedRecordsWritten = 0; - public HoodieMergeHandle(HoodieWriteConfig config, - String commitTime, - HoodieTable hoodieTable, - Iterator> recordItr, - String fileId) { + public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + Iterator> recordItr, String fileId) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); init(fileId, init(fileId, recordItr)); } - public HoodieMergeHandle(HoodieWriteConfig config, - String commitTime, - HoodieTable hoodieTable, - Map> keyToNewRecords, - String fileId) { + public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + Map> keyToNewRecords, String fileId) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); this.keyToNewRecords = keyToNewRecords; - init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath()); + init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()) + .getPartitionPath()); } /** * Extract old file path, initialize StorageWriter and WriteStatus - * @param fileId - * @param partitionPath */ private void init(String fileId, String partitionPath) { WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieWriteStat()); this.writeStatus = writeStatus; try { - String latestValidFilePath = fileSystemView - .getLatestDataFiles(partitionPath) - .filter(dataFile -> dataFile.getFileId().equals(fileId)) - .findFirst() - .get().getFileName(); + String latestValidFilePath = fileSystemView.getLatestDataFiles(partitionPath) + .filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst().get().getFileName(); writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, - commitTime, - new Path(config.getBasePath()), - new Path(config.getBasePath(), partitionPath)); + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, + new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath)); partitionMetadata.trySave(TaskContext.getPartitionId()); oldFilePath = new Path( - config.getBasePath() + "/" + partitionPath + "/" - + latestValidFilePath); + config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath); String relativePath = new Path(partitionPath + "/" + FSUtils .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); newFilePath = new Path(config.getBasePath(), relativePath); if (config.shouldUseTempFolderForCopyOnWriteForMerge()) { - this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), - fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); + this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), fileId, + TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); } // handle cases of partial failures, for update task @@ -122,8 +108,9 @@ private void init(String fileId, String partitionPath) { fs.delete(newFilePath, false); } - logger.info(String.format("Merging new data into oldPath %s, as newPath %s", - oldFilePath.toString(), getStorageWriterPath().toString())); + logger.info(String + .format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(), + getStorageWriterPath().toString())); // file name is same for all records, in this bunch writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); @@ -143,9 +130,6 @@ private void init(String fileId, String partitionPath) { /** * Load the new incoming records in a map and return partitionPath - * @param fileId - * @param newRecordsItr - * @return */ private String init(String fileId, Iterator> newRecordsItr) { try { @@ -153,7 +137,7 @@ private String init(String fileId, Iterator> newRecordsItr) { logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass())); - } catch(IOException io) { + } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } String partitionPath = null; @@ -164,14 +148,14 @@ private String init(String fileId, Iterator> newRecordsItr) { // update the new location of the record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); } - logger.debug("Number of entries in MemoryBasedMap => " + - ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() - + "Total size in bytes of MemoryBasedMap => " + - ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() - + "Number of entries in DiskBasedMap => " + - ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() - + "Size of file spilled to disk => " + - ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); + logger.debug("Number of entries in MemoryBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() + + "Total size in bytes of MemoryBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + + "Number of entries in DiskBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + + "Size of file spilled to disk => " + + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); return partitionPath; } @@ -189,7 +173,8 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, } writeStatus.markSuccess(hoodieRecord, recordMetadata); - // deflate record payload after recording success. This will help users access payload as a part of marking + // deflate record payload after recording success. This will help users access payload as a + // part of marking // record successful. hoodieRecord.deflate(); return true; @@ -201,8 +186,7 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, } /** - * Go through an old record. Here if we detect a newer version shows up, we write the new one to - * the file. + * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file. */ public void write(GenericRecord oldRecord) { String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); @@ -213,12 +197,12 @@ public void write(GenericRecord oldRecord) { Optional combinedAvroRecord = hoodieRecord.getData() .combineAndGetUpdateValue(oldRecord, schema); if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) { - /* ONLY WHEN - * 1) we have an update for this key AND - * 2) We are able to successfully write the the combined new value - * - * We no longer need to copy the old record over. - */ + /* ONLY WHEN + * 1) we have an update for this key AND + * 2) We are able to successfully write the the combined new value + * + * We no longer need to copy the old record over. + */ copyOldRecord = false; } keyToNewRecords.remove(key); @@ -236,10 +220,9 @@ public void write(GenericRecord oldRecord) { try { storageWriter.writeAvro(key, oldRecord); } catch (ClassCastException e) { - logger.error( - "Schema mismatch when rewriting old record " + oldRecord + " from file " - + getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema - .toString(true)); + logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + + getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema + .toString(true)); throw new HoodieUpsertException(errMsg, e); } catch (IOException e) { logger.error("Failed to merge old record into new file for key " + key + " from old file " diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java index 7304c11d88f82..8180f7dc18a43 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java @@ -53,8 +53,8 @@ public CompactionOperation(HoodieDataFile dataFile, String partitionPath, this.partitionPath = partitionPath; this.dataFileCommitTime = dataFile.getCommitTime(); this.dataFileSize = dataFile.getFileSize(); - this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()).collect( - Collectors.toList()); + this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) + .collect(Collectors.toList()); this.metrics = writeConfig.getCompactionStrategy() .captureMetrics(dataFile, partitionPath, logFiles); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index 9011a82b29df3..7475176c44efa 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -17,17 +17,15 @@ package com.uber.hoodie.io.compact; import com.uber.hoodie.WriteStatus; -import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - import java.io.Serializable; import java.util.Date; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; /** * A HoodieCompactor runs compaction on a hoodie table @@ -38,7 +36,7 @@ public interface HoodieCompactor extends Serializable { * Compact the delta files with the data files */ JavaRDD compact(JavaSparkContext jsc, final HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws Exception; + HoodieTable hoodieTable, String compactionCommitTime) throws Exception; // Helper methods diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 094ee0200e129..8d4f9b7a319d3 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -17,6 +17,7 @@ package com.uber.hoodie.io.compact; import static java.util.stream.Collectors.toList; + import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -70,9 +71,8 @@ public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig conf } private JavaRDD executeCompaction(JavaSparkContext jsc, - List operations, - HoodieTable hoodieTable, - HoodieWriteConfig config, String compactionCommitTime) throws IOException { + List operations, HoodieTable hoodieTable, HoodieWriteConfig config, + String compactionCommitTime) throws IOException { log.info("After filtering, Compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) @@ -80,18 +80,19 @@ private JavaRDD executeCompaction(JavaSparkContext jsc, .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); } - private List compact(HoodieTable hoodieTable, - HoodieWriteConfig config, CompactionOperation operation, String commitTime) - throws IOException { + private List compact(HoodieTable hoodieTable, HoodieWriteConfig config, + CompactionOperation operation, String commitTime) throws IOException { FileSystem fs = hoodieTable.getMetaClient().getFs(); - Schema readerSchema = - HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + Schema readerSchema = HoodieAvroUtils + .addMetadataFields(new Schema.Parser().parse(config.getSchema())); log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation .getDeltaFilePaths() + " for commit " + commitTime); // TODO - FIX THIS - // Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover). - // Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader + // Reads the entire avro file. Always only specific blocks should be read from the avro file + // (failure recover). + // Load all the delta commits since the last compaction commit and get all the blocks to be + // loaded and load it using CompositeAvroLogReader // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon. HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); String maxInstantTime = metaClient.getActiveTimeline() @@ -114,52 +115,47 @@ private List compact(HoodieTable hoodieTable, Iterator> result = table .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); Iterable> resultIterable = () -> result; - return StreamSupport.stream(resultIterable.spliterator(), false) - .flatMap(Collection::stream) + return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { s.getStat().setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate()); s.getStat().setTotalLogFiles(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); s.getStat().setPartitionPath(operation.getPartitionPath()); return s; - }) - .collect(toList()); + }).collect(toList()); } private List getCompactionWorkload(JavaSparkContext jsc, - HoodieTable hoodieTable, - HoodieWriteConfig config, String compactionCommitTime) + HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) throws IOException { - Preconditions.checkArgument( - hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, - "HoodieRealtimeTableCompactor can only compact table of type " - + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() - .getTableType().name()); + Preconditions + .checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, + "HoodieRealtimeTableCompactor can only compact table of type " + + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() + .getTableType().name()); //TODO : check if maxMemory is not greater than JVM or spark.executor memory // TODO - rollback any compactions in flight HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); - List partitionPaths = - FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), + List partitionPaths = FSUtils + .getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - List operations = - jsc.parallelize(partitionPaths, partitionPaths.size()) - .flatMap((FlatMapFunction) partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath) - .map(s -> new CompactionOperation(s.getDataFile().get(), - partitionPath, + List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) + .flatMap((FlatMapFunction) partitionPath -> fileSystemView + .getLatestFileSlices(partitionPath).map( + s -> new CompactionOperation(s.getDataFile().get(), partitionPath, s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed()) .collect(Collectors.toList()), config)) - .filter(c -> !c.getDeltaFilePaths().isEmpty()) - .collect(toList()).iterator()).collect(); + .filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator()).collect(); log.info("Total of " + operations.size() + " compactions are retrieved"); - // Filter the compactions with the passed in filter. This lets us choose most effective compactions only + // Filter the compactions with the passed in filter. This lets us choose most effective + // compactions only operations = config.getCompactionStrategy().orderAndFilter(config, operations); if (operations.isEmpty()) { log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java index 676b3e10e5377..9393a04790808 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java @@ -44,9 +44,9 @@ public Map captureMetrics(HoodieDataFile dataFile, String partit List logFiles) { Map metrics = Maps.newHashMap(); // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter( - Optional::isPresent).map(Optional::get).reduce( - (size1, size2) -> size1 + size2).orElse(0L); + Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) + .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) + .orElse(0L); // Total read will be the base file + all the log files Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize); // Total write will be similar to the size of the base file @@ -64,7 +64,8 @@ public Map captureMetrics(HoodieDataFile dataFile, String partit @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations) { - // Iterate through the operations in order and accept operations as long as we are within the IO limit + // Iterate through the operations in order and accept operations as long as we are within the + // IO limit // Preserves the original ordering of compactions List finalOperations = Lists.newArrayList(); long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index 8486774d7d6cb..c5a666cbd8f52 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -28,7 +28,7 @@ * Strategy for compaction. Pluggable implementation of define how compaction should be done. The * implementations of this interface can capture the relevant metrics to order and filter the final * list of compaction operation to run in a single compaction. - * + *

* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be * passed in every time * @@ -41,9 +41,9 @@ public interface CompactionStrategy extends Serializable { * Callback hook when a CompactionOperation is created. Individual strategies can capture the * metrics they need to decide on the priority. * - * @param dataFile - Base file to compact + * @param dataFile - Base file to compact * @param partitionPath - Partition path - * @param logFiles - List of log files to compact with the base file + * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ Map captureMetrics(HoodieDataFile dataFile, String partitionPath, @@ -54,7 +54,7 @@ Map captureMetrics(HoodieDataFile dataFile, String partitionPath * order and filter out compactions * * @param writeConfig - HoodieWriteConfig - config for this compaction is passed in - * @param operations - list of compactions collected + * @param operations - list of compactions collected * @return list of compactions to perform in this run */ List orderAndFilter(HoodieWriteConfig writeConfig, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java index 5aa9678694d52..3e50459521481 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java @@ -20,7 +20,6 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.io.compact.CompactionOperation; - import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Comparator; @@ -30,20 +29,21 @@ import java.util.stream.Collectors; /** - * This strategy orders compactions in reverse order of creation of Hive Partitions. - * It helps to compact data in latest partitions first and then older capped at the Total_IO allowed. + * This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to + * compact data in latest partitions first and then older capped at the Total_IO allowed. */ public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy { // For now, use SimpleDateFormat as default partition format private static String datePartitionFormat = "yyyy/MM/dd"; // Sorts compaction in LastInFirstCompacted order - private static Comparator comparator = (CompactionOperation leftC, CompactionOperation rightC) -> { + private static Comparator comparator = (CompactionOperation leftC, + CompactionOperation rightC) -> { try { Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH) - .parse(leftC.getPartitionPath()); + .parse(leftC.getPartitionPath()); Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH) - .parse(rightC.getPartitionPath()); + .parse(rightC.getPartitionPath()); return left.after(right) ? -1 : right.after(left) ? 1 : 0; } catch (ParseException e) { throw new HoodieException("Invalid Partition Date Format", e); @@ -55,8 +55,10 @@ public Comparator getComparator() { } @Override - public List orderAndFilter(HoodieWriteConfig writeConfig, List operations) { + public List orderAndFilter(HoodieWriteConfig writeConfig, + List operations) { // Iterate through the operations and accept operations as long as we are within the IO limit - return super.orderAndFilter(writeConfig, operations.stream().sorted(comparator).collect(Collectors.toList())); + return super.orderAndFilter(writeConfig, + operations.stream().sorted(comparator).collect(Collectors.toList())); } } \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java index 2fabc6596ebb0..a7f729e36cf9b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -44,9 +44,9 @@ public Map captureMetrics(HoodieDataFile dataFile, String partit Map metrics = super.captureMetrics(dataFile, partitionPath, logFiles); // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter( - Optional::isPresent).map(Optional::get).reduce( - (size1, size2) -> size1 + size2).orElse(0L); + Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) + .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) + .orElse(0L); // save the metrics needed during the order metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize); return metrics; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java index 9978c074737b3..edd2c062038e5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java @@ -36,8 +36,8 @@ * HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides * a way to check if the current file can take more records with the canWrite() */ -public class HoodieParquetWriter - extends ParquetWriter implements HoodieStorageWriter { +public class HoodieParquetWriter extends + ParquetWriter implements HoodieStorageWriter { private static AtomicLong recordIndex = new AtomicLong(1); @@ -49,29 +49,22 @@ public class HoodieParquetWriter HoodieStorageWriter getStorageWriter( - String commitTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, - Schema schema) - throws IOException { + public static HoodieStorageWriter + getStorageWriter(String commitTime, Path path, HoodieTable hoodieTable, + HoodieWriteConfig config, Schema schema) throws IOException { //TODO - based on the metadata choose the implementation of HoodieStorageWriter // Currently only parquet is supported return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable); } - private static HoodieStorageWriter newParquetStorageWriter( + private static HoodieStorageWriter + newParquetStorageWriter( String commitTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException { - BloomFilter filter = - new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP()); - HoodieAvroWriteSupport writeSupport = - new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter); + BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), + config.getBloomFilterFPP()); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( + new AvroSchemaConverter().convert(schema), schema, filter); HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java index a9d2a2298e910..390e73abedfb8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java @@ -59,8 +59,8 @@ */ public class HoodieWrapperFileSystem extends FileSystem { - private static final Set SUPPORT_SCHEMES; public static final String HOODIE_SCHEME_PREFIX = "hoodie-"; + private static final Set SUPPORT_SCHEMES; static { SUPPORT_SCHEMES = new HashSet<>(); @@ -69,18 +69,50 @@ public class HoodieWrapperFileSystem extends FileSystem { SUPPORT_SCHEMES.add("s3"); SUPPORT_SCHEMES.add("s3a"); - // Hoodie currently relies on underlying object store being fully // consistent so only regional buckets should be used. SUPPORT_SCHEMES.add("gs"); SUPPORT_SCHEMES.add("viewfs"); } - private ConcurrentMap openStreams = - new ConcurrentHashMap<>(); + private ConcurrentMap openStreams = new + ConcurrentHashMap<>(); private FileSystem fileSystem; private URI uri; + public static Path convertToHoodiePath(Path file, Configuration conf) { + try { + String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); + return convertPathWithScheme(file, getHoodieScheme(scheme)); + } catch (HoodieIOException e) { + throw e; + } + } + + private static Path convertPathWithScheme(Path oldPath, String newScheme) { + URI oldURI = oldPath.toUri(); + URI newURI; + try { + newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(), + oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment()); + return new Path(newURI); + } catch (URISyntaxException e) { + // TODO - Better Exception handling + throw new RuntimeException(e); + } + } + + public static String getHoodieScheme(String scheme) { + String newScheme; + if (SUPPORT_SCHEMES.contains(scheme)) { + newScheme = HOODIE_SCHEME_PREFIX + scheme; + } else { + throw new IllegalArgumentException( + "BlockAlignedAvroParquetWriter does not support scheme " + scheme); + } + return newScheme; + } + @Override public void initialize(URI uri, Configuration conf) throws IOException { // Get the default filesystem to decorate @@ -90,7 +122,8 @@ public void initialize(URI uri, Configuration conf) throws IOException { path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, "")); } this.fileSystem = FSUtils.getFs(path.toString(), conf); - // Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get + // Do not need to explicitly initialize the default filesystem, its done already in the above + // FileSystem.get // fileSystem.initialize(FileSystem.getDefaultUri(conf), conf); // fileSystem.setConf(conf); this.uri = uri; @@ -108,8 +141,7 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException { @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, - int bufferSize, short replication, long blockSize, Progressable progress) - throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { final Path translatedPath = convertToDefaultPath(f); return wrapOutputStream(f, fileSystem .create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, @@ -122,8 +154,8 @@ private FSDataOutputStream wrapOutputStream(final Path path, return fsDataOutputStream; } - SizeAwareFSDataOutputStream os = - new SizeAwareFSDataOutputStream(fsDataOutputStream, new Runnable() { + SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream, + new Runnable() { @Override public void run() { openStreams.remove(path.getName()); @@ -160,14 +192,13 @@ public FSDataOutputStream create(Path f, short replication, Progressable progres } @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) - throws IOException { + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize); } @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, - Progressable progress) throws IOException { + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) + throws IOException { return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress); } @@ -175,14 +206,12 @@ public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { return fileSystem - .create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, - progress); + .create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, - int bufferSize, short replication, long blockSize, Progressable progress) - throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { return fileSystem .create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress); @@ -197,7 +226,6 @@ public FSDataOutputStream create(Path f, FsPermission permission, EnumSet listCorruptFileBlocks(Path path) throws IOException } @Override - public FileStatus[] listStatus(Path f, PathFilter filter) - throws IOException { + public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { return fileSystem.listStatus(convertToDefaultPath(f), filter); } @Override - public FileStatus[] listStatus(Path[] files) - throws IOException { + public FileStatus[] listStatus(Path[] files) throws IOException { return fileSystem.listStatus(convertDefaults(files)); } @Override - public FileStatus[] listStatus(Path[] files, PathFilter filter) - throws IOException { + public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { return fileSystem.listStatus(convertDefaults(files), filter); } @@ -441,20 +463,17 @@ public FileStatus[] globStatus(Path pathPattern) throws IOException { } @Override - public FileStatus[] globStatus(Path pathPattern, PathFilter filter) - throws IOException { + public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter); } @Override - public RemoteIterator listLocatedStatus(Path f) - throws IOException { + public RemoteIterator listLocatedStatus(Path f) throws IOException { return fileSystem.listLocatedStatus(convertToDefaultPath(f)); } @Override - public RemoteIterator listFiles(Path f, boolean recursive) - throws IOException { + public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { return fileSystem.listFiles(convertToDefaultPath(f), recursive); } @@ -498,8 +517,8 @@ public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Pa @Override public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException { - fileSystem.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), - convertToDefaultPath(dst)); + fileSystem + .copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst)); } @Override @@ -525,15 +544,13 @@ public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLo } @Override - public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) - throws IOException { - return convertToHoodiePath(fileSystem.startLocalOutput(convertToDefaultPath(fsOutputFile), - convertToDefaultPath(tmpLocalFile))); + public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException { + return convertToHoodiePath(fileSystem + .startLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile))); } @Override - public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) - throws IOException { + public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException { fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile)); } @@ -574,23 +591,18 @@ public short getDefaultReplication(Path path) { } @Override - public void access(Path path, FsAction mode) - throws IOException { + public void access(Path path, FsAction mode) throws IOException { fileSystem.access(convertToDefaultPath(path), mode); } @Override - public void createSymlink(Path target, Path link, boolean createParent) - throws - IOException { + public void createSymlink(Path target, Path link, boolean createParent) throws IOException { fileSystem .createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent); } @Override - public FileStatus getFileLinkStatus(Path f) - throws - IOException { + public FileStatus getFileLinkStatus(Path f) throws IOException { return fileSystem.getFileLinkStatus(convertToDefaultPath(f)); } @@ -651,8 +663,7 @@ public void setTimes(Path p, long mtime, long atime) throws IOException { @Override public Path createSnapshot(Path path, String snapshotName) throws IOException { - return convertToHoodiePath( - fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName)); + return convertToHoodiePath(fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName)); } @Override @@ -718,8 +729,7 @@ public Map getXAttrs(Path path) throws IOException { } @Override - public Map getXAttrs(Path path, List names) - throws IOException { + public Map getXAttrs(Path path, List names) throws IOException { return fileSystem.getXAttrs(convertToDefaultPath(path), names); } @@ -734,13 +744,13 @@ public void removeXAttr(Path path, String name) throws IOException { } @Override - public void setConf(Configuration conf) { - // ignore this. we will set conf on init + public Configuration getConf() { + return fileSystem.getConf(); } @Override - public Configuration getConf() { - return fileSystem.getConf(); + public void setConf(Configuration conf) { + // ignore this. we will set conf on init } @Override @@ -762,15 +772,6 @@ public Path convertToHoodiePath(Path oldPath) { return convertPathWithScheme(oldPath, getHoodieScheme(fileSystem.getScheme())); } - public static Path convertToHoodiePath(Path file, Configuration conf) { - try { - String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); - return convertPathWithScheme(file, getHoodieScheme(scheme)); - } catch (HoodieIOException e) { - throw e; - } - } - private Path convertToDefaultPath(Path oldPath) { return convertPathWithScheme(oldPath, fileSystem.getScheme()); } @@ -783,30 +784,6 @@ private Path[] convertDefaults(Path[] psrcs) { return psrcsNew; } - private static Path convertPathWithScheme(Path oldPath, String newScheme) { - URI oldURI = oldPath.toUri(); - URI newURI; - try { - newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(), - oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment()); - return new Path(newURI); - } catch (URISyntaxException e) { - // TODO - Better Exception handling - throw new RuntimeException(e); - } - } - - public static String getHoodieScheme(String scheme) { - String newScheme; - if (SUPPORT_SCHEMES.contains(scheme)) { - newScheme = HOODIE_SCHEME_PREFIX + scheme; - } else { - throw new IllegalArgumentException( - "BlockAlignedAvroParquetWriter does not support scheme " + scheme); - } - return newScheme; - } - public long getBytesWritten(Path file) { if (openStreams.containsKey(file.getName())) { return openStreams.get(file.getName()).getBytesWritten(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index fe275e1597395..78ef4960c025f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -30,14 +30,14 @@ */ public class HoodieMetrics { - private HoodieWriteConfig config = null; - private String tableName = null; private static Logger logger = LogManager.getLogger(HoodieMetrics.class); // Some timers public String rollbackTimerName = null; public String cleanTimerName = null; public String commitTimerName = null; public String finalizeTimerName = null; + private HoodieWriteConfig config = null; + private String tableName = null; private Timer rollbackTimer = null; private Timer cleanTimer = null; private Timer commitTimer = null; @@ -113,8 +113,9 @@ public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) { if (config.isMetricsOn()) { - logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", - durationInMs, numFilesDeleted)); + logger.info(String + .format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", durationInMs, + numFilesDeleted)); registerGauge(getMetricsName("rollback", "duration"), durationInMs); registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted); } @@ -122,8 +123,9 @@ public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) { public void updateCleanMetrics(long durationInMs, int numFilesDeleted) { if (config.isMetricsOn()) { - logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", - durationInMs, numFilesDeleted)); + logger.info(String + .format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs, + numFilesDeleted)); registerGauge(getMetricsName("clean", "duration"), durationInMs); registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted); } @@ -131,8 +133,9 @@ public void updateCleanMetrics(long durationInMs, int numFilesDeleted) { public void updateFinalizeWriteMetrics(long durationInMs, int numFilesFinalized) { if (config.isMetricsOn()) { - logger.info(String.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", - durationInMs, numFilesFinalized)); + logger.info(String + .format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", + durationInMs, numFilesFinalized)); registerGauge(getMetricsName("finalize", "duration"), durationInMs); registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized); } @@ -140,8 +143,7 @@ public void updateFinalizeWriteMetrics(long durationInMs, int numFilesFinalized) @VisibleForTesting String getMetricsName(String action, String metric) { - 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) { @@ -154,7 +156,8 @@ public Long getValue() { } }); } catch (Exception e) { - // Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system + // 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); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java index c5a3573174954..924257493592e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java @@ -40,7 +40,7 @@ private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException { if (reporter == null) { throw new RuntimeException("Cannot initialize Reporter."); } -// reporter.start(); + // reporter.start(); Runtime.getRuntime().addShutdownHook(new Thread() { @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java index bc7d024a59ebb..354693a891b3b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java @@ -33,14 +33,13 @@ */ public class MetricsGraphiteReporter extends MetricsReporter { + private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class); private final MetricRegistry registry; private final GraphiteReporter graphiteReporter; private final HoodieWriteConfig config; private String serverHost; private int serverPort; - private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class); - public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) { this.registry = registry; this.config = config; @@ -49,8 +48,8 @@ public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry this.serverHost = config.getGraphiteServerHost(); this.serverPort = config.getGraphiteServerPort(); if (serverHost == null || serverPort == 0) { - throw new RuntimeException( - String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].", + throw new RuntimeException(String + .format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].", serverHost, serverPort)); } @@ -81,14 +80,10 @@ public Closeable getReporter() { } private GraphiteReporter createGraphiteReport() { - Graphite graphite = new Graphite( - new InetSocketAddress(serverHost, serverPort)); + Graphite graphite = new Graphite(new InetSocketAddress(serverHost, serverPort)); String reporterPrefix = config.getGraphiteMetricPrefix(); - return GraphiteReporter.forRegistry(registry) - .prefixedWith(reporterPrefix) - .convertRatesTo(TimeUnit.SECONDS) - .convertDurationsTo(TimeUnit.MILLISECONDS) - .filter(MetricFilter.ALL) - .build(graphite); + return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix) + .convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS) + .filter(MetricFilter.ALL).build(graphite); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java index 31bc555c7e9be..0d6e64fe1bcd4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java @@ -28,8 +28,7 @@ public class MetricsReporterFactory { private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class); - public static MetricsReporter createReporter(HoodieWriteConfig config, - MetricRegistry registry) { + public static MetricsReporter createReporter(HoodieWriteConfig config, MetricRegistry registry) { MetricsReporterType type = config.getMetricsReporterType(); MetricsReporter reporter = null; switch (type) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java index 46e128ddb64fd..1b4667ded609b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java @@ -21,6 +21,5 @@ * future. */ public enum MetricsReporterType { - GRAPHITE, - INMEMORY + GRAPHITE, INMEMORY } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index ea80630fad118..db32a908d4586 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -75,139 +75,534 @@ /** * Implementation of a very heavily read-optimized Hoodie Table where - * + *

* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing * file, to expand it - * + *

* UPDATES - Produce a new version of the file, just replacing the updated records with new values */ public class HoodieCopyOnWriteTable extends HoodieTable { + private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); + public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { super(config, metaClient); } - private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); + private static PairFlatMapFunction>, String, + PartitionCleanStat> deleteFilesFunc( + HoodieTable table) { + return (PairFlatMapFunction>, String, PartitionCleanStat>) + iter -> { + Map partitionCleanStatMap = new HashMap<>(); + + FileSystem fs = table.getMetaClient().getFs(); + while (iter.hasNext()) { + Tuple2 partitionDelFileTuple = iter.next(); + String partitionPath = partitionDelFileTuple._1(); + String deletePathStr = partitionDelFileTuple._2(); + Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); + if (!partitionCleanStatMap.containsKey(partitionPath)) { + partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath)); + } + PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); + partitionCleanStat.addDeleteFilePatterns(deletePathStr); + partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult); + } - enum BucketType { - UPDATE, - INSERT + return partitionCleanStatMap.entrySet().stream() + .map(e -> new Tuple2<>(e.getKey(), e.getValue())) + .collect(Collectors.toList()).iterator(); + }; } - /** - * Helper class for a small file's location and its actual size on disk - */ - class SmallFile implements Serializable { + private static PairFlatMapFunction getFilesToDeleteFunc(HoodieTable table, + HoodieWriteConfig config) { + return (PairFlatMapFunction) partitionPathToClean -> { + HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config); + return cleaner.getDeletePaths(partitionPathToClean).stream() + .map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())).iterator(); + }; + } - HoodieRecordLocation location; - long sizeBytes; + private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) + throws IOException { + Path deletePath = new Path(deletePathStr); + logger.debug("Working on delete path :" + deletePath); + boolean deleteResult = fs.delete(deletePath, false); + if (deleteResult) { + logger.debug("Cleaned file at path :" + deletePath); + } + return deleteResult; + } - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("SmallFile {"); - sb.append("location=").append(location).append(", "); - sb.append("sizeBytes=").append(sizeBytes); - sb.append('}'); - return sb.toString(); + @Override + public Partitioner getUpsertPartitioner(WorkloadProfile profile) { + if (profile == null) { + throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); } + return new UpsertPartitioner(profile); } - /** - * Helper class for an insert bucket along with the weight [0.0, 0.1] that defines the amount of - * incoming inserts that should be allocated to the bucket - */ - class InsertBucket implements Serializable { + @Override + public Partitioner getInsertPartitioner(WorkloadProfile profile) { + return getUpsertPartitioner(profile); + } - int bucketNumber; - // fraction of total inserts, that should go into this bucket - double weight; + @Override + public boolean isWorkloadProfileNeeded() { + return true; + } - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("WorkloadStat {"); - sb.append("bucketNumber=").append(bucketNumber).append(", "); - sb.append("weight=").append(weight); - sb.append('}'); - return sb.toString(); - } + @Override + public JavaRDD compact(JavaSparkContext jsc, String commitTime) { + throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); } - /** - * Helper class for a bucket's type (INSERT and UPDATE) and its file location - */ - class BucketInfo implements Serializable { + public Iterator> handleUpdate(String commitTime, String fileLoc, + Iterator> recordItr) throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr); + return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + } - BucketType bucketType; - String fileLoc; + public Iterator> handleUpdate(String commitTime, String fileLoc, + Map> keyToNewRecords) throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords); + return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + } - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("BucketInfo {"); - sb.append("bucketType=").append(bucketType).append(", "); - sb.append("fileLoc=").append(fileLoc); - sb.append('}'); - return sb.toString(); + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, + String commitTime, String fileLoc) throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc); + } else { + AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); + ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) + .withConf(getHadoopConf()).build(); + try { + IndexedRecord record; + while ((record = reader.read()) != null) { + // Two types of writes here (new record, and old record). + // We have already catch the exception during writing new records. + // But for old records, we should fail if any exception happens. + upsertHandle.write((GenericRecord) record); + } + } catch (IOException e) { + throw new HoodieUpsertException( + "Failed to read record from " + upsertHandle.getOldFilePath() + " with new Schema " + + upsertHandle.getSchema(), e); + } finally { + reader.close(); + upsertHandle.close(); + } + } + //TODO(vc): This needs to be revisited + if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + + upsertHandle.getWriteStatus()); } + return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())) + .iterator(); } + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, + Iterator> recordItr) { + return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); + } - /** - * Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition) - */ - class UpsertPartitioner extends Partitioner { - - /** - * Total number of RDD partitions, is determined by total buckets we want to pack the incoming - * workload into - */ - private int totalBuckets = 0; + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, + Map> keyToNewRecords) { + return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc); + } - /** - * Stat for the current workload. Helps in determining total inserts, upserts etc. - */ - private WorkloadStat globalStat; + public Iterator> handleInsert(String commitTime, + Iterator> recordItr) throws Exception { + return new LazyInsertIterable<>(recordItr, config, commitTime, this); + } - /** - * Helps decide which bucket an incoming update should go to. - */ - private HashMap updateLocationToBucket; + @SuppressWarnings("unchecked") + @Override + public Iterator> handleUpsertPartition(String commitTime, Integer partition, + Iterator recordItr, Partitioner partitioner) { + UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; + BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); + BucketType btype = binfo.bucketType; + try { + if (btype.equals(BucketType.INSERT)) { + return handleInsert(commitTime, recordItr); + } else if (btype.equals(BucketType.UPDATE)) { + return handleUpdate(commitTime, binfo.fileLoc, recordItr); + } else { + throw new HoodieUpsertException( + "Unknown bucketType " + btype + " for partition :" + partition); + } + } catch (Throwable t) { + String msg = "Error upserting bucketType " + btype + " for partition :" + partition; + logger.error(msg, t); + throw new HoodieUpsertException(msg, t); + } + } + @Override + public Iterator> handleInsertPartition(String commitTime, Integer partition, + Iterator recordItr, Partitioner partitioner) { + return handleUpsertPartition(commitTime, partition, recordItr, partitioner); + } - /** - * Helps us pack inserts into 1 or more buckets depending on number of incoming records. - */ - private HashMap> partitionPathToInsertBuckets; + /** + * Performs cleaning of partition paths according to cleaning policy and returns the number of + * files cleaned. Handles skews in partitions to clean by making files to clean as the unit of + * task distribution. + * + * @throws IllegalArgumentException if unknown cleaning policy is provided + */ + @Override + public List clean(JavaSparkContext jsc) { + try { + FileSystem fs = getMetaClient().getFs(); + List partitionsToClean = FSUtils + .getAllPartitionPaths(fs, getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning()); + logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config + .getCleanerPolicy()); + if (partitionsToClean.isEmpty()) { + logger.info("Nothing to clean here mom. It is already clean"); + return Collections.emptyList(); + } + return cleanPartitionPaths(partitionsToClean, jsc); + } catch (IOException e) { + throw new HoodieIOException("Failed to clean up after commit", e); + } + } + /** + * Common method used for cleaning out parquet files under a partition path during rollback of a + * set of commits + */ + protected Map deleteCleanedFiles(String partitionPath, List commits) + throws IOException { + logger.info("Cleaning path " + partitionPath); + FileSystem fs = getMetaClient().getFs(); + FileStatus[] toBeDeleted = fs + .listStatus(new Path(config.getBasePath(), partitionPath), path -> { + if (!path.toString().contains(".parquet")) { + return false; + } + String fileCommitTime = FSUtils.getCommitTime(path.getName()); + return commits.contains(fileCommitTime); + }); + Map results = Maps.newHashMap(); + for (FileStatus file : toBeDeleted) { + boolean success = fs.delete(file.getPath(), false); + results.put(file, success); + logger.info("Delete file " + file.getPath() + "\t" + success); + } + return results; + } - /** - * Remembers what type each bucket is for later. - */ - private HashMap bucketInfoMap; + @Override + public List rollback(JavaSparkContext jsc, List commits) + throws IOException { + String actionType = this.getCommitActionType(); + HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); + List inflights = this.getInflightCommitTimeline().getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - /** - * List of all small files to be corrected - */ - List smallFiles = new ArrayList(); + // Atomically unpublish all the commits + commits.stream().filter(s -> !inflights.contains(s)) + .map(s -> new HoodieInstant(false, actionType, s)) + .forEach(activeTimeline::revertToInflight); + logger.info("Unpublished " + commits); - UpsertPartitioner(WorkloadProfile profile) { - updateLocationToBucket = new HashMap<>(); - partitionPathToInsertBuckets = new HashMap<>(); - bucketInfoMap = new HashMap<>(); - globalStat = profile.getGlobalStat(); + // delete all the data files for all these commits + logger.info("Clean out all parquet files generated for commits: " + commits); + List stats = jsc.parallelize(FSUtils + .getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning())) + .map((Function) partitionPath -> { + // Scan all partitions files with this commit time + Map results = deleteCleanedFiles(partitionPath, commits); + return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) + .withDeletedFileResults(results).build(); + }).collect(); - assignUpdates(profile); - assignInserts(profile); + // clean temporary data files + cleanTemporaryDataFiles(jsc); - logger.info("Total Buckets :" + totalBuckets + ", " + - "buckets info => " + bucketInfoMap + ", \n" + - "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n" + - "UpdateLocations mapped to buckets =>" + updateLocationToBucket); + // Remove the rolled back inflight commits + commits.stream().map(s -> new HoodieInstant(true, actionType, s)) + .forEach(activeTimeline::deleteInflight); + logger.info("Deleted inflight commits " + commits); + return stats; + } + + /** + * Finalize the written data files + * + * @param writeStatuses List of WriteStatus + * @return number of files finalized + */ + @Override + @SuppressWarnings("unchecked") + public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { + if (!config.shouldUseTempFolderForCopyOnWrite()) { + return Optional.empty(); + } + + // This is to rename each data file from temporary path to its final location + List> results = jsc + .parallelize(writeStatuses, config.getFinalizeWriteParallelism()).map(writeStatus -> { + Tuple2 writeStatTuple2 = (Tuple2) + writeStatus; + HoodieWriteStat writeStat = writeStatTuple2._2(); + final FileSystem fs = getMetaClient().getFs(); + final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); + + if (writeStat.getTempPath() != null) { + final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); + boolean success; + try { + logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); + success = fs.rename(tempPath, finalPath); + } catch (IOException e) { + throw new HoodieIOException( + "Failed to rename file: " + tempPath + " to " + finalPath); + } + + if (!success) { + throw new HoodieIOException( + "Failed to rename file: " + tempPath + " to " + finalPath); + } + } + + return new Tuple2<>(writeStat.getPath(), true); + }).collect(); + + // clean temporary data files + cleanTemporaryDataFiles(jsc); + + return Optional.of(results.size()); + } + + /** + * Clean temporary data files that are produced from previous failed commit or retried spark + * stages. + */ + private void cleanTemporaryDataFiles(JavaSparkContext jsc) { + if (!config.shouldUseTempFolderForCopyOnWrite()) { + return; + } + + final FileSystem fs = getMetaClient().getFs(); + final Path temporaryFolder = new Path(config.getBasePath(), + HoodieTableMetaClient.TEMPFOLDER_NAME); + try { + if (!fs.exists(temporaryFolder)) { + logger.info("Temporary folder does not exist: " + temporaryFolder); + return; + } + List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); + List> results = jsc + .parallelize(fileStatusesList, config.getFinalizeWriteParallelism()).map(fileStatus -> { + FileSystem fs1 = getMetaClient().getFs(); + boolean success = fs1.delete(fileStatus.getPath(), false); + logger + .info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + success); + return new Tuple2<>(fileStatus.getPath().toString(), success); + }).collect(); + + for (Tuple2 result : results) { + if (!result._2()) { + logger.info("Failed to delete file: " + result._1()); + throw new HoodieIOException("Failed to delete file in temporary folder: " + result._1()); + } + } + } catch (IOException e) { + throw new HoodieIOException( + "Failed to clean data files in temporary folder: " + temporaryFolder); + } + } + + private List cleanPartitionPaths(List partitionsToClean, + JavaSparkContext jsc) { + int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); + logger.info("Using cleanerParallelism: " + cleanerParallelism); + List> partitionCleanStats = jsc + .parallelize(partitionsToClean, cleanerParallelism) + .flatMapToPair(getFilesToDeleteFunc(this, config)) + .repartition(cleanerParallelism) // repartition to remove skews + .mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey( + // merge partition level clean stats below + (Function2) (e1, e2) -> e1 + .merge(e2)).collect(); + + Map partitionCleanStatsMap = partitionCleanStats.stream() + .collect(Collectors.toMap(e -> e._1(), e -> e._2())); + + HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config); + // Return PartitionCleanStat for each partition passed. + return partitionsToClean.stream().map(partitionPath -> { + PartitionCleanStat partitionCleanStat = + (partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap + .get(partitionPath) : new PartitionCleanStat(partitionPath); + return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()) + .withPartitionPath(partitionPath) + .withEarliestCommitRetained(cleaner.getEarliestCommitToRetain()) + .withDeletePathPattern(partitionCleanStat.deletePathPatterns) + .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles) + .withFailedDeletes(partitionCleanStat.failedDeleteFiles).build(); + }).collect(Collectors.toList()); + } + + enum BucketType { + UPDATE, INSERT + } + + private static class PartitionCleanStat implements Serializable { + + private final String partitionPath; + private final List deletePathPatterns = new ArrayList<>(); + private final List successDeleteFiles = new ArrayList<>(); + private final List failedDeleteFiles = new ArrayList<>(); + + private PartitionCleanStat(String partitionPath) { + this.partitionPath = partitionPath; + } + + private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) { + if (deletedFileResult) { + successDeleteFiles.add(deletePathStr); + } else { + failedDeleteFiles.add(deletePathStr); + } + } + + private void addDeleteFilePatterns(String deletePathStr) { + deletePathPatterns.add(deletePathStr); + } + + private PartitionCleanStat merge(PartitionCleanStat other) { + if (!this.partitionPath.equals(other.partitionPath)) { + throw new RuntimeException(String + .format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath)); + } + successDeleteFiles.addAll(other.successDeleteFiles); + deletePathPatterns.addAll(other.deletePathPatterns); + failedDeleteFiles.addAll(other.failedDeleteFiles); + return this; + } + } + + /** + * Helper class for a small file's location and its actual size on disk + */ + class SmallFile implements Serializable { + + HoodieRecordLocation location; + long sizeBytes; + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("SmallFile {"); + sb.append("location=").append(location).append(", "); + sb.append("sizeBytes=").append(sizeBytes); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Helper class for an insert bucket along with the weight [0.0, 0.1] that defines the amount of + * incoming inserts that should be allocated to the bucket + */ + class InsertBucket implements Serializable { + + int bucketNumber; + // fraction of total inserts, that should go into this bucket + double weight; + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("WorkloadStat {"); + sb.append("bucketNumber=").append(bucketNumber).append(", "); + sb.append("weight=").append(weight); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Helper class for a bucket's type (INSERT and UPDATE) and its file location + */ + class BucketInfo implements Serializable { + + BucketType bucketType; + String fileLoc; + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("BucketInfo {"); + sb.append("bucketType=").append(bucketType).append(", "); + sb.append("fileLoc=").append(fileLoc); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition) + */ + class UpsertPartitioner extends Partitioner { + + /** + * List of all small files to be corrected + */ + List smallFiles = new ArrayList(); + /** + * Total number of RDD partitions, is determined by total buckets we want to pack the incoming + * workload into + */ + private int totalBuckets = 0; + /** + * Stat for the current workload. Helps in determining total inserts, upserts etc. + */ + private WorkloadStat globalStat; + /** + * Helps decide which bucket an incoming update should go to. + */ + private HashMap updateLocationToBucket; + /** + * Helps us pack inserts into 1 or more buckets depending on number of incoming records. + */ + private HashMap> partitionPathToInsertBuckets; + /** + * Remembers what type each bucket is for later. + */ + private HashMap bucketInfoMap; + + UpsertPartitioner(WorkloadProfile profile) { + updateLocationToBucket = new HashMap<>(); + partitionPathToInsertBuckets = new HashMap<>(); + bucketInfoMap = new HashMap<>(); + globalStat = profile.getGlobalStat(); + + assignUpdates(profile); + assignInserts(profile); + + logger.info( + "Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n" + + "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n" + + "UpdateLocations mapped to buckets =>" + updateLocationToBucket); } private void assignUpdates(WorkloadProfile profile) { // each update location gets a partition WorkloadStat gStat = profile.getGlobalStat(); - for (Map.Entry> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) { + for (Map.Entry> updateLocEntry : gStat.getUpdateLocationToCount() + .entrySet()) { addUpdateBucket(updateLocEntry.getKey()); } } @@ -270,10 +665,10 @@ private void assignInserts(WorkloadProfile profile) { } int insertBuckets = (int) Math.max(totalUnassignedInserts / insertRecordsPerBucket, 1L); - logger - .info("After small file assignment: unassignedInserts => " + totalUnassignedInserts - + ", totalInsertBuckets => " + insertBuckets - + ", recordsPerBucket => " + insertRecordsPerBucket); + logger.info( + "After small file assignment: unassignedInserts => " + totalUnassignedInserts + + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + + insertRecordsPerBucket); for (int b = 0; b < insertBuckets; b++) { bucketNumbers.add(totalBuckets); recordsPerBucket.add(totalUnassignedInserts / insertBuckets); @@ -339,8 +734,8 @@ protected List getSmallFiles(String partitionPath) { */ private long averageBytesPerRecord() { long avgSize = 0L; - HoodieTimeline commitTimeline = - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline() + .filterCompletedInstants(); try { if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); @@ -372,7 +767,8 @@ public int numPartitions() { @Override public int getPartition(Object key) { - Tuple2> keyLocation = (Tuple2>) key; + Tuple2> keyLocation = (Tuple2>) key; if (keyLocation._2().isDefined()) { HoodieRecordLocation location = keyLocation._2().get(); return updateLocationToBucket.get(location.getFileId()); @@ -396,420 +792,4 @@ public int getPartition(Object key) { } } } - - - @Override - public Partitioner getUpsertPartitioner(WorkloadProfile profile) { - if (profile == null) { - throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); - } - return new UpsertPartitioner(profile); - } - - @Override - public Partitioner getInsertPartitioner(WorkloadProfile profile) { - return getUpsertPartitioner(profile); - } - - @Override - public boolean isWorkloadProfileNeeded() { - return true; - } - - @Override - public JavaRDD compact(JavaSparkContext jsc, String commitTime) { - throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); - } - - - public Iterator> handleUpdate(String commitTime, String fileLoc, - Iterator> recordItr) - throws IOException { - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr); - return handleUpdateInternal(upsertHandle, commitTime, fileLoc); - } - - public Iterator> handleUpdate(String commitTime, String fileLoc, - Map> keyToNewRecords) - throws IOException { - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords); - return handleUpdateInternal(upsertHandle, commitTime, fileLoc); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc) - throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException("Error in finding the old file path at commit " + - commitTime + " at fileLoc: " + fileLoc); - } else { - AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); - ParquetReader reader = - AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf()) - .build(); - try { - IndexedRecord record; - while ((record = reader.read()) != null) { - // Two types of writes here (new record, and old record). - // We have already catch the exception during writing new records. - // But for old records, we should fail if any exception happens. - upsertHandle.write((GenericRecord) record); - } - } catch (IOException e) { - throw new HoodieUpsertException( - "Failed to read record from " + upsertHandle.getOldFilePath() - + " with new Schema " + upsertHandle.getSchema(), e); - } finally { - reader.close(); - upsertHandle.close(); - } - } - //TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { - logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() - + ", " + upsertHandle.getWriteStatus()); - } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())) - .iterator(); - } - - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, - Iterator> recordItr) { - return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); - } - - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, - Map> keyToNewRecords) { - return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc); - } - - public Iterator> handleInsert(String commitTime, - Iterator> recordItr) throws Exception { - return new LazyInsertIterable<>(recordItr, config, commitTime, this); - } - - - @SuppressWarnings("unchecked") - @Override - public Iterator> handleUpsertPartition(String commitTime, Integer partition, - Iterator recordItr, Partitioner partitioner) { - UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; - BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); - BucketType btype = binfo.bucketType; - try { - if (btype.equals(BucketType.INSERT)) { - return handleInsert(commitTime, recordItr); - } else if (btype.equals(BucketType.UPDATE)) { - return handleUpdate(commitTime, binfo.fileLoc, recordItr); - } else { - throw new HoodieUpsertException( - "Unknown bucketType " + btype + " for partition :" + partition); - } - } catch (Throwable t) { - String msg = "Error upserting bucketType " + btype + " for partition :" + partition; - logger.error(msg, t); - throw new HoodieUpsertException(msg, t); - } - } - - @Override - public Iterator> handleInsertPartition(String commitTime, Integer partition, - Iterator recordItr, - Partitioner partitioner) { - return handleUpsertPartition(commitTime, partition, recordItr, partitioner); - } - - /** - * Performs cleaning of partition paths according to cleaning policy and returns the number of - * files cleaned. Handles skews in partitions to clean by making files to clean as the unit of - * task distribution. - * - * @throws IllegalArgumentException if unknown cleaning policy is provided - */ - @Override - public List clean(JavaSparkContext jsc) { - try { - FileSystem fs = getMetaClient().getFs(); - List partitionsToClean = - FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); - logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config - .getCleanerPolicy()); - if (partitionsToClean.isEmpty()) { - logger.info("Nothing to clean here mom. It is already clean"); - return Collections.emptyList(); - } - return cleanPartitionPaths(partitionsToClean, jsc); - } catch (IOException e) { - throw new HoodieIOException("Failed to clean up after commit", e); - } - } - - /** - * Common method used for cleaning out parquet files under a partition path during rollback of a - * set of commits - */ - protected Map deleteCleanedFiles(String partitionPath, List commits) - throws IOException { - logger.info("Cleaning path " + partitionPath); - FileSystem fs = getMetaClient().getFs(); - FileStatus[] toBeDeleted = - fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> { - if (!path.toString().contains(".parquet")) { - return false; - } - String fileCommitTime = FSUtils.getCommitTime(path.getName()); - return commits.contains(fileCommitTime); - }); - Map results = Maps.newHashMap(); - for (FileStatus file : toBeDeleted) { - boolean success = fs.delete(file.getPath(), false); - results.put(file, success); - logger.info("Delete file " + file.getPath() + "\t" + success); - } - return results; - } - - @Override - public List rollback(JavaSparkContext jsc, List commits) - throws IOException { - String actionType = this.getCommitActionType(); - HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); - List inflights = this.getInflightCommitTimeline().getInstants() - .map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); - - // Atomically unpublish all the commits - commits.stream().filter(s -> !inflights.contains(s)) - .map(s -> new HoodieInstant(false, actionType, s)) - .forEach(activeTimeline::revertToInflight); - logger.info("Unpublished " + commits); - - // delete all the data files for all these commits - logger.info("Clean out all parquet files generated for commits: " + commits); - List stats = jsc.parallelize( - FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning())) - .map((Function) partitionPath -> { - // Scan all partitions files with this commit time - Map results = deleteCleanedFiles(partitionPath, commits); - return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) - .withDeletedFileResults(results).build(); - }).collect(); - - // clean temporary data files - cleanTemporaryDataFiles(jsc); - - // Remove the rolled back inflight commits - commits.stream().map(s -> new HoodieInstant(true, actionType, s)) - .forEach(activeTimeline::deleteInflight); - logger.info("Deleted inflight commits " + commits); - return stats; - } - - /** - * Finalize the written data files - * - * @param writeStatuses List of WriteStatus - * @return number of files finalized - */ - @Override - @SuppressWarnings("unchecked") - public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { - if (!config.shouldUseTempFolderForCopyOnWrite()) { - return Optional.empty(); - } - - // This is to rename each data file from temporary path to its final location - List> results = jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism()) - .map(writeStatus -> { - Tuple2 writeStatTuple2 = (Tuple2) writeStatus; - HoodieWriteStat writeStat = writeStatTuple2._2(); - final FileSystem fs = getMetaClient().getFs(); - final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); - - if (writeStat.getTempPath() != null) { - final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); - boolean success; - try { - logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); - success = fs.rename(tempPath, finalPath); - } catch (IOException e) { - throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath); - } - - if (!success) { - throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath); - } - } - - return new Tuple2<>(writeStat.getPath(), true); - }).collect(); - - // clean temporary data files - cleanTemporaryDataFiles(jsc); - - return Optional.of(results.size()); - } - - /** - * Clean temporary data files that are produced from previous failed commit or retried spark - * stages. - */ - private void cleanTemporaryDataFiles(JavaSparkContext jsc) { - if (!config.shouldUseTempFolderForCopyOnWrite()) { - return; - } - - final FileSystem fs = getMetaClient().getFs(); - final Path temporaryFolder = new Path(config.getBasePath(), - HoodieTableMetaClient.TEMPFOLDER_NAME); - try { - if (!fs.exists(temporaryFolder)) { - logger.info("Temporary folder does not exist: " + temporaryFolder); - return; - } - List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); - List> results = jsc - .parallelize(fileStatusesList, config.getFinalizeWriteParallelism()) - .map(fileStatus -> { - FileSystem fs1 = getMetaClient().getFs(); - boolean success = fs1.delete(fileStatus.getPath(), false); - logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" - + success); - return new Tuple2<>(fileStatus.getPath().toString(), success); - }).collect(); - - for (Tuple2 result : results) { - if (!result._2()) { - logger.info("Failed to delete file: " + result._1()); - throw new HoodieIOException( - "Failed to delete file in temporary folder: " + result._1()); - } - } - } catch (IOException e) { - throw new HoodieIOException( - "Failed to clean data files in temporary folder: " + temporaryFolder); - } - } - - private static class PartitionCleanStat implements Serializable { - - private final String partitionPath; - private final List deletePathPatterns = new ArrayList<>(); - private final List successDeleteFiles = new ArrayList<>(); - private final List failedDeleteFiles = new ArrayList<>(); - - private PartitionCleanStat(String partitionPath) { - this.partitionPath = partitionPath; - } - - private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) { - if (deletedFileResult) { - successDeleteFiles.add(deletePathStr); - } else { - failedDeleteFiles.add(deletePathStr); - } - } - - private void addDeleteFilePatterns(String deletePathStr) { - deletePathPatterns.add(deletePathStr); - } - - private PartitionCleanStat merge(PartitionCleanStat other) { - if (!this.partitionPath.equals(other.partitionPath)) { - throw new RuntimeException(String.format( - "partitionPath is not a match: (%s, %s)", - partitionPath, other.partitionPath)); - } - successDeleteFiles.addAll(other.successDeleteFiles); - deletePathPatterns.addAll(other.deletePathPatterns); - failedDeleteFiles.addAll(other.failedDeleteFiles); - return this; - } - } - - private List cleanPartitionPaths(List partitionsToClean, - JavaSparkContext jsc) { - int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); - logger.info("Using cleanerParallelism: " + cleanerParallelism); - List> partitionCleanStats = jsc - .parallelize(partitionsToClean, cleanerParallelism) - .flatMapToPair(getFilesToDeleteFunc(this, config)) - .repartition(cleanerParallelism) // repartition to remove skews - .mapPartitionsToPair(deleteFilesFunc(this)) - .reduceByKey( - // merge partition level clean stats below - (Function2) (e1, e2) -> e1 - .merge(e2)) - .collect(); - - Map partitionCleanStatsMap = partitionCleanStats - .stream().collect(Collectors.toMap(e -> e._1(), e -> e._2())); - - HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config); - // Return PartitionCleanStat for each partition passed. - return partitionsToClean.stream().map(partitionPath -> { - PartitionCleanStat partitionCleanStat = - (partitionCleanStatsMap.containsKey(partitionPath)) ? - partitionCleanStatsMap.get(partitionPath) - : new PartitionCleanStat(partitionPath); - return HoodieCleanStat.newBuilder() - .withPolicy(config.getCleanerPolicy()) - .withPartitionPath(partitionPath) - .withEarliestCommitRetained(cleaner.getEarliestCommitToRetain()) - .withDeletePathPattern(partitionCleanStat.deletePathPatterns) - .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles) - .withFailedDeletes(partitionCleanStat.failedDeleteFiles) - .build(); - }).collect(Collectors.toList()); - } - - private static PairFlatMapFunction>, String, PartitionCleanStat> deleteFilesFunc( - HoodieTable table) { - return (PairFlatMapFunction>, String, PartitionCleanStat>) iter -> { - Map partitionCleanStatMap = new HashMap<>(); - - FileSystem fs = table.getMetaClient().getFs(); - while (iter.hasNext()) { - Tuple2 partitionDelFileTuple = iter.next(); - String partitionPath = partitionDelFileTuple._1(); - String deletePathStr = partitionDelFileTuple._2(); - Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); - if (!partitionCleanStatMap.containsKey(partitionPath)) { - partitionCleanStatMap.put(partitionPath, - new PartitionCleanStat(partitionPath)); - } - PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); - partitionCleanStat.addDeleteFilePatterns(deletePathStr); - partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult); - } - - return partitionCleanStatMap.entrySet().stream() - .map(e -> new Tuple2<>(e.getKey(), e.getValue())) - .collect(Collectors.toList()).iterator(); - }; - } - - private static PairFlatMapFunction getFilesToDeleteFunc( - HoodieTable table, HoodieWriteConfig config) { - return (PairFlatMapFunction) partitionPathToClean -> { - HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config); - return cleaner.getDeletePaths(partitionPathToClean).stream() - .map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())) - .iterator(); - }; - } - - private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) - throws IOException { - Path deletePath = new Path(deletePathStr); - logger.debug("Working on delete path :" + deletePath); - boolean deleteResult = fs.delete(deletePath, false); - if (deleteResult) { - logger.debug("Cleaned file at path :" + deletePath); - } - return deleteResult; - } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 0787477743b23..79239ada8e8cd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -62,19 +62,12 @@ import org.apache.spark.api.java.function.Function; /** - * Implementation of a more real-time read-optimized Hoodie Table where - *

- * INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) - * Merge with the smallest existing file, to expand it - *

- *

- * UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the - * log file into the base file. - *

- *

- * WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an - * attempted commit action - *

+ * Implementation of a more real-time read-optimized Hoodie Table where

INSERTS - Same as + * HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the + * smallest existing file, to expand it

UPDATES - Appends the changes to a rolling log file + * maintained per file Id. Compaction merges the log file into the base file.

WARNING - MOR + * table type does not support nested rollbacks, every rollback must be followed by an attempted + * commit action

*/ public class HoodieMergeOnReadTable extends HoodieCopyOnWriteTable { @@ -88,57 +81,6 @@ public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient me super(config, metaClient); } - /** - * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet - * files to larger ones without the need for an index in the logFile. - */ - class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner { - - MergeOnReadUpsertPartitioner(WorkloadProfile profile) { - super(profile); - } - - @Override - protected List getSmallFiles(String partitionPath) { - - // smallFiles only for partitionPath - List smallFileLocations = new ArrayList<>(); - - // Init here since this class (and member variables) might not have been initialized - HoodieTimeline commitTimeline = getCompletedCommitTimeline(); - - if (!commitTimeline.empty()) { - HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); - // find smallest file in partition and append to it - Optional smallFileSlice = getRTFileSystemView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()) - .filter(fileSlice -> fileSlice.getLogFiles().count() < 1 && - fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit()) - .sorted((FileSlice left, FileSlice right) -> - left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() ? -1 : 1) - .findFirst(); - - if(smallFileSlice.isPresent()) { - String filename = smallFileSlice.get().getDataFile().get().getFileName(); - SmallFile sf = new SmallFile(); - sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), - FSUtils.getFileId(filename)); - sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize(); - smallFileLocations.add(sf); - // Update the global small files list - smallFiles.add(sf); - } - } - - return smallFileLocations; - } - - public List getSmallFileIds() { - return (List) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId()) - .collect(Collectors.toList()); - } - } - @Override public Partitioner getUpsertPartitioner(WorkloadProfile profile) { if (profile == null) { @@ -150,15 +92,16 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { @Override public Iterator> handleUpdate(String commitTime, String fileId, - Iterator> recordItr) throws IOException { + Iterator> recordItr) throws IOException { logger.info("Merging updates for commit " + commitTime + " for file " + fileId); - if(mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { - logger.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId); + if (mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { + logger.info( + "Small file corrections for updates for commit " + commitTime + " for file " + fileId); return super.handleUpdate(commitTime, fileId, recordItr); } else { - HoodieAppendHandle appendHandle = - new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr); + HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, commitTime, this, + fileId, recordItr); appendHandle.doAppend(); appendHandle.close(); return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())) @@ -202,13 +145,11 @@ public List rollback(JavaSparkContext jsc, List comm if (commits.size() > 1) { throw new UnsupportedOperationException("Nested Rollbacks are not supported"); } - Map commitsAndCompactions = - this.getActiveTimeline() - .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION)) - .getInstants() - .filter(i -> commits.contains(i.getTimestamp())) - .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); + Map commitsAndCompactions = this.getActiveTimeline() + .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, + HoodieActiveTimeline.DELTA_COMMIT_ACTION)).getInstants() + .filter(i -> commits.contains(i.getTimestamp())) + .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); // Atomically un-publish all non-inflight commits commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue()) @@ -218,9 +159,9 @@ public List rollback(JavaSparkContext jsc, List comm Long startTime = System.currentTimeMillis(); - List allRollbackStats = jsc.parallelize - (FSUtils.getAllPartitionPaths(this.metaClient.getFs(), - this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) + List allRollbackStats = jsc.parallelize(FSUtils + .getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning())) .map((Function>) partitionPath -> { return commits.stream().map(commit -> { HoodieInstant instant = commitsAndCompactions.get(commit); @@ -228,56 +169,63 @@ public List rollback(JavaSparkContext jsc, List comm switch (instant.getAction()) { case HoodieTimeline.COMMIT_ACTION: try { - Map results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit)); - hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) - .withDeletedFileResults(results).build(); + Map results = super + .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); + hoodieRollbackStats = HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath).withDeletedFileResults(results).build(); break; } catch (IOException io) { throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } case HoodieTimeline.DELTA_COMMIT_ACTION: try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + this.getCommitTimeline().getInstantDetails( + new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) + .get()); // read commit file and (either append delete blocks or delete file) Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); - // we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW) + // we do not know fileIds for inserts (first inserts are parquet files), delete + // all parquet files for the corresponding failed commit, if present (same as COW) filesToDeletedStatus = super .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); // append rollback blocks for updates - if(commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { + if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { - return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT + return wStat != null + && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null; - }) - .forEach(wStat -> { + }).forEach(wStat -> { HoodieLogFormat.Writer writer = null; try { - writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath)) + writer = HoodieLogFormat.newWriterBuilder().onParentPath( + new Path(this.getMetaClient().getBasePath(), partitionPath)) .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) .withFs(this.metaClient.getFs()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; // generate metadata - Map header = Maps.newHashMap(); + Map header = + Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit); - header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, + commit); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String + .valueOf( + HoodieCommandBlock.HoodieCommandBlockTypeEnum + .ROLLBACK_PREVIOUS_BLOCK + .ordinal())); // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock( - header)); + writer = writer.appendBlock(new HoodieCommandBlock(header)); numRollbackBlocks++; - filesToNumBlocksRollback - .put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()), - numRollbackBlocks); + filesToNumBlocksRollback.put(this.getMetaClient().getFs() + .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); } catch (IOException | InterruptedException io) { throw new HoodieRollbackException( "Failed to rollback for commit " + commit, io); @@ -289,7 +237,8 @@ public List rollback(JavaSparkContext jsc, List comm } } }); - hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) + hoodieRollbackStats = HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath) .withDeletedFileResults(filesToDeletedStatus) .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); } @@ -297,17 +246,19 @@ public List rollback(JavaSparkContext jsc, List comm } catch (IOException io) { throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } + default: + break; } return hoodieRollbackStats; }).collect(Collectors.toList()); }).flatMap(x -> x.iterator()).filter(x -> x != null).collect(); - commitsAndCompactions.entrySet().stream() - .map(entry -> new HoodieInstant(true, entry.getValue().getAction(), - entry.getValue().getTimestamp())) - .forEach(this.getActiveTimeline()::deleteInflight); + commitsAndCompactions.entrySet().stream().map( + entry -> new HoodieInstant(true, entry.getValue().getAction(), + entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight); - logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); + logger + .debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); return allRollbackStats; } @@ -317,4 +268,56 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) // do nothing for MOR tables return Optional.empty(); } + + /** + * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet + * files to larger ones without the need for an index in the logFile. + */ + class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner { + + MergeOnReadUpsertPartitioner(WorkloadProfile profile) { + super(profile); + } + + @Override + protected List getSmallFiles(String partitionPath) { + + // smallFiles only for partitionPath + List smallFileLocations = new ArrayList<>(); + + // Init here since this class (and member variables) might not have been initialized + HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + + if (!commitTimeline.empty()) { + HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); + // find smallest file in partition and append to it + Optional smallFileSlice = getRTFileSystemView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( + fileSlice -> fileSlice.getLogFiles().count() < 1 + && fileSlice.getDataFile().get().getFileSize() < config + .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> + left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() + ? -1 : 1).findFirst(); + + if (smallFileSlice.isPresent()) { + String filename = smallFileSlice.get().getDataFile().get().getFileName(); + SmallFile sf = new SmallFile(); + sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), + FSUtils.getFileId(filename)); + sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize(); + smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); + } + } + + return smallFileLocations; + } + + public List getSmallFileIds() { + return (List) smallFiles.stream() + .map(smallFile -> ((SmallFile) smallFile).location.getFileId()) + .collect(Collectors.toList()); + } + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 728fc560ad031..6f62533c33f13 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -60,18 +60,28 @@ protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient this.metaClient = metaClient; } + public static HoodieTable getHoodieTable( + HoodieTableMetaClient metaClient, HoodieWriteConfig config) { + switch (metaClient.getTableType()) { + case COPY_ON_WRITE: + return new HoodieCopyOnWriteTable<>(config, metaClient); + case MERGE_ON_READ: + return new HoodieMergeOnReadTable<>(config, metaClient); + default: + throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); + } + } + /** * Provides a partitioner to perform the upsert operation, based on the workload profile */ public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile); - /** * Provides a partitioner to perform the insert operation, based on the workload profile */ public abstract Partitioner getInsertPartitioner(WorkloadProfile profile); - /** * Return whether this HoodieTable implementation can benefit from workload profiling */ @@ -131,7 +141,6 @@ public HoodieTimeline getInflightCommitTimeline() { return getCommitsTimeline().filterInflights(); } - /** * Get only the completed (no-inflights) clean timeline */ @@ -162,12 +171,12 @@ public Stream getSavepointedDataFiles(String savepointTime) { throw new HoodieSavepointException( "Could not get data files for savepoint " + savepointTime + ". No such savepoint."); } - HoodieInstant instant = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); + HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime); HoodieSavepointMetadata metadata = null; try { - metadata = AvroUtils.deserializeHoodieSavepointMetadata( - getActiveTimeline().getInstantDetails(instant).get()); + metadata = AvroUtils + .deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get()); } catch (IOException e) { throw new HoodieSavepointException( "Could not get savepointed data files for savepoint " + savepointTime, e); @@ -189,7 +198,8 @@ public HoodieTimeline getCommitsTimeline() { return getActiveTimeline().getCommitTimeline(); case MERGE_ON_READ: // We need to include the parquet files written out in delta commits - // Include commit action to be able to start doing a MOR over a COW dataset - no migration required + // Include commit action to be able to start doing a MOR over a COW dataset - no + // migration required return getActiveTimeline().getCommitsTimeline(); default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); @@ -219,9 +229,10 @@ public String getCommitActionType() { return HoodieActiveTimeline.COMMIT_ACTION; case MERGE_ON_READ: return HoodieActiveTimeline.DELTA_COMMIT_ACTION; + default: + throw new HoodieCommitException( + "Could not commit on unknown storage type " + metaClient.getTableType()); } - throw new HoodieCommitException( - "Could not commit on unknown storage type " + metaClient.getTableType()); } /** @@ -236,21 +247,9 @@ public abstract Iterator> handleUpsertPartition(String commitT public abstract Iterator> handleInsertPartition(String commitTime, Integer partition, Iterator> recordIterator, Partitioner partitioner); - public static HoodieTable getHoodieTable( - HoodieTableMetaClient metaClient, HoodieWriteConfig config) { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - return new HoodieCopyOnWriteTable<>(config, metaClient); - case MERGE_ON_READ: - return new HoodieMergeOnReadTable<>(config, metaClient); - default: - throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); - } - } - /** - * Run Compaction on the table. - * Compaction arranges the data so that it is optimized for data access + * Run Compaction on the table. Compaction arranges the data so that it is optimized for data + * access */ public abstract JavaRDD compact(JavaSparkContext jsc, String commitTime); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java b/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java index 2ca51a31f185b..6adb8a982b01a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecord; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java index 07e86369013f0..ed5431f48c7f7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java @@ -16,7 +16,6 @@ package com.uber.hoodie.table; - import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -31,7 +30,7 @@ /** * Information about incoming records for upsert/insert obtained either via sampling or * introspecting the data fully - * + *

* TODO(vc): Think about obtaining this directly from index.tagLocation */ public class WorkloadProfile implements Serializable { @@ -60,11 +59,9 @@ public WorkloadProfile(JavaRDD> taggedRecords) { private void buildProfile() { Map>, Long> partitionLocationCounts = taggedRecords - .mapToPair(record -> - new Tuple2<>( - new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), - record)) - .countByKey(); + .mapToPair(record -> new Tuple2<>( + new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), + record)).countByKey(); for (Map.Entry>, Long> e : partitionLocationCounts .entrySet()) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java index 33764fa59de02..a93ac55af9414 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java @@ -17,10 +17,9 @@ package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecordLocation; -import org.apache.commons.lang3.tuple.Pair; - import java.io.Serializable; import java.util.HashMap; +import org.apache.commons.lang3.tuple.Pair; /** * Wraps stats about a single partition path. diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index 166bd22de21b9..ab81e2aca1c56 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -14,7 +14,6 @@ * limitations under the License. */ - import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.uber.hoodie.HoodieWriteClient; @@ -38,25 +37,20 @@ import org.apache.spark.api.java.JavaSparkContext; /** - * Driver program that uses the Hoodie client with synthetic workload, and performs basic - * operations.

+ * Driver program that uses the Hoodie client with synthetic workload, and performs basic operations.

*/ public class HoodieClientExample { + private static Logger logger = LogManager.getLogger(HoodieClientExample.class); + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; @Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table") private String tablePath = "file:///tmp/hoodie/sample-table"; - @Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table") private String tableName = "hoodie_rt"; - @Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ") private String tableType = HoodieTableType.COPY_ON_WRITE.name(); - @Parameter(names = {"--help", "-h"}, help = true) - public Boolean help = false; - - private static Logger logger = LogManager.getLogger(HoodieClientExample.class); - public static void main(String[] args) throws Exception { HoodieClientExample cli = new HoodieClientExample(); JCommander cmd = new JCommander(cli, args); @@ -92,10 +86,10 @@ public void run() throws Exception { // Create the write client to write some records in HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()) - .build(); + .forTable(tableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); /** diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index c5544760d8625..881d0b7a8aa30 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -118,13 +118,12 @@ private HoodieWriteConfig getConfig() { } private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) - .forTable("test-trip-table").withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } private void assertNoWriteErrors(List statuses) { @@ -136,10 +135,8 @@ private void assertNoWriteErrors(List statuses) { private void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException { for (String partitionPath : partitionPaths) { - assertTrue( - HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath))); - HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, - new Path(basePath, partitionPath)); + assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath))); + HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath)); pmeta.readFromFS(); assertEquals(3, pmeta.getPartitionDepth()); } @@ -148,8 +145,7 @@ private void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) thr private void checkTaggedRecords(List taggedRecords, String commitTime) { for (HoodieRecord rec : taggedRecords) { assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown()); - assertEquals( - "All records should have commit time " + commitTime + ", since updates were made", + assertEquals("All records should have commit time " + commitTime + ", since updates were made", rec.getCurrentLocation().getCommitTime(), commitTime); } } @@ -194,8 +190,7 @@ public void testAutoCommit() throws Exception { List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - JavaRDD result = client.bulkInsertPreppedRecords(writeRecords, newCommitTime, - Option.empty()); + JavaRDD result = client.bulkInsertPreppedRecords(writeRecords, newCommitTime, Option.empty()); assertFalse("If Autocommit is false, then commit should not be made automatically", HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); @@ -270,22 +265,16 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, - timeline.lastInstant().get().getTimestamp()); - assertEquals("Must contain 200 records", - records.size(), + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals("Must contain 200 records", records.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // Should have 100 records in table (check using Index), all in locations marked at commit HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); - List taggedRecords = readClient.tagLocation(jsc.parallelize(records, 1)) - .collect(); + List taggedRecords = readClient.tagLocation(jsc.parallelize(records, 1)).collect(); checkTaggedRecords(taggedRecords, "001"); /** @@ -309,10 +298,8 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc // verify there are now 2 commits timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), - newCommitTime); + assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); + assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); // Index should be able to locate all updates in correct locations. readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); @@ -324,8 +311,7 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc for (int i = 0; i < fullPartitionPaths.length; i++) { fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } - assertEquals("Must contain 200 records", - 200, + assertEquals("Must contain 200 records", 200, HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); // Check that the incremental consumption from time 000 @@ -339,10 +325,8 @@ private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exc @Test public void testUpsertsWithFinalizeWrite() throws Exception { - HoodieWriteConfig hoodieWriteConfig = getConfigBuilder() - .withUseTempFolderCopyOnWriteForCreate(true) - .withUseTempFolderCopyOnWriteForMerge(true) - .build(); + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(true).build(); testUpsertsInternal(hoodieWriteConfig); } @@ -372,22 +356,16 @@ public void testDeletes() throws Exception { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, - timeline.lastInstant().get().getTimestamp()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); assertEquals("Must contain 200 records", fewRecordsForInsert.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - List taggedRecords = index - .tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); + List taggedRecords = index.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); checkTaggedRecords(taggedRecords, "001"); /** @@ -408,10 +386,8 @@ public void testDeletes() throws Exception { // verify there are now 2 commits timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), - newCommitTime); + assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); + assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); // Check the entire dataset has 150 records(200-50) still String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; @@ -422,14 +398,11 @@ public void testDeletes() throws Exception { HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); // Check that the incremental consumption from time 000 - assertEquals("Incremental consumption from latest commit, should give 50 updated records", - 50, + assertEquals("Incremental consumption from latest commit, should give 50 updated records", 50, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); - assertEquals("Incremental consumption from time 001, should give 50 updated records", - 50, + assertEquals("Incremental consumption from time 001, should give 50 updated records", 50, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); - assertEquals("Incremental consumption from time 000, should give 150", - 150, + assertEquals("Incremental consumption from time 000, should give 150", 150, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count()); } @@ -437,12 +410,10 @@ public void testDeletes() throws Exception { @Test public void testCreateSavepoint() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTestDataGenerator - .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); /** * Write 1 (only inserts) @@ -451,8 +422,7 @@ public void testCreateSavepoint() throws Exception { client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); - List statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime) - .collect(); + List statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(statuses); /** @@ -496,12 +466,10 @@ public void testCreateSavepoint() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - List partitionPaths = FSUtils - .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, getConfig()); + List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), + getConfig().shouldAssumeDatePartitioning()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); List dataFiles = partitionPaths.stream().flatMap(s -> { return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); @@ -511,8 +479,7 @@ public void testCreateSavepoint() throws Exception { // Delete savepoint assertFalse(table.getCompletedSavepointTimeline().empty()); - client.deleteSavepoint( - table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp()); + client.deleteSavepoint(table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp()); // rollback and reupsert 004 client.rollback(newCommitTime); @@ -535,12 +502,10 @@ public void testCreateSavepoint() throws Exception { @Test public void testRollbackToSavepoint() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTestDataGenerator - .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); /** * Write 1 (only inserts) @@ -577,10 +542,9 @@ public void testRollbackToSavepoint() throws Exception { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); - List partitionPaths = FSUtils - .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), + getConfig().shouldAssumeDatePartitioning()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); @@ -618,8 +582,7 @@ public void testRollbackToSavepoint() throws Exception { } // rollback to savepoint 002 - HoodieInstant savepoint = - table.getCompletedSavepointTimeline().getInstants().findFirst().get(); + HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get(); client.rollbackToSavepoint(savepoint.getTimestamp()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -646,8 +609,7 @@ public void testRollbackToSavepoint() throws Exception { public void testInsertAndCleanByVersions() throws Exception { int maxVersions = 2; // keep upto 2 versions for each file HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) .retainFileVersions(maxVersions).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); @@ -666,26 +628,22 @@ public void testInsertAndCleanByVersions() throws Exception { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); // Should have 100 records in table (check using Index), all in locations marked at commit HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = - table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); + String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); assertFalse(table.getCompletedCleanTimeline().empty()); assertEquals("The clean instant should be the same as the commit instant", commitTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) - .collect(); + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); - // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. + // Keep doing some writes and clean inline. Make sure we have expected number of files + // remaining. for (int writeCnt = 2; writeCnt < 10; writeCnt++) { Thread.sleep(1100); // make sure commits are unique @@ -696,8 +654,7 @@ public void testInsertAndCleanByVersions() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metadata, getConfig()); timeline = table.getCommitsTimeline(); @@ -707,20 +664,17 @@ public void testInsertAndCleanByVersions() throws Exception { // compute all the versions of all files, from time 0 HashMap> fileIdToVersions = new HashMap<>(); for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(entry).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(entry).get()); for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) { if (!fileIdToVersions.containsKey(wstat.getFileId())) { fileIdToVersions.put(wstat.getFileId(), new TreeSet<>()); } - fileIdToVersions.get(wstat.getFileId()) - .add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName())); + fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName())); } } - List fileGroups = fsView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { // No file has no more than max versions @@ -733,10 +687,8 @@ public void testInsertAndCleanByVersions() throws Exception { // Each file, has the latest N versions (i.e cleaning gets rid of older versions) List commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId)); for (int i = 0; i < dataFiles.size(); i++) { - assertEquals( - "File " + fileId + " does not have latest versions on commits" + commitedVersions, - Iterables.get(dataFiles, i).getCommitTime(), - commitedVersions.get(commitedVersions.size() - 1 - i)); + assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions, + Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i)); } } } @@ -747,8 +699,7 @@ public void testInsertAndCleanByVersions() throws Exception { public void testInsertAndCleanByCommits() throws Exception { int maxCommits = 3; // keep upto 3 commits from the past HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) .retainCommits(maxCommits).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); @@ -767,28 +718,23 @@ public void testInsertAndCleanByCommits() throws Exception { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = - table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); + String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); assertFalse(table.getCompletedCleanTimeline().empty()); assertEquals("The clean instant should be the same as the commit instant", commitTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) - .collect(); + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); - // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. + // Keep doing some writes and clean inline. Make sure we have expected number of files + // remaining. for (int writeCnt = 2; writeCnt < 10; writeCnt++) { Thread.sleep(1100); // make sure commits are unique newCommitTime = client.startCommit(); @@ -798,18 +744,14 @@ public void testInsertAndCleanByCommits() throws Exception { // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); - Optional - earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); - Set acceptableCommits = - activeTimeline.getInstants().collect(Collectors.toSet()); + Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); + Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); if (earliestRetainedCommit.isPresent()) { acceptableCommits.removeAll( - activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()) - .getInstants() + activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()).getInstants() .collect(Collectors.toSet())); acceptableCommits.add(earliestRetainedCommit.get()); } @@ -817,8 +759,7 @@ public void testInsertAndCleanByCommits() throws Exception { TableFileSystemView fsView = table1.getFileSystemView(); // Need to ensure the following for (String partitionPath : dataGen.getPartitionPaths()) { - List fileGroups = fsView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { Set commitTimes = new HashSet<>(); fileGroup.getAllDataFiles().forEach(value -> { @@ -826,8 +767,7 @@ public void testInsertAndCleanByCommits() throws Exception { commitTimes.add(value.getCommitTime()); }); assertEquals("Only contain acceptable versions of file should be present", - acceptableCommits.stream().map(HoodieInstant::getTimestamp) - .collect(Collectors.toSet()), commitTimes); + acceptableCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), commitTimes); } } } @@ -841,8 +781,7 @@ public void testRollbackCommit() throws Exception { String commitTime3 = "20160506030611"; new File(basePath + "/.hoodie").mkdirs(); HoodieTestDataGenerator - .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // Only first two have commit files HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2); @@ -864,10 +803,8 @@ public void testRollbackCommit() throws Exception { String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32"); String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY) - .build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, config, false); @@ -882,9 +819,9 @@ public void testRollbackCommit() throws Exception { // Rollback commit3 client.rollback(commitTime3); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); // simulate partial failure, where .inflight was not deleted, but data files were. HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3); @@ -895,9 +832,9 @@ public void testRollbackCommit() throws Exception { client.rollback(commitTime2); assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); // simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a // .inflight commit and a bunch of data files around. @@ -909,20 +846,19 @@ public void testRollbackCommit() throws Exception { client.rollback(commitTime2); assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); // Let's rollback commit1, Check results client.rollback(commitTime1); assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); } - @Test public void testAutoRollbackCommit() throws Exception { // Let's create some commit files and parquet files @@ -931,8 +867,7 @@ public void testAutoRollbackCommit() throws Exception { String commitTime3 = "20160506030611"; new File(basePath + "/.hoodie").mkdirs(); HoodieTestDataGenerator - .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // One good commit HoodieTestUtils.createCommitFiles(basePath, commitTime1); @@ -955,10 +890,8 @@ public void testAutoRollbackCommit() throws Exception { String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); // Turn auto rollback off - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY) - .build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); new HoodieWriteClient(jsc, config, false); @@ -966,62 +899,58 @@ public void testAutoRollbackCommit() throws Exception { assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); // Turn auto rollback on new HoodieWriteClient(jsc, config, true); assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); } private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { HoodieWriteConfig.Builder builder = getConfigBuilder(); return builder.withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15) + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15) .insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records - .withStorageConfig(HoodieStorageConfig.newBuilder() - .limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20) - .build()) + .withStorageConfig( + HoodieStorageConfig.newBuilder().limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20).build()) .build(); } @Test public void testSmallInsertHandlingForUpserts() throws Exception { - final String TEST_PARTITION_PATH = "2016/09/26"; - final int INSERT_SPLIT_LIMIT = 100; + final String testPartitionPath = "2016/09/26"; + final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig( - INSERT_SPLIT_LIMIT); // hold upto 200 records max - dataGen = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH}); + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); HoodieWriteClient client = new HoodieWriteClient(jsc, config); // Inserts => will write file1 String commitTime1 = "001"; client.startCommitWithTime(commitTime1); - List inserts1 = dataGen - .generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb + List inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb Set keys1 = HoodieClientTestUtils.getRecordKeys(inserts1); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); @@ -1031,10 +960,8 @@ public void testSmallInsertHandlingForUpserts() throws Exception { assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); - assertEquals("file should contain 100 records", - ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), - 100); + assertEquals("file should contain 100 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), + new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100); // Update + Inserts such that they just expand file1 String commitTime2 = "002"; @@ -1051,20 +978,16 @@ public void testSmallInsertHandlingForUpserts() throws Exception { assertEquals("Just 1 file needs to be updated.", 1, statuses.size()); assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId()); - assertEquals("Existing file should be expanded", commitTime1, - statuses.get(0).getStat().getPrevCommit()); - Path newFile = new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); + assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit()); + Path newFile = new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); assertEquals("file should contain 140 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140); List records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile); for (GenericRecord record : records) { String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - assertEquals("only expect commit2", commitTime2, - record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); - assertTrue("key expected to be part of commit2", - keys2.contains(recordKey) || keys1.contains(recordKey)); + assertEquals("only expect commit2", commitTime2, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + assertTrue("key expected to be part of commit2", keys2.contains(recordKey) || keys1.contains(recordKey)); } // update + inserts such that file1 is updated and expanded, a new file2 is created. @@ -1083,9 +1006,8 @@ public void testSmallInsertHandlingForUpserts() throws Exception { HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView(); - List files = fileSystemView - .getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3).collect( - Collectors.toList()); + List files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) + .collect(Collectors.toList()); int numTotalInsertsInCommit3 = 0; for (HoodieDataFile file : files) { if (file.getFileName().contains(file1)) { @@ -1103,8 +1025,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { } } } - assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, - keys2.size()); + assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, keys2.size()); } else { assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime()); records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); @@ -1123,32 +1044,28 @@ public void testSmallInsertHandlingForUpserts() throws Exception { @Test public void testSmallInsertHandlingForInserts() throws Exception { - final String TEST_PARTITION_PATH = "2016/09/26"; - final int INSERT_SPLIT_LIMIT = 100; + final String testPartitionPath = "2016/09/26"; + final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig( - INSERT_SPLIT_LIMIT); // hold upto 200 records max - dataGen = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH}); + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); HoodieWriteClient client = new HoodieWriteClient(jsc, config); // Inserts => will write file1 String commitTime1 = "001"; client.startCommitWithTime(commitTime1); - List inserts1 = dataGen - .generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb + List inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb Set keys1 = HoodieClientTestUtils.getRecordKeys(inserts1); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); List statuses = client.insert(insertRecordsRDD1, commitTime1).collect(); assertNoWriteErrors(statuses); - assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, fs); + assertPartitionMetadata(new String[] {testPartitionPath}, fs); assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); - assertEquals("file should contain 100 records", - ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), - 100); + assertEquals("file should contain 100 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), + new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100); // Second, set of Inserts should just expand file1 String commitTime2 = "002"; @@ -1161,10 +1078,8 @@ public void testSmallInsertHandlingForInserts() throws Exception { assertEquals("Just 1 file needs to be updated.", 1, statuses.size()); assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId()); - assertEquals("Existing file should be expanded", commitTime1, - statuses.get(0).getStat().getPrevCommit()); - Path newFile = new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); + assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit()); + Path newFile = new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); assertEquals("file should contain 140 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140); @@ -1187,12 +1102,11 @@ public void testSmallInsertHandlingForInserts() throws Exception { assertNoWriteErrors(statuses); assertEquals("2 files needs to be committed.", 2, statuses.size()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); - List files = - table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3) - .collect(Collectors.toList()); + List files = table.getROFileSystemView() + .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) + .collect(Collectors.toList()); assertEquals("Total of 2 valid data files", 2, files.size()); int totalInserts = 0; @@ -1207,21 +1121,18 @@ public void testSmallInsertHandlingForInserts() throws Exception { @Test public void testKeepLatestFileVersions() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .retainFileVersions(1).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); // make 1 commit, with 1 file per partition HoodieTestUtils.createCommitFiles(basePath, "000"); String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); - HoodieTable table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -1233,15 +1144,10 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); - table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), - config); - - String file2P0C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[0], "001"); // insert - String file2P1C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[1], "001"); // insert + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config); + + String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert + String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update @@ -1257,10 +1163,8 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); - table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update @@ -1284,14 +1188,13 @@ public void testKeepLatestFileVersions() throws IOException { @Test public void testKeepLatestFileVersionsMOR() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .retainFileVersions(1).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); - HoodieTableMetaClient metaClient = HoodieTestUtils - .initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); + HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, + HoodieTableType.MERGE_ON_READ); // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); @@ -1304,12 +1207,9 @@ public void testKeepLatestFileVersionsMOR() throws IOException { // Make 4 files, one base file and 3 log files associated with base file HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0); - file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); - file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); - file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); + file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); + file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); + file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); @@ -1318,19 +1218,15 @@ public void testKeepLatestFileVersionsMOR() throws IOException { assertEquals("Must clean three files, one parquet and 2 log files", 3, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0)); - assertFalse(HoodieTestUtils - .doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.empty())); - assertFalse(HoodieTestUtils - .doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.of(2))); + assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.empty())); + assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.of(2))); } @Test public void testKeepLatestCommits() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(2).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); // make 1 commit, with 1 file per partition HoodieTestUtils.createCommitFiles(basePath, "000"); @@ -1351,13 +1247,11 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); - table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); - String file2P0C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[0], "001"); // insert - String file2P1C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[1], "001"); // insert + String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert + String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update @@ -1373,36 +1267,30 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); - table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002"); List hoodieCleanStatsThree = table.clean(jsc); - assertEquals( - "Must not clean any file. We have to keep 1 version before the latest commit time to keep", - 0, getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size()); + assertEquals("Must not clean any file. We have to keep 1 version before the latest commit time to keep", 0, + getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size()); assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "003"); - table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003"); List hoodieCleanStatsFour = table.clean(jsc); - assertEquals( - "Must not clean one old file", 1, + assertEquals("Must not clean one old file", 1, getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size()); assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); @@ -1424,34 +1312,27 @@ public void testKeepLatestCommits() throws IOException { @Test public void testCleaningWithZeroPartitonPaths() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(2).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); // Make a commit, although there are no partitionPaths. // Example use-case of this is when a client wants to create a table // with just some commit metadata, but no data/partitionPaths. HoodieTestUtils.createCommitFiles(basePath, "000"); - HoodieTable table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); - assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", - hoodieCleanStatsOne.isEmpty()); + assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); } @Test public void testCleaningSkewedPartitons() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(2).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); Map stageOneShuffleReadTaskRecordsCountMap = new HashMap<>(); // Since clean involves repartition in order to uniformly distribute data, @@ -1465,16 +1346,12 @@ public void testCleaningSkewedPartitons() throws IOException { @Override public void onTaskEnd(SparkListenerTaskEnd taskEnd) { - Iterator> iterator = taskEnd.taskMetrics().accumulators() - .iterator(); + Iterator> iterator = taskEnd.taskMetrics().accumulators().iterator(); while (iterator.hasNext()) { AccumulatorV2 accumulator = iterator.next(); - if (taskEnd.stageId() == 1 && - accumulator.isRegistered() && - accumulator.name().isDefined() && - accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) { - stageOneShuffleReadTaskRecordsCountMap - .put(taskEnd.taskInfo().taskId(), (Long) accumulator.value()); + if (taskEnd.stageId() == 1 && accumulator.isRegistered() && accumulator.name().isDefined() + && accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) { + stageOneShuffleReadTaskRecordsCountMap.put(taskEnd.taskInfo().taskId(), (Long) accumulator.value()); } } } @@ -1501,63 +1378,53 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { updateAllFilesInPartition(filesP1C0, partitionPaths[1], "003"); updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003"); - HoodieTable table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStats = table.clean(jsc); - assertEquals(100, - getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals(10, - getCleanStat(hoodieCleanStats, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertEquals(10, - getCleanStat(hoodieCleanStats, partitionPaths[2]).getSuccessDeleteFiles().size()); + assertEquals(100, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); + assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[1]).getSuccessDeleteFiles().size()); + assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[2]).getSuccessDeleteFiles().size()); // 3 tasks are expected since the number of partitions is 3 assertEquals(3, stageOneShuffleReadTaskRecordsCountMap.keySet().size()); // Sum of all records processed = total number of files to clean - assertEquals(120, stageOneShuffleReadTaskRecordsCountMap - .values().stream().reduce((a, b) -> a + b).get().intValue()); + assertEquals(120, + stageOneShuffleReadTaskRecordsCountMap.values().stream().reduce((a, b) -> a + b).get().intValue()); assertTrue("The skew in handling files to clean is not removed. " + "Each task should handle more records than the partitionPath with least files " + "and less records than the partitionPath with most files.", - stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100) - .count() == 3); + stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3); } @Test public void testCleanTemporaryDataFiles() throws IOException { HoodieTestUtils.createCommitFiles(basePath, "000"); List tempFiles = createTempFiles("000", 10); - assertEquals("Some temp files are created.",10, tempFiles.size()); - assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles()); + assertEquals("Some temp files are created.", 10, tempFiles.size()); + assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withUseTempFolderCopyOnWriteForCreate(false) .withUseTempFolderCopyOnWriteForMerge(false).build(); - HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), - config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); table.rollback(jsc, Collections.emptyList()); - assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles()); + assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); - config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withUseTempFolderCopyOnWriteForCreate(true) + config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) .withUseTempFolderCopyOnWriteForMerge(false).build(); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), - config.getBasePath(), true), + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); table.rollback(jsc, Collections.emptyList()); - assertEquals("All temp files are deleted.",0, getTotalTempFiles()); + assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); } public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); String commitTime = "000"; @@ -1574,11 +1441,10 @@ public void testCommitWritesRelativePaths() throws Exception { // Get parquet file paths from commit metadata String actionType = table.getCommitActionType(); - HoodieInstant commitInstant = - new HoodieInstant(false, actionType, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants(); - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); String basePath = table.getMetaClient().getBasePath(); Collection commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values(); @@ -1596,15 +1462,12 @@ public void testCommitWritesRelativePaths() throws Exception { } } - private HoodieCleanStat getCleanStat(List hoodieCleanStatsTwo, - String partitionPath) { - return hoodieCleanStatsTwo.stream() - .filter(e -> e.getPartitionPath().equals(partitionPath)) - .findFirst().get(); + private HoodieCleanStat getCleanStat(List hoodieCleanStatsTwo, String partitionPath) { + return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().get(); } - private void updateAllFilesInPartition(List files, String partitionPath, - String commitTime) throws IOException { + private void updateAllFilesInPartition(List files, String partitionPath, String commitTime) + throws IOException { for (String fileId : files) { HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java index 0dc7a816f9921..c9969f2a663ed 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java @@ -16,7 +16,6 @@ package com.uber.hoodie; - import static org.junit.Assert.assertEquals; import com.uber.hoodie.common.HoodieClientTestUtils; @@ -58,11 +57,11 @@ public class TestMultiFS implements Serializable { private static MiniDFSCluster dfsCluster; private static DistributedFileSystem dfs; private static Logger logger = LogManager.getLogger(TestMultiFS.class); + private static JavaSparkContext jsc; + private static SQLContext sqlContext; private String tablePath = "file:///tmp/hoodie/sample-table"; private String tableName = "hoodie_rt"; private String tableType = HoodieTableType.COPY_ON_WRITE.name(); - private static JavaSparkContext jsc; - private static SQLContext sqlContext; @BeforeClass public static void initClass() throws Exception { @@ -92,7 +91,8 @@ public static void cleanupClass() throws Exception { hdfsTestService.stop(); dfsCluster.shutdown(); } - // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM FileSystem.closeAll(); } @@ -111,8 +111,7 @@ public void readLocalWriteHDFS() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg); // Write generated data to hdfs (only inserts) @@ -125,10 +124,8 @@ public void readLocalWriteHDFS() throws Exception { // Read from hdfs FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - Dataset readRecords = HoodieClientTestUtils - .readCommit(dfsBasePath, sqlContext, timeline, readCommitTime); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + Dataset readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime); assertEquals("Should contain 100 records", readRecords.count(), records.size()); // Write to local @@ -138,8 +135,7 @@ public void readLocalWriteHDFS() throws Exception { HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig); String writeCommitTime = localWriteClient.startCommit(); @@ -153,8 +149,7 @@ public void readLocalWriteHDFS() throws Exception { fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf()); metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath); timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - Dataset localReadRecords = HoodieClientTestUtils - .readCommit(tablePath, sqlContext, timeline, writeCommitTime); + Dataset localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime); assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size()); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index 295d415259ed4..b87e1768c95c7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -69,8 +69,7 @@ public static Set getRecordKeys(List hoodieRecords) { return keys; } - private static void fakeMetaFile(String basePath, String commitTime, String suffix) - throws IOException { + private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException { String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME; new File(parentPath).mkdirs(); new File(parentPath + "/" + commitTime + suffix).createNewFile(); @@ -85,55 +84,48 @@ public static void fakeInFlightFile(String basePath, String commitTime) throws I fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION); } - public static void fakeDataFile(String basePath, String partitionPath, String commitTime, - String fileId) throws Exception { + public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) + throws Exception { fakeDataFile(basePath, partitionPath, commitTime, fileId, 0); } - public static void fakeDataFile(String basePath, String partitionPath, String commitTime, - String fileId, long length) throws Exception { + public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length) + throws Exception { String parentPath = String.format("%s/%s", basePath, partitionPath); new File(parentPath).mkdirs(); - String path = String - .format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId)); + String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId)); new File(path).createNewFile(); new RandomAccessFile(path, "rw").setLength(length); } public static SparkConf getSparkConfForTest(String appName) { - SparkConf sparkConf = new SparkConf() - .setAppName(appName) + System.out.println("HIII" + "HII2"); + SparkConf sparkConf = new SparkConf().setAppName(appName) .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .setMaster("local[1]"); return HoodieReadClient.addHoodieSupport(sparkConf); } - public static HashMap getLatestFileIDsToFullPath(String basePath, - HoodieTimeline commitTimeline, + public static HashMap getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline, List commitsToReturn) throws IOException { HashMap fileIdToFullPath = new HashMap<>(); for (HoodieInstant commit : commitsToReturn) { - HoodieCommitMetadata metadata = - HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get()); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get()); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath)); } return fileIdToFullPath; } - public static Dataset readCommit(String basePath, - SQLContext sqlContext, - HoodieTimeline commitTimeline, + public static Dataset readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, String commitTime) { - HoodieInstant commitInstant = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!commitTimeline.containsInstant(commitInstant)) { new HoodieException("No commit exists at " + commitTime); } try { HashMap paths = getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant)); - return sqlContext.read() - .parquet(paths.values().toArray(new String[paths.size()])) + return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()])) .filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime)); } catch (Exception e) { throw new HoodieException("Error reading commit " + commitTime, e); @@ -143,50 +135,37 @@ public static Dataset readCommit(String basePath, /** * Obtain all new data written into the Hoodie dataset since the given timestamp. */ - public static Dataset readSince(String basePath, - SQLContext sqlContext, - HoodieTimeline commitTimeline, + public static Dataset readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, String lastCommitTime) { - List commitsToReturn = - commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE) - .getInstants().collect(Collectors.toList()); + List commitsToReturn = commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE) + .getInstants().collect(Collectors.toList()); try { // Go over the commit metadata, and obtain the new files that need to be read. - HashMap fileIdToFullPath = getLatestFileIDsToFullPath(basePath, - commitTimeline, commitsToReturn); - return sqlContext.read() - .parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()])) - .filter( - String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime)); + HashMap fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn); + return sqlContext.read().parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()])) + .filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime)); } catch (IOException e) { - throw new HoodieException( - "Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e); + throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e); } } /** * Reads the paths under the a hoodie dataset out as a DataFrame */ - public static Dataset read(String basePath, - SQLContext sqlContext, - FileSystem fs, - String... paths) { + public static Dataset read(String basePath, SQLContext sqlContext, FileSystem fs, String... paths) { List filteredPaths = new ArrayList<>(); try { HoodieTable hoodieTable = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); for (String path : paths) { TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( - hoodieTable.getMetaClient(), - hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); - List latestFiles = fileSystemView.getLatestDataFiles().collect( - Collectors.toList()); + hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); + List latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList()); for (HoodieDataFile file : latestFiles) { filteredPaths.add(file.getPath()); } } - return sqlContext.read() - .parquet(filteredPaths.toArray(new String[filteredPaths.size()])); + return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()])); } catch (Exception e) { throw new HoodieException("Error reading hoodie dataset as a dataframe", e); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java index c7e3e915618aa..d925c5669b679 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java @@ -42,8 +42,7 @@ */ public class HoodieMergeOnReadTestUtils { - public static List getRecordsUsingInputFormat(List inputPaths, - String basePath) + public static List getRecordsUsingInputFormat(List inputPaths, String basePath) throws IOException { JobConf jobConf = new JobConf(); Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA)); @@ -59,7 +58,8 @@ public static List getRecordsUsingInputFormat(List inputP ArrayWritable writable = (ArrayWritable) recordReader.createValue(); while (recordReader.next(key, writable)) { GenericRecordBuilder newRecord = new GenericRecordBuilder(schema); - // writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno] + // writable returns an array with [field1, field2, _hoodie_commit_time, + // _hoodie_commit_seqno] Writable[] values = writable.get(); schema.getFields().forEach(field -> { newRecord.set(field, values[2]); @@ -76,12 +76,11 @@ public static List getRecordsUsingInputFormat(List inputP }).get(); } - private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, - Schema schema, String basePath) { + private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema, + String basePath) { List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); - String postions = fields.stream().map(f -> String.valueOf(f.pos())) - .collect(Collectors.joining(",")); + String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(",")); Configuration conf = HoodieTestUtils.getDefaultHadoopConf(); jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index beb76df651ab7..f623fc7cbe235 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -41,20 +41,15 @@ /** * Class to be used in tests to keep generating test inserts and updates against a corpus. - * + *

* Test data uses a toy Uber trips, data model. */ public class HoodieTestDataGenerator { - static class KeyPartition { - - HoodieKey key; - String partitionPath; - } - - public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," - + "\"name\": \"triprec\"," - + "\"fields\": [ " + // based on examination of sample file, the schema produces the following per record size + public static final int SIZE_PER_RECORD = 50 * 1024; + public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; + public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"double\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"}," + "{\"name\": \"rider\", \"type\": \"string\"}," @@ -64,39 +59,79 @@ static class KeyPartition { + "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"}," + "{\"name\":\"fare\",\"type\": \"double\"}]}"; + public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); + private static Random rand = new Random(46474747); + private List existingKeysList = new ArrayList<>(); + private String[] partitionPaths = DEFAULT_PARTITION_PATHS; - // based on examination of sample file, the schema produces the following per record size - public static final int SIZE_PER_RECORD = 50 * 1024; - - public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; + public HoodieTestDataGenerator(String[] partitionPaths) { + this.partitionPaths = partitionPaths; + } + public HoodieTestDataGenerator() { + this(new String[] {"2016/03/15", "2015/03/16", "2015/03/17"}); + } - public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, - String basePath) { + public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) { for (String partitionPath : partitionPaths) { - new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)) - .trySave(0); + new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0); } } - private List existingKeysList = new ArrayList<>(); - public static Schema avroSchema = HoodieAvroUtils - .addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); - private static Random rand = new Random(46474747); - private String[] partitionPaths = DEFAULT_PARTITION_PATHS; + /** + * Generates a new avro record of the above schema format, retaining the key if optionally provided. + */ + public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException { + 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); + } - public HoodieTestDataGenerator(String[] partitionPaths) { - this.partitionPaths = partitionPaths; + public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName, + double timestamp) { + GenericRecord rec = new GenericData.Record(avroSchema); + rec.put("_row_key", rowKey); + rec.put("timestamp", timestamp); + rec.put("rider", riderName); + rec.put("driver", driverName); + rec.put("begin_lat", rand.nextDouble()); + rec.put("begin_lon", rand.nextDouble()); + rec.put("end_lat", rand.nextDouble()); + rec.put("end_lon", rand.nextDouble()); + rec.put("fare", rand.nextDouble() * 100); + return rec; } - public HoodieTestDataGenerator() { - this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"}); + public static void createCommitFile(String basePath, String commitTime) throws IOException { + Path commitFile = new Path( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime)); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FSDataOutputStream os = fs.create(commitFile, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + try { + // Write empty commit metadata + os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } finally { + os.close(); + } } + public static void createSavepointFile(String basePath, String commitTime) throws IOException { + Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + + "/" + HoodieTimeline.makeSavePointFileName(commitTime)); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FSDataOutputStream os = fs.create(commitFile, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + try { + // Write empty commit metadata + os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } finally { + os.close(); + } + } /** - * Generates new inserts, uniformly across the partition paths above. It also updates the list of - * existing keys. + * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ public List generateInserts(String commitTime, int n) throws IOException { List inserts = new ArrayList<>(); @@ -119,8 +154,7 @@ public List generateDeletes(String commitTime, int n) throws IOExc return generateDeletesFromExistingRecords(inserts); } - public List generateDeletesFromExistingRecords(List existingRecords) - throws IOException { + public List generateDeletesFromExistingRecords(List existingRecords) throws IOException { List deletes = new ArrayList<>(); for (HoodieRecord existingRecord : existingRecords) { HoodieRecord record = generateDeleteRecord(existingRecord); @@ -132,17 +166,15 @@ public List generateDeletesFromExistingRecords(List public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException { HoodieKey key = existingRecord.getKey(); - TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), - key.getPartitionPath(), null, true); + TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), + null, true); return new HoodieRecord(key, payload); } - public List generateUpdates(String commitTime, List baseRecords) - throws IOException { + public List generateUpdates(String commitTime, List baseRecords) throws IOException { List updates = new ArrayList<>(); for (HoodieRecord baseRecord : baseRecords) { - HoodieRecord record = new HoodieRecord(baseRecord.getKey(), - generateRandomValue(baseRecord.getKey(), commitTime)); + HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime)); updates.add(record); } return updates; @@ -161,68 +193,13 @@ public List generateUpdates(String commitTime, int n) throws IOExc return updates; } - - /** - * Generates a new avro record of the above schema format, retaining the key if optionally - * provided. - */ - public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) - throws IOException { - 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); - } - - public static GenericRecord generateGenericRecord(String rowKey, String riderName, - String driverName, double timestamp) { - GenericRecord rec = new GenericData.Record(avroSchema); - rec.put("_row_key", rowKey); - rec.put("timestamp", timestamp); - rec.put("rider", riderName); - rec.put("driver", driverName); - rec.put("begin_lat", rand.nextDouble()); - rec.put("begin_lon", rand.nextDouble()); - rec.put("end_lat", rand.nextDouble()); - rec.put("end_lon", rand.nextDouble()); - rec.put("fare", rand.nextDouble() * 100); - return rec; - } - - public static void createCommitFile(String basePath, String commitTime) throws IOException { - Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - FSDataOutputStream os = fs.create(commitFile, true); - HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - try { - // Write empty commit metadata - os.writeBytes(new String(commitMetadata.toJsonString().getBytes( - StandardCharsets.UTF_8))); - } finally { - os.close(); - } + public String[] getPartitionPaths() { + return partitionPaths; } - public static void createSavepointFile(String basePath, String commitTime) throws IOException { - Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeSavePointFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - FSDataOutputStream os = fs.create(commitFile, true); - HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - try { - // Write empty commit metadata - os.writeBytes(new String(commitMetadata.toJsonString().getBytes( - StandardCharsets.UTF_8))); - } finally { - os.close(); - } - } + static class KeyPartition { - public String[] getPartitionPaths() { - return partitionPaths; + HoodieKey key; + String partitionPath; } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java b/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java index 11c790f665a6c..b73cba791169d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java @@ -43,15 +43,15 @@ */ public class TestRawTripPayload implements HoodieRecordPayload { - private transient static final ObjectMapper mapper = new ObjectMapper(); + private static final transient ObjectMapper mapper = new ObjectMapper(); private String partitionPath; private String rowKey; private byte[] jsonDataCompressed; private int dataSize; private boolean isDeleted; - public TestRawTripPayload(Optional jsonData, String rowKey, String partitionPath, - String schemaStr, Boolean isDeleted) throws IOException { + public TestRawTripPayload(Optional jsonData, String rowKey, String partitionPath, String schemaStr, + Boolean isDeleted) throws IOException { if (jsonData.isPresent()) { this.jsonDataCompressed = compressData(jsonData.get()); this.dataSize = jsonData.get().length(); @@ -61,8 +61,7 @@ public TestRawTripPayload(Optional jsonData, String rowKey, String parti this.isDeleted = isDeleted; } - public TestRawTripPayload(String jsonData, String rowKey, String partitionPath, - String schemaStr) throws IOException { + public TestRawTripPayload(String jsonData, String rowKey, String partitionPath, String schemaStr) throws IOException { this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false); } @@ -86,8 +85,7 @@ public TestRawTripPayload preCombine(TestRawTripPayload another) { } @Override - public Optional combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) - throws IOException { + public Optional combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException { return this.getInsertValue(schema); } @@ -120,8 +118,7 @@ public String getJsonData() throws IOException { private byte[] compressData(String jsonData) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DeflaterOutputStream dos = - new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true); + DeflaterOutputStream dos = new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true); try { dos.write(jsonData.getBytes()); } finally { @@ -140,59 +137,54 @@ private String unCompressData(byte[] data) throws IOException { } /** - * A custom {@link WriteStatus} that merges passed metadata key value map to {@code - * WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}. + * A custom {@link WriteStatus} that merges passed metadata key value map to {@code WriteStatus.markSuccess()} and + * {@code WriteStatus.markFailure()}. */ public static class MetadataMergeWriteStatus extends WriteStatus { private Map mergedMetadataMap = new HashMap<>(); - @Override - public void markSuccess(HoodieRecord record, Optional> recordMetadata) { - super.markSuccess(record, recordMetadata); - if (recordMetadata.isPresent()) { - mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); - } - } - - @Override - public void markFailure(HoodieRecord record, Throwable t, - Optional> recordMetadata) { - super.markFailure(record, t, recordMetadata); - if (recordMetadata.isPresent()) { - mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); - } - } - - public static Map mergeMetadataForWriteStatuses( - List writeStatuses) { + public static Map mergeMetadataForWriteStatuses(List writeStatuses) { Map allWriteStatusMergedMetadataMap = new HashMap<>(); for (WriteStatus writeStatus : writeStatuses) { - MetadataMergeWriteStatus.mergeMetadataMaps( - ((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(), + MetadataMergeWriteStatus.mergeMetadataMaps(((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(), allWriteStatusMergedMetadataMap); } return allWriteStatusMergedMetadataMap; } - private static void mergeMetadataMaps(Map mergeFromMap, - Map mergeToMap) { + private static void mergeMetadataMaps(Map mergeFromMap, Map mergeToMap) { for (Entry entry : mergeFromMap.entrySet()) { String key = entry.getKey(); if (!mergeToMap.containsKey(key)) { mergeToMap.put(key, "0"); } - mergeToMap - .put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key))); + mergeToMap.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key))); } } - private Map getMergedMetadataMap() { - return mergedMetadataMap; - } - private static String addStrsAsInt(String a, String b) { return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b)); } + + @Override + public void markSuccess(HoodieRecord record, Optional> recordMetadata) { + super.markSuccess(record, recordMetadata); + if (recordMetadata.isPresent()) { + mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); + } + } + + @Override + public void markFailure(HoodieRecord record, Throwable t, Optional> recordMetadata) { + super.markFailure(record, t, recordMetadata); + if (recordMetadata.isPresent()) { + mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); + } + } + + private Map getMergedMetadataMap() { + return mergedMetadataMap; + } } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java b/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java index 395197bf98461..3e6fc8e680fb8 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java @@ -49,8 +49,7 @@ public void testPropertyLoading() throws IOException { assertEquals(config.getMinCommitsToKeep(), 2); } - private ByteArrayOutputStream saveParamsIntoOutputStream(Map params) - throws IOException { + private ByteArrayOutputStream saveParamsIntoOutputStream(Map params) throws IOException { Properties properties = new Properties(); properties.putAll(params); ByteArrayOutputStream outStream = new ByteArrayOutputStream(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java index 35e36ef1dd5dd..6c47f2d4c3e4d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java @@ -16,18 +16,13 @@ package com.uber.hoodie.func; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.exception.HoodieException; -import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.io.FileUtils; -import org.apache.spark.util.SizeEstimator; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -37,171 +32,168 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.Semaphore; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.io.FileUtils; +import org.apache.spark.util.SizeEstimator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; public class TestBufferedIterator { - private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); - private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); - private ExecutorService recordReader = null; + private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); + private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); + private ExecutorService recordReader = null; - @Before - public void beforeTest() { - this.recordReader = Executors.newFixedThreadPool(1); - } + @Before + public void beforeTest() { + this.recordReader = Executors.newFixedThreadPool(1); + } - @After - public void afterTest() { - if (this.recordReader != null) { - this.recordReader.shutdownNow(); - this.recordReader = null; - } + @After + public void afterTest() { + if (this.recordReader != null) { + this.recordReader.shutdownNow(); + this.recordReader = null; } - - // Test to ensure that we are reading all records from buffered iterator in the same order without any exceptions. - @Test(timeout = 60000) - public void testRecordReading() throws IOException, ExecutionException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - final BufferedIterator bufferedIterator = - new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, HoodieTestDataGenerator.avroSchema); - Future result = - recordReader.submit( - () -> { - bufferedIterator.startBuffering(); - return true; - } - ); - final Iterator originalRecordIterator = hoodieRecords.iterator(); - int recordsRead = 0; - while (bufferedIterator.hasNext()) { - final HoodieRecord originalRecord = originalRecordIterator.next(); - final Optional originalInsertValue = - originalRecord.getData().getInsertValue(HoodieTestDataGenerator.avroSchema); - final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next(); - // Ensure that record ordering is guaranteed. - Assert.assertEquals(originalRecord, payload.record); - // cached insert value matches the expected insert value. - Assert.assertEquals(originalInsertValue, payload.insertValue); - recordsRead++; - } - Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); - // all the records should be read successfully. - Assert.assertEquals(numRecords, recordsRead); - // should not throw any exceptions. - Assert.assertTrue(result.get()); + } + + // Test to ensure that we are reading all records from buffered iterator in the same order + // without any exceptions. + @Test(timeout = 60000) + public void testRecordReading() throws IOException, ExecutionException, InterruptedException { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, + HoodieTestDataGenerator.avroSchema); + Future result = recordReader.submit(() -> { + bufferedIterator.startBuffering(); + return true; + }); + final Iterator originalRecordIterator = hoodieRecords.iterator(); + int recordsRead = 0; + while (bufferedIterator.hasNext()) { + final HoodieRecord originalRecord = originalRecordIterator.next(); + final Optional originalInsertValue = originalRecord.getData() + .getInsertValue(HoodieTestDataGenerator.avroSchema); + final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next(); + // Ensure that record ordering is guaranteed. + Assert.assertEquals(originalRecord, payload.record); + // cached insert value matches the expected insert value. + Assert.assertEquals(originalInsertValue, payload.insertValue); + recordsRead++; } - - // Test to ensure that record buffering is throttled when we hit memory limit. - @Test(timeout = 60000) - public void testMemoryLimitForBuffering() throws IOException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // maximum number of records to keep in memory. - final int recordLimit = 5; - final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); - final BufferedIterator bufferedIterator = - new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); - Future result = - recordReader.submit( - () -> { - bufferedIterator.startBuffering(); - return true; - } - ); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); - - // try to read 2 records. - Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record); - Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record); - - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - // No change is expected in rate limit or number of buffered records. We only expect buffering thread to read - // 2 more records into the buffer. - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); + Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); + // all the records should be read successfully. + Assert.assertEquals(numRecords, recordsRead); + // should not throw any exceptions. + Assert.assertTrue(result.get()); + } + + // Test to ensure that record buffering is throttled when we hit memory limit. + @Test(timeout = 60000) + public void testMemoryLimitForBuffering() throws IOException, InterruptedException { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // maximum number of records to keep in memory. + final int recordLimit = 5; + final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); + final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, + HoodieTestDataGenerator.avroSchema); + Future result = recordReader.submit(() -> { + bufferedIterator.startBuffering(); + return true; + }); + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator.rateLimiter)) { + Thread.sleep(10); } - - // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread is propagated to - // another thread. - @Test(timeout = 60000) - public void testException() throws IOException, InterruptedException { - final int numRecords = 256; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // buffer memory limit - final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); - - // first let us throw exception from bufferIterator reader and test that buffering thread stops and throws - // correct exception back. - BufferedIterator bufferedIterator1 = - new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); - Future result = - recordReader.submit( - () -> { - bufferedIterator1.startBuffering(); - return true; - } - ); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator1.rateLimiter)) { - Thread.sleep(10); - } - // notify buffering thread of an exception and ensure that it exits. - final Exception e = new Exception("Failing it :)"); - bufferedIterator1.markAsFailed(e); - try { - result.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e1) { - Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); - Assert.assertEquals(e, e1.getCause().getCause()); - } - - // second let us raise an exception while doing record buffering. this exception should get propagated to - // buffered iterator reader. - final RuntimeException expectedException = new RuntimeException("failing record reading"); - final Iterator mockHoodieRecordsIterator = mock(Iterator.class); - when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); - when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); - BufferedIterator bufferedIterator2 = - new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); - Future result2 = - recordReader.submit( - () -> { - bufferedIterator2.startBuffering(); - return true; - } - ); - try { - bufferedIterator2.hasNext(); - Assert.fail("exception is expected"); - } catch (Exception e1) { - Assert.assertEquals(expectedException, e1.getCause()); - } - // buffering thread should also have exited. make sure that it is not running. - try { - result2.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e2) { - Assert.assertEquals(expectedException, e2.getCause()); - } + Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); + Assert.assertEquals(recordLimit, bufferedIterator.size()); + Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); + + // try to read 2 records. + Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record); + Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record); + + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator.rateLimiter)) { + Thread.sleep(10); + } + // No change is expected in rate limit or number of buffered records. We only expect + // buffering thread to read + // 2 more records into the buffer. + Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); + Assert.assertEquals(recordLimit, bufferedIterator.size()); + Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); + } + + // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread + // is propagated to + // another thread. + @Test(timeout = 60000) + public void testException() throws IOException, InterruptedException { + final int numRecords = 256; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // buffer memory limit + final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); + + // first let us throw exception from bufferIterator reader and test that buffering thread + // stops and throws + // correct exception back. + BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, + HoodieTestDataGenerator.avroSchema); + Future result = recordReader.submit(() -> { + bufferedIterator1.startBuffering(); + return true; + }); + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator1.rateLimiter)) { + Thread.sleep(10); + } + // notify buffering thread of an exception and ensure that it exits. + final Exception e = new Exception("Failing it :)"); + bufferedIterator1.markAsFailed(e); + try { + result.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e1) { + Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); + Assert.assertEquals(e, e1.getCause().getCause()); } - private boolean isQueueFull(Semaphore rateLimiter) { - return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); + // second let us raise an exception while doing record buffering. this exception should get + // propagated to + // buffered iterator reader. + final RuntimeException expectedException = new RuntimeException("failing record reading"); + final Iterator mockHoodieRecordsIterator = mock(Iterator.class); + when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); + when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); + BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, + HoodieTestDataGenerator.avroSchema); + Future result2 = recordReader.submit(() -> { + bufferedIterator2.startBuffering(); + return true; + }); + try { + bufferedIterator2.hasNext(); + Assert.fail("exception is expected"); + } catch (Exception e1) { + Assert.assertEquals(expectedException, e1.getCause()); } + // buffering thread should also have exited. make sure that it is not running. + try { + result2.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e2) { + Assert.assertEquals(expectedException, e2.getCause()); + } + } + + private boolean isQueueFull(Semaphore rateLimiter) { + return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index 0316372cad5cf..dcd5a94a5eb45 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -55,32 +55,24 @@ public void init() throws Exception { public void testSchemaEvolutionOnUpdate() throws Exception { // Create a bunch of records with a old version of schema HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient( - HoodieTestUtils.getDefaultHadoopConf(), basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); - String recordStr1 = - "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = - "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = - "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; List records = new ArrayList<>(); TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - records.add( - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1)); + records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - records.add( - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), - rowChange2)); + records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - records.add( - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), - rowChange3)); + records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); Iterator> insertResult = table.handleInsert("100", records.iterator()); - Path commitFile = - new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); + Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile); // Now try an update with an evolved schema @@ -92,21 +84,20 @@ public void testSchemaEvolutionOnUpdate() throws Exception { table = new HoodieCopyOnWriteTable(config, metaClient); // New content with values for the newly added field - recordStr1 = - "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; + recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; records = new ArrayList<>(); rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); record1.setCurrentLocation(new HoodieRecordLocation("100", fileId)); records.add(record1); try { table.handleUpdate("101", fileId, records.iterator()); } catch (ClassCastException e) { - fail( - "UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt"); + fail("UpdateFunction could not read records written with exampleSchema.txt using the " + + "exampleEvolvedSchema.txt"); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java index 89f31fa2e9bc5..eb4a878f903a8 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -16,6 +16,12 @@ package com.uber.hoodie.index; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.times; + import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieTestDataGenerator; @@ -23,19 +29,16 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableConfig; import com.uber.hoodie.common.table.HoodieTableMetaClient; -import com.uber.hoodie.common.table.TableFileSystemView; -import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.hbase.HBaseIndex; import com.uber.hoodie.table.HoodieTable; +import java.io.File; +import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -56,37 +59,26 @@ import org.junit.rules.TemporaryFolder; import org.junit.runners.MethodSorters; import org.mockito.Mockito; -import scala.Tuple2; - -import java.io.File; -import java.io.IOException; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.atMost; -import static org.mockito.Mockito.times; /** - * Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown - * across tests, (see one problem here : https://issues.apache.org/jira/browse/HBASE-15835). - * Hence, the need to use MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter - * the order of tests running carefully. + * Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown across tests, + * (see one problem here : https://issues.apache .org/jira/browse/HBASE-15835). Hence, the need to use + * MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter the order of tests running carefully. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class TestHbaseIndex { private static JavaSparkContext jsc = null; - private String basePath = null; - private transient FileSystem fs; private static HBaseTestingUtility utility; private static Configuration hbaseConfig; private static String tableName = "test_table"; + private String basePath = null; + private transient FileSystem fs; private HoodieTableMetaClient metaClient; + public TestHbaseIndex() throws Exception { + } + @AfterClass public static void clean() throws Exception { if (jsc != null) { @@ -97,6 +89,20 @@ public static void clean() throws Exception { } } + @BeforeClass + public static void init() throws Exception { + + // Initialize HbaseMiniCluster + utility = new HBaseTestingUtility(); + utility.startMiniCluster(); + hbaseConfig = utility.getConnection().getConfiguration(); + utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s")); + // Initialize a local spark env + SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]"); + jsc = new JavaSparkContext(sparkConf); + jsc.hadoopConfiguration().addResource(utility.getConfiguration()); + } + @After public void clear() throws Exception { if (basePath != null) { @@ -112,25 +118,8 @@ public void before() throws Exception { basePath = folder.getRoot().getAbsolutePath(); // Initialize table metaClient = HoodieTableMetaClient - .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, - tableName, HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); - } - - public TestHbaseIndex() throws Exception { - } - - @BeforeClass - public static void init() throws Exception { - - // Initialize HbaseMiniCluster - utility = new HBaseTestingUtility(); - utility.startMiniCluster(); - hbaseConfig = utility.getConnection().getConfiguration(); - utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s")); - // Initialize a local spark env - SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]"); - jsc = new JavaSparkContext(sparkConf); - jsc.hadoopConfiguration().addResource(utility.getConfiguration()); + .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName, + HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); } @Test @@ -156,7 +145,8 @@ public void testSimpleTagLocationAndUpdate() throws Exception { JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); assertNoWriteErrors(writeStatues.collect()); - // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed commit + // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed + // commit javaRDD = index.tagLocation(writeRecords, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); @@ -167,8 +157,9 @@ public void testSimpleTagLocationAndUpdate() throws Exception { javaRDD = index.tagLocation(writeRecords, hoodieTable); assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); - assertTrue(javaRDD.filter(record -> (record.getCurrentLocation() != null - && record.getCurrentLocation().getCommitTime().equals(newCommitTime))).distinct().count() == 200); + assertTrue(javaRDD.filter( + record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getCommitTime() + .equals(newCommitTime))).distinct().count() == 200); } @@ -208,7 +199,8 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Rollback the last commit writeClient.rollback(newCommitTime); - // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled back commit + // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled + // back commit javaRDD = index.tagLocation(writeRecords, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); @@ -302,12 +294,10 @@ private HoodieWriteConfig getConfig() { } private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(1, 1) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) - .withInlineCompaction(false).build()) - .withAutoCommit(false) + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(1, 1).withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false) + .build()).withAutoCommit(false) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) .forTable("test-trip-table").withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java index 2fba006930de8..16f754a4bd491 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java @@ -31,16 +31,14 @@ public void testCreateIndex() throws Exception { HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); // Different types - HoodieWriteConfig config = clientConfigBuilder.withPath("") - .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()) - .build(); + HoodieWriteConfig config = clientConfigBuilder.withPath("").withIndexConfig( + indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build(); assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex); - config = clientConfigBuilder.withPath("").withIndexConfig( - indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + config = clientConfigBuilder.withPath("") + .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex); config = clientConfigBuilder.withPath("") - .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 4bf91001f3b24..8e649c73c2076 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -98,31 +98,33 @@ public void init() throws IOException { @Test public void testLoadUUIDsInMemory() throws IOException { // Create one RDD of hoodie record - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); - JavaRDD recordRDD = jsc - .parallelize(Arrays.asList(record1, record2, record3, record4)); + JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); // Load to memory - Map> map = recordRDD - .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())) - .groupByKey().collectAsMap(); + Map> map = recordRDD.mapToPair( + record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())).groupByKey().collectAsMap(); assertEquals(map.size(), 2); List list1 = Lists.newArrayList(map.get("2016/01/31")); List list2 = Lists.newArrayList(map.get("2015/01/31")); @@ -132,44 +134,40 @@ public void testLoadUUIDsInMemory() throws IOException { @Test public void testLoadInvolvedFiles() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder() - .withPath(basePath) - .build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); // Create some partitions, and put some files // "2016/01/21": 0 file // "2016/04/01": 1 file (2_0_20160401010101.parquet) - // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet) + // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, + // 4_0_20150312101010.parquet) new File(basePath + "/2016/01/21").mkdirs(); new File(basePath + "/2016/04/01").mkdirs(); new File(basePath + "/2015/03/12").mkdirs(); TestRawTripPayload rowChange1 = new TestRawTripPayload( "{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload( "{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload( "{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload( "{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); - writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, - false); - writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, + writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, false); + writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, false); + writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema, null, false); + writeParquetFile("2015/03/12", "4_0_20150312101010.parquet", Arrays.asList(record2, record3, record4), schema, null, false); - writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema, - null, false); - writeParquetFile("2015/03/12", "4_0_20150312101010.parquet", - Arrays.asList(record2, record3, record4), schema, null, false); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -198,51 +196,32 @@ public void testLoadInvolvedFiles() throws IOException { List> expected = Arrays.asList( new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")), new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1_0_20150312101010.parquet")), - new Tuple2<>("2015/03/12", - new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")), - new Tuple2<>("2015/03/12", - new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003")) - ); + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")), + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"))); assertEquals(expected, filesList); } @Test public void testRangePruning() { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder() - .withPath(basePath) - .build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); final Map> partitionToFileIndexInfo = new HashMap<>(); - partitionToFileIndexInfo.put("2017/10/22", Arrays.asList( - new BloomIndexFileInfo("f1"), - new BloomIndexFileInfo("f2", "000", "000"), - new BloomIndexFileInfo("f3", "001", "003"), - new BloomIndexFileInfo("f4", "002", "007"), - new BloomIndexFileInfo("f5", "009", "010") - )); - - JavaPairRDD partitionRecordKeyPairRDD = jsc - .parallelize(Arrays.asList( - new Tuple2<>("2017/10/22", "003"), - new Tuple2<>("2017/10/22", "002"), - new Tuple2<>("2017/10/22", "005"), - new Tuple2<>("2017/10/22", "004") - )) - .mapToPair(t -> t); - - List>> comparisonKeyList = index - .explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD) - .collect(); + partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), + new BloomIndexFileInfo("f2", "000", "000"), new BloomIndexFileInfo("f3", "001", "003"), + new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010"))); + + JavaPairRDD partitionRecordKeyPairRDD = jsc.parallelize(Arrays.asList( + new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), + new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t); + + List>> comparisonKeyList = index.explodeRecordRDDWithFileComparisons( + partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect(); assertEquals(10, comparisonKeyList.size()); - Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(Collectors.groupingBy( - t -> t._2()._2().getRecordKey(), - Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList() - ) - )); + Map> recordKeyToFileComps = comparisonKeyList.stream().collect(Collectors.groupingBy( + t -> t._2()._2().getRecordKey(), Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002")); @@ -252,32 +231,35 @@ public void testRangePruning() { } @Test - public void testCheckUUIDsAgainstOneFile() - throws IOException, InterruptedException, ClassNotFoundException { + public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException, ClassNotFoundException { // Create some records to use - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); - // We write record1, record2 to a parquet file, but the bloom filter contains (record1, record2, record3). + // We write record1, record2 to a parquet file, but the bloom filter contains (record1, + // record2, record3). BloomFilter filter = new BloomFilter(10000, 0.0000001); filter.add(record3.getRecordKey()); - String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, - filter, true); + String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, filter, true); // The bloom filter contains 3 records assertTrue(filter.mightContain(record1.getRecordKey())); @@ -286,17 +268,16 @@ public void testCheckUUIDsAgainstOneFile() assertFalse(filter.mightContain(record4.getRecordKey())); // Compare with file - List uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), - record3.getRecordKey(), record4.getRecordKey()); + List uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), + record4.getRecordKey()); - List results = HoodieBloomIndexCheckFunction - .checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids, + List results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids, new Path(basePath + "/2016/01/31/" + filename)); assertEquals(results.size(), 2); - assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") - || results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); - assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0") - || results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")); + assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") || results.get(1).equals( + "1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); + assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0") || results.get(1).equals( + "2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")); // TODO(vc): Need more coverage on actual filenames //assertTrue(results.get(0)._2().equals(filename)); //assertTrue(results.get(1)._2().equals(filename)); @@ -317,8 +298,7 @@ public void testTagLocationWithEmptyRDD() throws Exception { try { bloomIndex.tagLocation(recordRDD, table); } catch (IllegalArgumentException e) { - fail( - "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required"); + fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required"); } } @@ -327,24 +307,27 @@ public void testTagLocationWithEmptyRDD() throws Exception { public void testTagLocation() throws Exception { // We have some records to be tagged (two different partitions) - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); - JavaRDD recordRDD = jsc - .parallelize(Arrays.asList(record1, record2, record3, record4)); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); + JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -389,10 +372,14 @@ public void testTagLocation() throws Exception { public void testCheckExists() throws Exception { // We have some records to be tagged (two different partitions) - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()); HoodieRecord record1 = new HoodieRecord(key1, rowChange1); @@ -414,8 +401,7 @@ public void testCheckExists() throws Exception { // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaPairRDD> taggedRecordRDD = bloomIndex - .fetchRecordLocation(keysRDD, table); + JavaPairRDD> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); // Should not find any files for (Tuple2> record : taggedRecordRDD.collect()) { @@ -456,16 +442,18 @@ public void testCheckExists() throws Exception { @Test public void testBloomFilterFalseError() throws IOException, InterruptedException { // We have two hoodie records - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; // We write record1 to a parquet file, using a bloom filter having both records TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); BloomFilter filter = new BloomFilter(10000, 0.0000001); filter.add(record2.getRecordKey()); @@ -492,8 +480,8 @@ public void testBloomFilterFalseError() throws IOException, InterruptedException } } - private String writeParquetFile(String partitionPath, List records, Schema schema, - BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException { + private String writeParquetFile(String partitionPath, List records, Schema schema, BloomFilter filter, + boolean createCommitTime) throws IOException, InterruptedException { Thread.sleep(1000); String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); String fileId = UUID.randomUUID().toString(); @@ -502,15 +490,14 @@ private String writeParquetFile(String partitionPath, List records return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime); } - private String writeParquetFile(String partitionPath, String filename, List records, - Schema schema, + private String writeParquetFile(String partitionPath, String filename, List records, Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException { if (filter == null) { filter = new BloomFilter(10000, 0.0000001); } - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(schema), schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, + filter); String commitTime = FSUtils.getCommitTime(filename); HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, @@ -525,9 +512,7 @@ private String writeParquetFile(String partitionPath, String filename, List originalCommits = timeline.getInstants().collect(Collectors.toList()); assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants()); @@ -118,13 +117,12 @@ public void testArchiveDatasetWithArchival() throws IOException { assertTrue(archiveLog.archiveIfRequired()); //reload the timeline and remove the remaining commits - timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); //read the file - HoodieLogFormat.Reader reader = HoodieLogFormat - .newReader(fs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, + new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema()); int archivedRecordsCount = 0; @@ -137,8 +135,7 @@ public void testArchiveDatasetWithArchival() throws IOException { assertEquals("Archived and read records for each block are same", 8, records.size()); archivedRecordsCount += records.size(); } - assertEquals("Total archived records and total read records are the same count", 8, - archivedRecordsCount); + assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount); //make sure the archived commits are the same as the (originalcommits - commitsleft) List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> { @@ -146,10 +143,8 @@ public void testArchiveDatasetWithArchival() throws IOException { }).collect(Collectors.toList()); Collections.sort(readCommits); - assertEquals( - "Read commits map should match the originalCommits - commitsLoadedFromArchival", - originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), - readCommits); + assertEquals("Read commits map should match the originalCommits - commitsLoadedFromArchival", + originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), readCommits); // verify in-flight instants after archive verifyInflightInstants(metaClient, 3); @@ -168,15 +163,12 @@ public void testArchiveDatasetWithNoArchival() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "102"); HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); - assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, - timeline.countInstants()); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); } @Test @@ -194,21 +186,15 @@ public void testArchiveCommitSafety() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("100")); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("101")); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("102")); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("103")); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("101")); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("102")); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("103")); } @Test @@ -227,16 +213,14 @@ public void testArchiveCommitSavepointNoHole() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals( - "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)", - 5, timeline.countInstants()); + "Since we have a savepoint at 101, we should never archive any commit after 101 (we only " + "archive 100)", 5, + timeline.countInstants()); assertTrue("Archived commits should always be safe", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101"))); assertTrue("Archived commits should always be safe", @@ -248,7 +232,7 @@ public void testArchiveCommitSavepointNoHole() throws IOException { private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { HoodieTimeline timeline = metaClient.getActiveTimeline().reload() .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights(); - assertEquals("Loaded inflight clean actions and the count should match", - expectedTotalInstants, timeline.countInstants()); + assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants, + timeline.countInstants()); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index 5a1ff7ba80a0d..2c93f3d181dd5 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -93,32 +93,27 @@ private HoodieWriteConfig getConfig() { } private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) - .withInlineCompaction(false).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) - .forTable("test-trip-table").withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false) + .build()).withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } @Test(expected = IllegalArgumentException.class) public void testCompactionOnCopyOnWriteFail() throws Exception { HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); } @Test public void testCompactionEmpty() throws Exception { - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = getConfig(); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); String newCommitTime = writeClient.startCommit(); @@ -126,10 +121,9 @@ public void testCompactionEmpty() throws Exception { JavaRDD recordsRDD = jsc.parallelize(records, 1); writeClient.insert(recordsRDD, newCommitTime).collect(); - JavaRDD result = - compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); - assertTrue("If there is nothing to compact, result will be empty", - result.isEmpty()); + JavaRDD result = compactor + .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + assertTrue("If there is nothing to compact, result will be empty", result.isEmpty()); } @Test @@ -145,8 +139,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); newCommitTime = "101"; @@ -159,19 +152,16 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, - updatedRecords); + .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, updatedRecords); // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toList()); + List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toList()); for (FileSlice fileSlice : groupedLogFiles) { - assertEquals("There should be 1 log file written for every data file", 1, - fileSlice.getLogFiles().count()); + assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count()); } } @@ -179,18 +169,19 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); - JavaRDD result = - compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + JavaRDD result = compactor + .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { List writeStatuses = result.collect(); assertTrue(writeStatuses.stream() - .filter(writeStatus -> writeStatus.getStat().getPartitionPath() - .contentEquals(partitionPath)).count() > 0); + .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)) + .count() > 0); } } - // TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make sure the data read is the updated data (compaction correctness) + // TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make + // sure the data read is the updated data (compaction correctness) // TODO - add more test cases for compactions after a failed commit/compaction } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index 55898c37367d7..c3aee4f2bb041 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -16,6 +16,9 @@ package com.uber.hoodie.io.strategy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import com.beust.jcommander.internal.Lists; import com.google.common.collect.Maps; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -25,20 +28,16 @@ import com.uber.hoodie.io.compact.strategy.DayBasedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; -import org.junit.Test; - import java.util.List; import java.util.Map; import java.util.Random; import java.util.stream.Collectors; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.junit.Test; public class TestHoodieCompactionStrategy { private static final long MB = 1024 * 1024L; - private String [] partitionPaths = {"2017/01/01", "2017/01/02", "2017/01/03"}; + private String[] partitionPaths = {"2017/01/01", "2017/01/02", "2017/01/03"}; @Test public void testUnBounded() { @@ -48,9 +47,8 @@ public void testUnBounded() { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); assertEquals("UnBounded should not re-order or filter", operations, returned); @@ -64,23 +62,19 @@ public void testBoundedIOSimple() { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); BoundedIOCompactionStrategy strategy = new BoundedIOCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) - .withTargetIOPerCompactionInMB(400).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); - assertTrue("BoundedIOCompaction should have resulted in fewer compactions", - returned.size() < operations.size()); - assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", - 2, returned.size()); + assertTrue("BoundedIOCompaction should have resulted in fewer compactions", returned.size() < operations.size()); + assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", 2, returned.size()); // Total size of all the log files - Long returnedSize = returned.stream() - .map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)).map(s -> (Long) s) - .reduce((size1, size2) -> size1 + size2).orElse(0L); - assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", - 610, (long) returnedSize); + Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) + .map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L); + assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 610, + (long) returnedSize); } @Test @@ -91,23 +85,20 @@ public void testLogFileSizeCompactionSimple() { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); LogFileSizeBasedCompactionStrategy strategy = new LogFileSizeBasedCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) - .withTargetIOPerCompactionInMB(400).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); assertTrue("LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions", returned.size() < operations.size()); - assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", - 1, returned.size()); + assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", 1, returned.size()); // Total size of all the log files - Long returnedSize = returned.stream() - .map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)).map(s -> (Long) s) - .reduce((size1, size2) -> size1 + size2).orElse(0L); - assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", - 1204, (long) returnedSize); + Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) + .map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L); + assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 1204, + (long) returnedSize); } @Test @@ -118,20 +109,18 @@ public void testPartitionAwareCompactionSimple() { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) - .withTargetIOPerCompactionInMB(400).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); assertTrue("DayBasedCompactionStrategy should have resulted in fewer compactions", - returned.size() < operations.size()); + returned.size() < operations.size()); int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1), returned.get(0)); // Either the partition paths are sorted in descending order or they are equal - assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", - comparision >= 0); + assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", comparision >= 0); } private List createCompactionOperations(HoodieWriteConfig config, @@ -139,9 +128,8 @@ private List createCompactionOperations(HoodieWriteConfig c List operations = Lists.newArrayList(sizesMap.size()); sizesMap.forEach((k, v) -> { operations.add(new CompactionOperation(TestHoodieDataFile.newDataFile(k), - partitionPaths[new Random().nextInt(partitionPaths.length - 1)], - v.stream().map(TestHoodieLogFile::newLogFile).collect( - Collectors.toList()), config)); + partitionPaths[new Random().nextInt(partitionPaths.length - 1)], + v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config)); }); return operations; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java index 564d95218912c..88058b3e343b0 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java @@ -28,6 +28,10 @@ public TestHoodieDataFile(long size) { this.size = size; } + public static HoodieDataFile newDataFile(long size) { + return new TestHoodieDataFile(size); + } + @Override public String getPath() { return "/tmp/test"; @@ -43,13 +47,8 @@ public String getCommitTime() { return "100"; } - @Override public long getFileSize() { return size; } - - public static HoodieDataFile newDataFile(long size) { - return new TestHoodieDataFile(size); - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java index d23cbf27ca433..df46ec214e6be 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java @@ -29,6 +29,10 @@ public TestHoodieLogFile(long size) { this.size = size; } + public static HoodieLogFile newLogFile(long size) { + return new TestHoodieLogFile(size); + } + @Override public Path getPath() { return new Path("/tmp/test-log"); @@ -38,8 +42,4 @@ public Path getPath() { public Optional getFileSize() { return Optional.of(size); } - - public static HoodieLogFile newLogFile(long size) { - return new TestHoodieLogFile(size); - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java b/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java index 911f97417d2d7..cb1a439698f87 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java @@ -40,7 +40,6 @@ public void start() throws ConfigurationException { @Test public void testRegisterGauge() { metrics.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")); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index ab9ba8be4ed16..ebd31b25a13d7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -89,14 +89,13 @@ public void testMakeNewPath() throws Exception { String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieWriteConfig config = makeHoodieClientConfig(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath); Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName); - assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils - .makeDataFileName(commitTime, unitNumber, fileName))); + assertTrue(newPath.toString().equals( + this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName))); } private HoodieWriteConfig makeHoodieClientConfig() throws Exception { @@ -105,8 +104,7 @@ private HoodieWriteConfig makeHoodieClientConfig() throws Exception { private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception { // Prepare the AvroParquetIO - String schemaStr = IOUtils - .toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); + String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr); } @@ -122,28 +120,27 @@ public void testUpdateRecords() throws Exception { HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); // Get some records belong to the same partition (2016/01/31) - String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}"; + String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}"; List records = new ArrayList<>(); TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - records.add( - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1)); + records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - records.add( - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), - rowChange2)); + records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - records.add( - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), - rowChange3)); + records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator())); - // We should have a parquet file generated (TODO: better control # files after we revise AvroParquetIO) + // We should have a parquet file generated (TODO: better control # files after we revise + // AvroParquetIO) File parquetFile = null; for (File file : new File(this.basePath + partitionPath).listFiles()) { if (file.getName().endsWith(".parquet")) { @@ -155,18 +152,17 @@ public void testUpdateRecords() throws Exception { // Read out the bloom filter and make sure filter can answer record exist or not Path parquetFilePath = new Path(parquetFile.getAbsolutePath()); - BloomFilter filter = ParquetUtils - .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath); + BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath); for (HoodieRecord record : records) { assertTrue(filter.mightContain(record.getRecordKey())); } // Create a commit file - new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile(); + new File( + this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + FSUtils.getCommitTime(parquetFile.getName()) + + ".commit").createNewFile(); // Read the parquet file, check the record content - List fileRecords = ParquetUtils - .readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath); + List fileRecords = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath); GenericRecord newRecord; int index = 0; for (GenericRecord record : fileRecords) { @@ -175,13 +171,12 @@ public void testUpdateRecords() throws Exception { } // We update the 1st record & add a new record - String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1); HoodieRecord updatedRecord1 = new HoodieRecord( - new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), - updateRowChanges1); - updatedRecord1.setCurrentLocation( - new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName()))); + new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1); + updatedRecord1.setCurrentLocation(new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName()))); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); HoodieRecord insertedRecord1 = new HoodieRecord( @@ -201,10 +196,9 @@ public void testUpdateRecords() throws Exception { File updatedParquetFile = null; for (File file : new File(basePath + "/2016/01/31").listFiles()) { if (file.getName().endsWith(".parquet")) { - if (FSUtils.getFileId(file.getName()) - .equals(FSUtils.getFileId(parquetFile.getName())) && - HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()), - FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) { + if (FSUtils.getFileId(file.getName()).equals(FSUtils.getFileId(parquetFile.getName())) + && HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()), + FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) { updatedParquetFile = file; break; } @@ -213,8 +207,8 @@ public void testUpdateRecords() throws Exception { assertTrue(updatedParquetFile != null); // Check whether the record has been updated Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath()); - BloomFilter updatedFilter = ParquetUtils - .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath); + BloomFilter updatedFilter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), + updatedParquetFilePath); for (HoodieRecord record : records) { // No change to the _row_key assertTrue(updatedFilter.mightContain(record.getRecordKey())); @@ -223,8 +217,7 @@ public void testUpdateRecords() throws Exception { assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey())); records.add(insertedRecord1);// add this so it can further check below - ParquetReader updatedReader = ParquetReader - .builder(new AvroReadSupport<>(), updatedParquetFilePath).build(); + ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build(); index = 0; while ((newRecord = (GenericRecord) updatedReader.read()) != null) { assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey())); @@ -246,13 +239,9 @@ private List newHoodieRecords(int n, String time) throws Exception List records = new ArrayList<>(); for (int i = 0; i < n; i++) { String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", - UUID.randomUUID().toString(), - time, - i); + UUID.randomUUID().toString(), time, i); TestRawTripPayload rowChange = new TestRawTripPayload(recordStr); - records.add(new HoodieRecord( - new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), - rowChange)); + records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } return records; } @@ -261,31 +250,28 @@ private List newHoodieRecords(int n, String time) throws Exception @Test public void testMetadataAggregateFromWriteStatus() throws Exception { // Prepare the AvroParquetIO - HoodieWriteConfig config = makeHoodieClientConfigBuilder() - .withWriteStatusClass(MetadataMergeWriteStatus.class).build(); + HoodieWriteConfig config = makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class) + .build(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); // Get some records belong to the same partition (2016/01/31) - String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; List records = new ArrayList<>(); TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - records.add( - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1)); + records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - records.add( - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), - rowChange2)); + records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - records.add( - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), - rowChange3)); + records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records List writeStatuses = HoodieClientTestUtils @@ -293,7 +279,8 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { Map allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus .mergeMetadataForWriteStatuses(writeStatuses); assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000")); - // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * 3 + // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this + // should be 2 * 3 assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000")); } @@ -314,26 +301,19 @@ public void testInsertWithPartialFailures() throws Exception { List statuses = HoodieClientTestUtils .collectStatuses(table.handleInsert(commitTime, records.iterator())); WriteStatus status = statuses.get(0); - Path partialFile = new Path(String.format("%s/%s/%s", - basePath, - status.getPartitionPath(), - FSUtils.makeDataFileName(commitTime, 0, status.getFileId())) - ); + Path partialFile = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(), + FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))); assertTrue(fs.exists(partialFile)); // When we retry records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z"); records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z")); - statuses = HoodieClientTestUtils - .collectStatuses(table.handleInsert(commitTime, records.iterator())); + statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator())); status = statuses.get(0); - Path retriedFIle = new Path(String.format("%s/%s/%s", - basePath, - status.getPartitionPath(), - FSUtils.makeDataFileName(commitTime, 0, status.getFileId())) - ); + Path retriedFIle = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(), + FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))); assertTrue(fs.exists(retriedFIle)); assertFalse(fs.exists(partialFile)); } @@ -371,8 +351,7 @@ public void testInsertRecords() throws Exception { records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z")); // Insert new records - returnedStatuses = HoodieClientTestUtils - .collectStatuses(table.handleInsert(commitTime, records.iterator())); + returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator())); assertEquals(3, returnedStatuses.size()); assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath()); @@ -389,8 +368,8 @@ public void testInsertRecords() throws Exception { @Test public void testFileSizeUpsertRecords() throws Exception { HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig( - HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024) - .parquetPageSize(64 * 1024).build()).build(); + HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024) + .build()).build(); String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); @@ -398,12 +377,11 @@ public void testFileSizeUpsertRecords() throws Exception { List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB for (int i = 0; i < 2000; i++) { - String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() - + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}"; + String recordStr = + "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + + "}"; TestRawTripPayload rowChange = new TestRawTripPayload(recordStr); - records - .add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), - rowChange)); + records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } // Insert new records @@ -412,39 +390,30 @@ public void testFileSizeUpsertRecords() throws Exception { // Check the updated file int counts = 0; for (File file : new File(basePath + "/2016/01/31").listFiles()) { - if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()) - .equals(commitTime)) { + if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(commitTime)) { System.out.println(file.getName() + "-" + file.length()); counts++; } } - assertEquals( - "If the number of records are more than 1150, then there should be a new file", 3, - counts); + assertEquals("If the number of records are more than 1150, then there should be a new file", 3, counts); } - private List testUpsertPartitioner(int smallFileSize, - int numInserts, - int numUpdates, - int fileSize, - boolean autoSplitInserts) throws Exception { - final String TEST_PARTITION_PATH = "2016/09/26"; - HoodieWriteConfig config = makeHoodieClientConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(smallFileSize).insertSplitSize(100) - .autoTuneInsertSplits(autoSplitInserts).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()) - .build(); + private List testUpsertPartitioner(int smallFileSize, int numInserts, + int numUpdates, int fileSize, boolean autoSplitInserts) throws Exception { + final String testPartitionPath = "2016/09/26"; + HoodieWriteConfig config = makeHoodieClientConfigBuilder().withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize).insertSplitSize(100) + .autoTuneInsertSplits(autoSplitInserts).build()).withStorageConfig( + HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build(); HoodieClientTestUtils.fakeCommitFile(basePath, "001"); - HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize); + HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator( - new String[]{TEST_PARTITION_PATH}); + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath}); List insertRecords = dataGenerator.generateInserts("001", numInserts); List updateRecords = dataGenerator.generateUpdates("001", numUpdates); for (HoodieRecord updateRec : updateRecords) { @@ -454,8 +423,8 @@ private List testUpsertPartitioner(int smal records.addAll(insertRecords); records.addAll(updateRecords); WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records)); - HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner) - table.getUpsertPartitioner(profile); + HoodieCopyOnWriteTable.UpsertPartitioner partitioner = + (HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile); assertEquals("Should have 3 partitions", 3, partitioner.numPartitions()); assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE, @@ -464,40 +433,35 @@ private List testUpsertPartitioner(int smal partitioner.getBucketInfo(1).bucketType); assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, partitioner.getBucketInfo(2).bucketType); - assertEquals("Update record should have gone to the 1 update partiton", 0, - partitioner.getPartition(new Tuple2<>(updateRecords.get(0).getKey(), - Option.apply(updateRecords.get(0).getCurrentLocation())))); - return partitioner.getInsertBuckets(TEST_PARTITION_PATH); + assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition( + new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation())))); + return partitioner.getInsertBuckets(testPartitionPath); } @Test public void testUpsertPartitioner() throws Exception { // Inserts + Updates... Check all updates go together & inserts subsplit - List insertBuckets = testUpsertPartitioner(0, 200, 100, - 1024, false); + List insertBuckets = testUpsertPartitioner(0, 200, 100, 1024, false); assertEquals("Total of 2 insert buckets", 2, insertBuckets.size()); } @Test public void testUpsertPartitionerWithSmallInsertHandling() throws Exception { - // Inserts + Updates .. Check updates go together & inserts subsplit, after expanding smallest file - List insertBuckets = testUpsertPartitioner(1000 * 1024, - 400, 100, 800 * 1024, false); + // Inserts + Updates .. Check updates go together & inserts subsplit, after expanding + // smallest file + List insertBuckets = testUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, + false); assertEquals("Total of 3 insert buckets", 3, insertBuckets.size()); - assertEquals("First insert bucket must be same as update bucket", 0, - insertBuckets.get(0).bucketNumber); - assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, - 0.01); + assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber); + assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, 0.01); // Now with insert split size auto tuned insertBuckets = testUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, true); assertEquals("Total of 3 insert buckets", 3, insertBuckets.size()); - assertEquals("First insert bucket must be same as update bucket", 0, - insertBuckets.get(0).bucketNumber); - assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, - insertBuckets.get(0).weight, 0.01); + assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber); + assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, insertBuckets.get(0).weight, 0.01); } @After diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 23f201b451e30..67f8d6d6e5c0c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -18,7 +18,6 @@ package com.uber.hoodie.table; - import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -76,15 +75,14 @@ public class TestMergeOnReadTable { - private transient JavaSparkContext jsc = null; - private transient SQLContext sqlContext; private static String basePath = null; - //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) //The implementation and gurantees of many API's differ, for example check rename(src,dst) private static MiniDFSCluster dfsCluster; private static DistributedFileSystem dfs; private static HdfsTestService hdfsTestService; + private transient JavaSparkContext jsc = null; + private transient SQLContext sqlContext; @AfterClass public static void cleanUp() throws Exception { @@ -92,13 +90,15 @@ public static void cleanUp() throws Exception { hdfsTestService.stop(); dfsCluster.shutdown(); } - // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM FileSystem.closeAll(); } @BeforeClass public static void setUpDFS() throws IOException { - // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM FileSystem.closeAll(); if (hdfsTestService == null) { hdfsTestService = new HdfsTestService(); @@ -111,8 +111,7 @@ public static void setUpDFS() throws IOException { @Before public void init() throws IOException { // Initialize a local spark env - jsc = new JavaSparkContext( - HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable")); + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable")); // Create a temp folder as the base path TemporaryFolder folder = new TemporaryFolder(); @@ -154,28 +153,23 @@ public void testSimpleInsertAndUpdate() throws Exception { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -209,21 +203,17 @@ public void testSimpleInsertAndUpdate() throws Exception { client.compact(compactionCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), - getConfig(false)); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false)); HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); - assertTrue(HoodieTimeline - .compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); + assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); assertEquals("Must contain 200 records", 200, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count()); @@ -232,8 +222,7 @@ public void testSimpleInsertAndUpdate() throws Exception { // Check if record level metadata is aggregated properly at the end of write. @Test public void testMetadataAggregateFromWriteStatus() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder(false) - .withWriteStatusClass(MetadataMergeWriteStatus.class).build(); + HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); String newCommitTime = "001"; @@ -248,7 +237,8 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { Map allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus .mergeMetadataForWriteStatuses(statuses); assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000")); - // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * records.size() + // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this + // should be 2 * records.size() assertEquals(String.valueOf(2 * records.size()), allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000")); } @@ -271,28 +261,23 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -329,15 +314,12 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { assertFalse(commit.isPresent()); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) - .collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles, basePath); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); //Wrote 40 records and deleted 20 records, so remaining 40-20 = 20 assertEquals("Must contain 20 records", 20, recordsRead.size()); } @@ -365,10 +347,8 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { //verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertTrue(commit.isPresent()); assertEquals("commit should be 001", "001", commit.get().getTimestamp()); @@ -391,10 +371,8 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { client.rollback(newCommitTime); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable - .getHoodieTable(metaClient, cfg); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); @@ -428,28 +406,23 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -473,10 +446,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) - .collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles, basePath); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -485,8 +456,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); @@ -512,11 +482,10 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); - final String compactedCommitTime = metaClient.getActiveTimeline().reload() - .getCommitsTimeline().lastInstant().get().getTimestamp(); + final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() + .getTimestamp(); assertTrue(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -531,8 +500,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); assertFalse(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -564,30 +532,28 @@ public void testUpsertPartitioner() throws Exception { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); - Map parquetFileIdToSize = dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + Map parquetFileIdToSize = dataFilesToRead.collect( + Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); /** - * Write 2 (only updates + inserts, written to .log file + correction of existing parquet file size) + * Write 2 (only updates + inserts, written to .log file + correction of existing parquet + * file size) */ newCommitTime = "002"; client.startCommitWithTime(newCommitTime); @@ -608,18 +574,17 @@ public void testUpsertPartitioner() throws Exception { assertFalse(commit.isPresent()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getActiveTimeline().reload() - .getCommitsTimeline().filterCompletedInstants(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, + hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); - Map parquetFileIdToNewSize = dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + Map parquetFileIdToNewSize = dataFilesToRead.collect( + Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); assertTrue(parquetFileIdToNewSize.entrySet().stream() .filter(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue()).count() > 0); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) - .collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles, basePath); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); //Wrote 20 records in 2 batches assertEquals("Must contain 40 records", 40, recordsRead.size()); } @@ -639,8 +604,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); newCommitTime = "101"; @@ -653,19 +617,17 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), - HoodieTestDataGenerator.avroSchema, updatedRecords); + .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, + updatedRecords); // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toList()); + List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toList()); for (FileSlice fileSlice : groupedLogFiles) { - assertEquals("There should be 1 log file written for every data file", 1, - fileSlice.getLogFiles().count()); + assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count()); } } @@ -674,31 +636,27 @@ public void testLogFileCountsAfterCompaction() throws Exception { table = HoodieTable.getHoodieTable(metaClient, config); String commitTime = writeClient.startCompaction(); - JavaRDD result = - writeClient.compact(commitTime); + JavaRDD result = writeClient.compact(commitTime); // Verify that recently written compacted data file has no log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - assertTrue("Compaction commit should be > than last insert", - HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime, - HoodieTimeline.GREATER)); + assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps( + timeline.lastInstant().get().getTimestamp(), newCommitTime, HoodieTimeline.GREATER)); for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = table.getRTFileSystemView() - .getLatestFileSlices(partitionPath) + List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) .collect(Collectors.toList()); for (FileSlice slice : groupedLogFiles) { - assertTrue( - "After compaction there should be no log files visiable on a Realtime view", + assertTrue("After compaction there should be no log files visiable on a Realtime view", slice.getLogFiles().collect(Collectors.toList()).isEmpty()); } List writeStatuses = result.collect(); assertTrue(writeStatuses.stream() - .filter(writeStatus -> writeStatus.getStat().getPartitionPath() - .contentEquals(partitionPath)).count() > 0); + .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)) + .count() > 0); } } @@ -707,16 +665,13 @@ private HoodieWriteConfig getConfig(Boolean autoCommit) { } private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withAutoCommit(autoCommit) - .withAssumeDatePartitioning(true) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false) + .withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) - .forTable("test-trip-table").withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } private void assertNoWriteErrors(List statuses) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java index f793c1539623f..65e7133350089 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java @@ -33,10 +33,10 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport { private String maxRecordKey; - public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = + public static final String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = "com.uber.hoodie.bloomfilter"; - public final static String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key"; - public final static String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key"; + public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key"; + public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key"; public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java index 827da3093531b..b0e06c0288e8c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java @@ -16,6 +16,7 @@ package com.uber.hoodie.avro; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -24,11 +25,11 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import com.fasterxml.jackson.databind.ObjectMapper; /** - * Marjority of this is copied from https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java - * Adjusted for expected behavior of our use cases + * Marjority of this is copied from + * https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/ + * common/JsonConverter.java Adjusted for expected behavior of our use cases */ public class MercifulJsonConverter { @@ -132,10 +133,10 @@ private Object typeConvert(Object value, String name, Schema schema) throws IOEx } private boolean isOptional(Schema schema) { - return schema.getType().equals(Schema.Type.UNION) && - schema.getTypes().size() == 2 && - (schema.getTypes().get(0).getType().equals(Schema.Type.NULL) || - schema.getTypes().get(1).getType().equals(Schema.Type.NULL)); + return schema.getType().equals(Schema.Type.UNION) + && schema.getTypes().size() == 2 + && (schema.getTypes().get(0).getType().equals(Schema.Type.NULL) + || schema.getTypes().get(1).getType().equals(Schema.Type.NULL)); } private Schema getNonNull(Schema schema) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java index d2de837efdedd..d38a2a660f7b1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java @@ -113,9 +113,8 @@ public Builder withPartitionPath(String partitionPath) { } public Builder withEarliestCommitRetained(Optional earliestCommitToRetain) { - this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ? - earliestCommitToRetain.get().getTimestamp() : - "-1"; + this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) + ? earliestCommitToRetain.get().getTimestamp() : "-1"; return this; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index 21c1816f5bccc..3f1b69080f786 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -210,12 +210,18 @@ public long fetchTotalWriteErrors() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } HoodieCommitMetadata that = (HoodieCommitMetadata) o; - if (!partitionToWriteStats.equals(that.partitionToWriteStats)) return false; + if (!partitionToWriteStats.equals(that.partitionToWriteStats)) { + return false; + } return compacted.equals(that.compacted); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java index d5884fb993709..f3111a1098264 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java @@ -39,7 +39,6 @@ public static Comparator getReverseCommitTimeComparator() { }; } - /** * Partition containing the file group. */ @@ -107,10 +106,10 @@ public String getPartitionPath() { */ private boolean isFileSliceCommitted(FileSlice slice) { String maxCommitTime = lastInstant.get().getTimestamp(); - return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) && - HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(), - maxCommitTime, - HoodieTimeline.LESSER_OR_EQUAL); + return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) + && HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(), + maxCommitTime, + HoodieTimeline.LESSER_OR_EQUAL); } @@ -128,7 +127,7 @@ public Stream getAllFileSlices() { /** * Gets the latest slice - this can contain either - * + *

* - just the log files without data file - (or) data file with 0 or more log files */ public Optional getLatestFileSlice() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java index f4545809cc465..0581a5a20de7a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java @@ -21,7 +21,7 @@ /** * HoodieKey consists of - * + *

* - recordKey : a recordKey that acts as primary key for a record - partitionPath : path to the * partition that contains the record */ @@ -54,8 +54,8 @@ public boolean equals(Object o) { return false; } HoodieKey otherKey = (HoodieKey) o; - return Objects.equal(recordKey, otherKey.recordKey) && - Objects.equal(partitionPath, otherKey.partitionPath); + return Objects.equal(recordKey, otherKey.recordKey) + && Objects.equal(partitionPath, otherKey.partitionPath); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java index 0c587c67199e0..7dfaf0bdf9f48 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java @@ -30,7 +30,7 @@ /** * Abstracts a single log file. Contains methods to extract metadata like the fileId, version and * extension from the log file path. - * + *

* Also contains logic to roll-over the log file */ public class HoodieLogFile implements Serializable { @@ -103,8 +103,12 @@ public static Comparator getLogVersionComparator() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } HoodieLogFile that = (HoodieLogFile) o; return path != null ? path.equals(that.path) : that.path == null; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java index ebe801811bc53..960042a84bd9a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java @@ -101,9 +101,8 @@ public void trySave(int taskPartitionId) { } } catch (IOException ioe) { log.warn( - "Error trying to save partition metadata (this is okay, as long as atleast 1 of these succced), " - + - partitionPath, ioe); + "Error trying to save partition metadata (this is okay, as long as " + + "atleast 1 of these succced), " + partitionPath, ioe); } finally { if (!metafileExists) { try { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java index 43b023020079a..16d75491a8d49 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -118,10 +118,10 @@ public boolean equals(Object o) { return false; } HoodieRecord that = (HoodieRecord) o; - return Objects.equal(key, that.key) && - Objects.equal(data, that.data) && - Objects.equal(currentLocation, that.currentLocation) && - Objects.equal(newLocation, that.newLocation); + return Objects.equal(key, that.key) + && Objects.equal(data, that.data) + && Objects.equal(currentLocation, that.currentLocation) + && Objects.equal(newLocation, that.newLocation); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java index fa5f7a04fa28c..51edf49513b24 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java @@ -42,8 +42,8 @@ public boolean equals(Object o) { return false; } HoodieRecordLocation otherLoc = (HoodieRecordLocation) o; - return Objects.equal(commitTime, otherLoc.commitTime) && - Objects.equal(fileId, otherLoc.fileId); + return Objects.equal(commitTime, otherLoc.commitTime) + && Objects.equal(fileId, otherLoc.fileId); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java index c7fe8fff8e513..36f53ba1198f4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java @@ -38,13 +38,13 @@ public interface HoodieRecordPayload extends Seri /** * This methods lets you write custom merging/combining logic to produce new values as a function * of current value on storage and whats contained in this object. - * + *

* eg: 1) You are updating counters, you may want to add counts to currentValue and write back * updated counts 2) You may be reading DB redo logs, and merge them with current image for a * database row on storage * * @param currentValue Current value in storage, to merge/combine this payload with - * @param schema Schema used for record + * @param schema Schema used for record * @return new combined/merged value to be written back to storage. EMPTY to skip writing this * record. */ diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java index 00564d62782a3..b9bb92544ec4d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java @@ -18,16 +18,16 @@ /** * Type of the Hoodie Table. - * + *

* Currently, 1 type is supported - * + *

* COPY_ON_WRITE - Performs upserts by versioning entire files, with later versions containing newer * value of a record. - * + *

* In the future, following might be added. - * + *

* MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up. - * + *

* SIMPLE_LSM - A simple 2 level LSM tree. */ public enum HoodieTableType { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index dccfd3105e079..4706e25a3f260 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -17,9 +17,8 @@ package com.uber.hoodie.common.model; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; - -import javax.annotation.Nullable; import java.io.Serializable; +import javax.annotation.Nullable; import org.apache.hadoop.fs.Path; /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java index dc9e04043c878..751f73c50d27c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java @@ -25,39 +25,39 @@ */ public class SizeAwareDataInputStream { - private final DataInputStream dis; - private final AtomicInteger numberOfBytesRead; - - public SizeAwareDataInputStream(DataInputStream dis) { - this.dis = dis; - this.numberOfBytesRead = new AtomicInteger(0); - } - - public int readInt() throws IOException { - numberOfBytesRead.addAndGet(Integer.BYTES); - return dis.readInt(); - } - - public void readFully(byte b[], int off, int len) throws IOException { - numberOfBytesRead.addAndGet(len); - dis.readFully(b, off, len); - } - - public void readFully(byte b[]) throws IOException { - numberOfBytesRead.addAndGet(b.length); - dis.readFully(b); - } - - public int skipBytes(int n) throws IOException { - numberOfBytesRead.addAndGet(n); - return dis.skipBytes(n); - } - - public void close() throws IOException { - dis.close(); - } - - public Integer getNumberOfBytesRead() { - return numberOfBytesRead.get(); - } + private final DataInputStream dis; + private final AtomicInteger numberOfBytesRead; + + public SizeAwareDataInputStream(DataInputStream dis) { + this.dis = dis; + this.numberOfBytesRead = new AtomicInteger(0); + } + + public int readInt() throws IOException { + numberOfBytesRead.addAndGet(Integer.BYTES); + return dis.readInt(); + } + + public void readFully(byte[] b, int off, int len) throws IOException { + numberOfBytesRead.addAndGet(len); + dis.readFully(b, off, len); + } + + public void readFully(byte[] b) throws IOException { + numberOfBytesRead.addAndGet(b.length); + dis.readFully(b); + } + + public int skipBytes(int n) throws IOException { + numberOfBytesRead.addAndGet(n); + return dis.skipBytes(n); + } + + public void close() throws IOException { + dis.close(); + } + + public Integer getNumberOfBytesRead() { + return numberOfBytesRead.get(); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 577d7cf4e346c..56921fb0ca2e5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -42,7 +42,7 @@ */ public class HoodieTableConfig implements Serializable { - private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class); + private static final transient Logger log = LogManager.getLogger(HoodieTableConfig.class); public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties"; public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name"; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 1388823af2f51..9462c301db349 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -48,7 +48,7 @@ */ public class HoodieTableMetaClient implements Serializable { - private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class); + private static final transient Logger log = LogManager.getLogger(HoodieTableMetaClient.class); public static String METAFOLDER_NAME = ".hoodie"; public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 5dcb2c83cd407..7cd4ee30772ab 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -16,6 +16,9 @@ package com.uber.hoodie.common.table.log; +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; + import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; @@ -28,17 +31,9 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; @@ -49,30 +44,28 @@ import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; - -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** - * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of - * records which will be used as a lookup table when merging the base columnar file with the redo - * log file. - * NOTE: If readBlockLazily is turned on, does not merge, instead keeps reading log blocks and merges everything at once - * This is an optimization to avoid seek() back and forth to read new block (forward seek()) - * and lazily read content of seen block (reverse and forward seek()) during merge - * | | Read Block 1 Metadata | | Read Block 1 Data | - * | | Read Block 2 Metadata | | Read Block 2 Data | - * | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | - * | | Read Block N Metadata | | Read Block N Data | - * - * This results in two I/O passes over the log file. - * + * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will + * be used as a lookup table when merging the base columnar file with the redo log file. NOTE: If readBlockLazily is + * turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to + * avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and + * forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2 + * Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | | + * | Read Block N Metadata | | Read Block N Data |

This results in two I/O passes over the log file. */ public class HoodieCompactedLogRecordScanner implements Iterable> { - private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); + private static final Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); // Final map of compacted/merged records private final ExternalSpillableMap> records; @@ -116,10 +109,10 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List Running - // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1 + // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct + // DataBlock (B1) with commitTime C1 // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2) // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2) // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same. // Say, commit C1 eventually failed and a rollback is triggered. - // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file, - // but in reality we need to rollback (B1 & B2) - // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2 + // Rollback will write only 1 rollback block (R1) since it assumes one block is + // written per ingestion batch for a file but in reality we need to rollback (B1 & B2) + // The following code ensures the same rollback block (R1) is used to rollback + // both B1 & B2 log.info("Reading a command block from file " + logFile.getPath()); // This is a command block - take appropriate action based on the command HoodieCommandBlock commandBlock = (HoodieCommandBlock) r; @@ -163,10 +158,11 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List> merge( HoodieAvroDataBlock dataBlock) throws IOException { @@ -291,6 +287,9 @@ private void merge(Map> reco case CORRUPT_BLOCK: log.warn("Found a corrupt block which was not rolled back"); break; + default: + //TODO : Need to understand if COMMAND_BLOCK has to be handled? + break; } } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java index 1d4cda06121f5..1ccf43cfab041 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -28,18 +28,17 @@ import com.uber.hoodie.exception.CorruptedLogFileException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.EOFException; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** * Scans a log file and provides block level iterator on the log file Loads the entire block @@ -49,28 +48,29 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { private static final int DEFAULT_BUFFER_SIZE = 4096; - private final static Logger log = LogManager.getLogger(HoodieLogFileReader.class); + private static final Logger log = LogManager.getLogger(HoodieLogFileReader.class); private final FSDataInputStream inputStream; private final HoodieLogFile logFile; private static final byte[] oldMagicBuffer = new byte[4]; private static final byte[] magicBuffer = new byte[6]; private final Schema readerSchema; - private LogFormatVersion nextBlockVersion; + private HoodieLogFormat.LogFormatVersion nextBlockVersion; private boolean readBlockLazily; private long reverseLogFilePosition; private long lastReverseLogFilePosition; private boolean reverseReader; HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, - boolean readBlockLazily, boolean reverseReader) throws IOException { + boolean readBlockLazily, boolean reverseReader) throws IOException { this.inputStream = fs.open(logFile.getPath(), bufferSize); this.logFile = logFile; this.readerSchema = readerSchema; this.readBlockLazily = readBlockLazily; this.reverseReader = reverseReader; - if(this.reverseReader) { - this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen(); + if (this.reverseReader) { + this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs + .getFileStatus(logFile.getPath()).getLen(); } addShutDownHook(); } @@ -80,7 +80,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader); } - HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { + HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) + throws IOException { this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false); } @@ -105,7 +106,8 @@ public void run() { }); } - // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows for max of Integer size + // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows + // for max of Integer size private HoodieLogBlock readBlock() throws IOException { int blocksize = -1; @@ -137,8 +139,8 @@ private HoodieLogBlock readBlock() throws IOException { } // We may have had a crash which could have written this block partially - // Skip blocksize in the stream and we should either find a sync marker (start of the next block) or EOF - // If we did not find either of it, then this block is a corrupted block. + // Skip blocksize in the stream and we should either find a sync marker (start of the next + // block) or EOF. If we did not find either of it, then this block is a corrupted block. boolean isCorrupted = isBlockCorrupt(blocksize); if (isCorrupted) { return createCorruptBlock(); @@ -168,7 +170,8 @@ private HoodieLogBlock readBlock() throws IOException { } // 6. Read the content or skip content based on IO vs Memory trade-off by client - // TODO - have a max block size and reuse this buffer in the ByteBuffer (hard to guess max block size for now) + // TODO - have a max block size and reuse this buffer in the ByteBuffer + // (hard to guess max block size for now) long contentPosition = inputStream.getPos(); byte[] content = HoodieLogBlock.readOrSkipContent(inputStream, contentLength, readBlockLazily); @@ -178,7 +181,8 @@ private HoodieLogBlock readBlock() throws IOException { footer = HoodieLogBlock.getLogMetadata(inputStream); } - // 8. Read log block length, if present. This acts as a reverse pointer when traversing a log file in reverse + // 8. Read log block length, if present. This acts as a reverse pointer when traversing a + // log file in reverse long logBlockLength = 0; if (nextBlockVersion.hasLogBlockLength()) { logBlockLength = inputStream.readLong(); @@ -193,15 +197,18 @@ private HoodieLogBlock readBlock() throws IOException { if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) { return HoodieAvroDataBlock.getBlock(content, readerSchema); } else { - return HoodieAvroDataBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, readerSchema, header, footer); + return HoodieAvroDataBlock + .getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, readerSchema, header, footer); } case DELETE_BLOCK: - return HoodieDeleteBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, header, footer); + return HoodieDeleteBlock + .getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, header, footer); case COMMAND_BLOCK: - return HoodieCommandBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, header, footer); + return HoodieCommandBlock + .getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, header, footer); default: throw new HoodieNotSupportedException("Unsupported Block " + blockType); } @@ -216,9 +223,12 @@ private HoodieLogBlock createCorruptBlock() throws IOException { log.info("Next available block in " + logFile + " starts at " + nextBlockOffset); int corruptedBlockSize = (int) (nextBlockOffset - currentPos); long contentPosition = inputStream.getPos(); - byte[] corruptedBytes = HoodieLogBlock.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); - return HoodieCorruptBlock.getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily, - contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), new HashMap<>()); + byte[] corruptedBytes = HoodieLogBlock + .readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); + return HoodieCorruptBlock + .getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily, + contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), + new HashMap<>()); } private boolean isBlockCorrupt(int blocksize) throws IOException { @@ -278,12 +288,10 @@ public boolean hasNext() { } /** - * Read log format version from log file, if present - * For old log files written with Magic header OLD_MAGIC and without version, return DEFAULT_VERSION - * - * @throws IOException + * Read log format version from log file, if present For old log files written with Magic header + * OLD_MAGIC and without version, return DEFAULT_VERSION */ - private LogFormatVersion readVersion() throws IOException { + private HoodieLogFormat.LogFormatVersion readVersion() throws IOException { // If not old log file format (written with Magic header OLD_MAGIC), then read log version if (Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { Arrays.fill(oldMagicBuffer, (byte) 0); @@ -309,7 +317,8 @@ private boolean readMagic() throws IOException { inputStream.readFully(oldMagicBuffer, 0, 4); if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { throw new CorruptedLogFileException( - logFile + "could not be read. Did not find the magic bytes at the start of the block"); + logFile + + "could not be read. Did not find the magic bytes at the start of the block"); } } return false; @@ -324,19 +333,17 @@ public HoodieLogBlock next() { try { // hasNext() must be called before next() return readBlock(); - } catch(IOException io) { + } catch (IOException io) { throw new HoodieIOException("IOException when reading logblock from log file " + logFile, io); } } /** * hasPrev is not idempotent - * - * @return */ public boolean hasPrev() { try { - if(!this.reverseReader) { + if (!this.reverseReader) { throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); } reverseLogFilePosition = lastReverseLogFilePosition; @@ -351,17 +358,13 @@ public boolean hasPrev() { } /** - * This is a reverse iterator - * Note: At any point, an instance of HoodieLogFileReader should either iterate reverse (prev) - * or forward (next). Doing both in the same instance is not supported + * This is a reverse iterator Note: At any point, an instance of HoodieLogFileReader should either + * iterate reverse (prev) or forward (next). Doing both in the same instance is not supported * WARNING : Every call to prev() should be preceded with hasPrev() - * - * @return - * @throws IOException */ public HoodieLogBlock prev() throws IOException { - if(!this.reverseReader) { + if (!this.reverseReader) { throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); } long blockSize = inputStream.readLong(); @@ -372,8 +375,9 @@ public HoodieLogBlock prev() throws IOException { } catch (Exception e) { // this could be a corrupt block inputStream.seek(blockEndPos); - throw new CorruptedLogFileException("Found possible corrupted block, cannot read log file in reverse, " + - "fallback to forward reading of logfile"); + throw new CorruptedLogFileException( + "Found possible corrupted block, cannot read log file in reverse, " + + "fallback to forward reading of logfile"); } boolean hasNext = hasNext(); reverseLogFilePosition -= blockSize; @@ -382,16 +386,14 @@ public HoodieLogBlock prev() throws IOException { } /** - * Reverse pointer, does not read the block. Return the current position of the log file (in reverse) - * If the pointer (inputstream) is moved in any way, it is the job of the client of this class to - * seek/reset it back to the file position returned from the method to expect correct results - * - * @return - * @throws IOException + * Reverse pointer, does not read the block. Return the current position of the log file (in + * reverse) If the pointer (inputstream) is moved in any way, it is the job of the client of this + * class to seek/reset it back to the file position returned from the method to expect correct + * results */ public long moveToPrev() throws IOException { - if(!this.reverseReader) { + if (!this.reverseReader) { throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); } inputStream.seek(lastReverseLogFilePosition); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java index d5deb9d03f51f..cb12f9ef8aff1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java @@ -19,22 +19,21 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; - /** - * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a OLD_MAGIC - * sync marker. A Block can either be a Data block, Command block or Delete Block. Data Block - - * Contains log records serialized as Avro Binary Format Command Block - Specific commands like - * RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of keys - * to delete - tombstone for keys + * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a + * OLD_MAGIC sync marker. A Block can either be a Data block, Command block or Delete Block. Data + * Block - Contains log records serialized as Avro Binary Format Command Block - Specific commands + * like RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of + * keys to delete - tombstone for keys */ public interface HoodieLogFormat { @@ -43,19 +42,18 @@ public interface HoodieLogFormat { * this file specific (generate a random 4 byte magic and stick it in the file header), but this I * think is suffice for now - PR */ - byte[] OLD_MAGIC = new byte[]{'H', 'U', 'D', 'I'}; + byte[] OLD_MAGIC = new byte[] {'H', 'U', 'D', 'I'}; /** - * Magic 6 bytes we put at the start of every block in the log file. - * This is added to maintain backwards compatiblity due to lack of log format/block - * version in older log files. All new log block will now write this OLD_MAGIC value + * Magic 6 bytes we put at the start of every block in the log file. This is added to maintain + * backwards compatiblity due to lack of log format/block version in older log files. All new log + * block will now write this OLD_MAGIC value */ - byte[] MAGIC = new byte[]{'#', 'H', 'U', 'D', 'I', '#'}; + byte[] MAGIC = new byte[] {'#', 'H', 'U', 'D', 'I', '#'}; /** - * The current version of the log format. Anytime the log format changes - * this version needs to be bumped and corresponding changes need to be made to - * {@link HoodieLogFormatVersion} + * The current version of the log format. Anytime the log format changes this version needs to be + * bumped and corresponding changes need to be made to {@link HoodieLogFormatVersion} */ int currentVersion = 1; @@ -94,7 +92,7 @@ interface Reader extends Closeable, Iterator { */ class WriterBuilder { - private final static Logger log = LogManager.getLogger(WriterBuilder.class); + private static final Logger log = LogManager.getLogger(WriterBuilder.class); // Default max log file size 512 MB public static final long DEFAULT_SIZE_THRESHOLD = 512 * 1024 * 1024L; @@ -112,7 +110,8 @@ class WriterBuilder { private String logFileId; // File Commit Time stamp private String commitTime; - // version number for this log file. If not specified, then the current version will be computed by inspecting the file system + // version number for this log file. If not specified, then the current version will be + // computed by inspecting the file system private Integer logVersion; // Location of the directory containing the log private Path parentPath; @@ -215,4 +214,35 @@ static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Sc throws IOException { return new HoodieLogFileReader(fs, logFile, readerSchema, false, false); } + + /** + * A set of feature flags associated with a log format. Versions are changed when the log format + * changes. TODO(na) - Implement policies around major/minor versions + */ + abstract class LogFormatVersion { + + private final int version; + + LogFormatVersion(int version) { + this.version = version; + } + + public int getVersion() { + return version; + } + + public abstract boolean hasMagicHeader(); + + public abstract boolean hasContent(); + + public abstract boolean hasContentLength(); + + public abstract boolean hasOrdinal(); + + public abstract boolean hasHeader(); + + public abstract boolean hasFooter(); + + public abstract boolean hasLogBlockLength(); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index e5f8f5c99be65..00a3a74985e76 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -19,11 +19,10 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; - import java.io.IOException; import java.util.List; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -36,16 +35,16 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final boolean readBlocksLazily; private final boolean reverseLogReader; - private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class); + private static final Logger log = LogManager.getLogger(HoodieLogFormatReader.class); HoodieLogFormatReader(FileSystem fs, List logFiles, - Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { + Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { this.logFiles = logFiles; this.fs = fs; this.readerSchema = readerSchema; this.readBlocksLazily = readBlocksLazily; this.reverseLogReader = reverseLogReader; - if(logFiles.size() > 0) { + if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, false); @@ -53,7 +52,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { } HoodieLogFormatReader(FileSystem fs, List logFiles, - Schema readerSchema) throws IOException { + Schema readerSchema) throws IOException { this(fs, logFiles, readerSchema, false, false); } @@ -67,16 +66,15 @@ public void close() throws IOException { @Override public boolean hasNext() { - if(currentReader == null) { + if (currentReader == null) { return false; - } - else if (currentReader.hasNext()) { + } else if (currentReader.hasNext()) { return true; - } - else if (logFiles.size() > 0) { + } else if (logFiles.size() > 0) { try { HoodieLogFile nextLogFile = logFiles.remove(0); - this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, + readBlocksLazily, false); } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatVersion.java similarity index 68% rename from hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatVersion.java index 8bba078c74322..dc0c75ca538ea 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatVersion.java @@ -17,46 +17,17 @@ package com.uber.hoodie.common.table.log; /** - * A set of feature flags associated with a log format. - * Versions are changed when the log format changes. - * TODO(na) - Implement policies around major/minor versions + * Implements logic to determine behavior for feature flags for + * {@link HoodieLogFormat.LogFormatVersion}. */ -abstract class LogFormatVersion { - private final int version; +final class HoodieLogFormatVersion extends HoodieLogFormat.LogFormatVersion { - LogFormatVersion(int version) { - this.version = version; - } - - public int getVersion() { - return version; - } - - public abstract boolean hasMagicHeader(); - - public abstract boolean hasContent(); - - public abstract boolean hasContentLength(); - - public abstract boolean hasOrdinal(); - - public abstract boolean hasHeader(); - - public abstract boolean hasFooter(); - - public abstract boolean hasLogBlockLength(); -} - -/** - * Implements logic to determine behavior for feature flags for {@link LogFormatVersion} - */ -final class HoodieLogFormatVersion extends LogFormatVersion { - - public final static int DEFAULT_VERSION = 0; + public static final int DEFAULT_VERSION = 0; HoodieLogFormatVersion(int version) { super(version); } + @Override public boolean hasMagicHeader() { switch (super.getVersion()) { @@ -114,8 +85,9 @@ public boolean hasFooter() { return false; case 1: return true; + default: + return false; } - return false; } @Override @@ -125,7 +97,8 @@ public boolean hasLogBlockLength() { return false; case 1: return true; + default: + return false; } - return false; } } \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index 9ea4600a5326c..ee1dcf559fdd6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -22,6 +22,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -31,15 +32,13 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.IOException; - /** * HoodieLogFormatWriter can be used to append blocks to a log file Use * HoodieLogFormat.WriterBuilder to construct */ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { - private final static Logger log = LogManager.getLogger(HoodieLogFormatWriter.class); + private static final Logger log = LogManager.getLogger(HoodieLogFormatWriter.class); private HoodieLogFile logFile; private final FileSystem fs; @@ -49,7 +48,6 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { private FSDataOutputStream output; /** - * * @param fs * @param logFile * @param bufferSize @@ -71,7 +69,8 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { try { this.output = fs.append(path, bufferSize); } catch (RemoteException e) { - // this happens when either another task executor writing to this file died or data node is going down + // this happens when either another task executor writing to this file died or + // data node is going down if (e.getClassName().equals(AlreadyBeingCreatedException.class.getName()) && fs instanceof DistributedFileSystem) { log.warn("Trying to recover log on path " + path); @@ -120,21 +119,23 @@ public Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException { // Find current version - LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.currentVersion); + HoodieLogFormat.LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion( + HoodieLogFormat.currentVersion); long currentSize = this.output.size(); // 1. Write the magic header for the start of the block this.output.write(HoodieLogFormat.MAGIC); // bytes for header - byte [] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); + byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); // content bytes - byte [] content = block.getContentBytes(); + byte[] content = block.getContentBytes(); // bytes for footer - byte [] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); + byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); // 2. Write the total size of the block (excluding Magic) - this.output.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); + this.output + .writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); // 3. Write the version of this log block this.output.writeInt(currentLogFormatVersion.getVersion()); @@ -149,7 +150,8 @@ public Writer appendBlock(HoodieLogBlock block) this.output.write(content); // 8. Write the footers for the log block this.output.write(footerBytes); - // 9. Write the total size of the log block (including magic) which is everything written until now (for reverse pointer) + // 9. Write the total size of the log block (including magic) which is everything written + // until now (for reverse pointer) this.output.writeLong(this.output.size() - currentSize); // Flush every block to disk flush(); @@ -159,35 +161,27 @@ public Writer appendBlock(HoodieLogBlock block) } /** - * - * This method returns the total LogBlock Length which is the sum of - * 1. Number of bytes to write version - * 2. Number of bytes to write ordinal - * 3. Length of the headers - * 4. Number of bytes used to write content length - * 5. Length of the content - * 6. Length of the footers - * 7. Number of bytes to write totalLogBlockLength - * @param contentLength - * @param headerLength - * @param footerLength - * @return + * This method returns the total LogBlock Length which is the sum of 1. Number of bytes to write + * version 2. Number of bytes to write ordinal 3. Length of the headers 4. Number of bytes used to + * write content length 5. Length of the content 6. Length of the footers 7. Number of bytes to + * write totalLogBlockLength */ private int getLogBlockLength(int contentLength, int headerLength, int footerLength) { return Integer.BYTES + // Number of bytes to write version - Integer.BYTES + // Number of bytes to write ordinal - headerLength + // Length of the headers - Long.BYTES + // Number of bytes used to write content length - contentLength + // Length of the content - footerLength + // Length of the footers - Long.BYTES; // Number of bytes to write totalLogBlockLength at end of block (for reverse pointer) + Integer.BYTES + // Number of bytes to write ordinal + headerLength + // Length of the headers + Long.BYTES + // Number of bytes used to write content length + contentLength + // Length of the content + footerLength + // Length of the footers + Long.BYTES; // bytes to write totalLogBlockLength at end of block (for reverse ptr) } private Writer rolloverIfNeeded() throws IOException, InterruptedException { // Roll over if the size is past the threshold if (getCurrentSize() > sizeThreshold) { - //TODO - make an end marker which seals the old log file (no more appends possible to that file). + //TODO - make an end marker which seals the old log file (no more appends possible to that + // file). log.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold + ". Rolling over to the next version"); HoodieLogFile newLogFile = logFile.rollOver(fs); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java index eb2b5f45ffd25..b8822521f819a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java @@ -44,12 +44,9 @@ import org.apache.hadoop.fs.FSDataInputStream; /** - * DataBlock contains a list of records serialized using Avro. - * The Datablock contains - * 1. Data Block version - * 2. Total number of records in the block - * 3. Size of a record - * 4. Actual avro serialized content of the record + * DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Data Block + * version 2. Total number of records in the block 3. Size of a record 4. Actual avro serialized + * content of the record */ public class HoodieAvroDataBlock extends HoodieLogBlock { @@ -216,7 +213,7 @@ private void createRecordsFromContentBytes() throws IOException { deflate(); } - /*****************************************************DEPRECATED METHODS**********************************************/ + /*********************************DEPRECATED METHODS***********************************/ @Deprecated @VisibleForTesting @@ -232,7 +229,8 @@ public HoodieAvroDataBlock(List records, Schema schema) { @Deprecated /** - * This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using HoodieLogFormat V1 + * This method is retained to provide backwards compatibility to HoodieArchivedLogs which + * were written using HoodieLogFormat V1 */ public static HoodieLogBlock getBlock(byte[] content, Schema readerSchema) throws IOException { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java new file mode 100644 index 0000000000000..796d364e529be --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java @@ -0,0 +1,37 @@ +/* + * 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.table.log.block; + +/** + * A set of feature flags associated with a data log block format. Versions are changed when the log + * block format changes. TODO(na) - Implement policies around major/minor versions + */ +final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion { + + HoodieAvroDataBlockVersion(int version) { + super(version); + } + + public boolean hasRecordCount() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java index b9b11cc8f59c5..ba520f9cedf0a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java @@ -17,11 +17,10 @@ package com.uber.hoodie.common.table.log.block; import com.uber.hoodie.common.model.HoodieLogFile; -import org.apache.hadoop.fs.FSDataInputStream; - import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.hadoop.fs.FSDataInputStream; /** * Command block issues a specific command to the scanner @@ -30,7 +29,9 @@ public class HoodieCommandBlock extends HoodieLogBlock { private final HoodieCommandBlockTypeEnum type; - public enum HoodieCommandBlockTypeEnum {ROLLBACK_PREVIOUS_BLOCK} + public enum HoodieCommandBlockTypeEnum { + ROLLBACK_PREVIOUS_BLOCK + } public HoodieCommandBlock(Map header) { this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java new file mode 100644 index 0000000000000..96a56f572add0 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java @@ -0,0 +1,28 @@ +/* + * 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.table.log.block; + +/** + * A set of feature flags associated with a command log block format. Versions are changed when the + * log block format changes. TODO(na) - Implement policies around major/minor versions + */ +final class HoodieCommandBlockVersion extends HoodieLogBlockVersion { + + HoodieCommandBlockVersion(int version) { + super(version); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java index c75c8ea622325..fc0402f08d5f8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java @@ -17,11 +17,10 @@ package com.uber.hoodie.common.table.log.block; import com.uber.hoodie.common.model.HoodieLogFile; -import org.apache.hadoop.fs.FSDataInputStream; - import java.io.IOException; import java.util.Map; import java.util.Optional; +import org.apache.hadoop.fs.FSDataInputStream; /** * Corrupt block is emitted whenever the scanner finds the length of the block written at the @@ -61,6 +60,7 @@ public static HoodieLogBlock getBlock(HoodieLogFile logFile, Map footer) throws IOException { return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily, - Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), + header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index 4de25b5daef7f..2f4ed7421f0c0 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -19,9 +19,6 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FSDataInputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -31,6 +28,8 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FSDataInputStream; /** * Delete block contains a list of keys to be deleted from scanning the blocks so far @@ -65,7 +64,8 @@ public byte[] getContentBytes() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); - byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',').getBytes(Charset.forName("utf-8")); + byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',') + .getBytes(Charset.forName("utf-8")); output.writeInt(HoodieLogBlock.version); output.writeInt(bytesToWrite.length); output.write(bytesToWrite); @@ -80,7 +80,8 @@ public String[] getKeysToDelete() { inflate(); } SizeAwareDataInputStream dis = - new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get()))); + new SizeAwareDataInputStream( + new DataInputStream(new ByteArrayInputStream(getContent().get()))); int version = dis.readInt(); int dataLength = dis.readInt(); byte[] data = new byte[dataLength]; @@ -110,6 +111,7 @@ public static HoodieLogBlock getBlock(HoodieLogFile logFile, Map footer) throws IOException { return new HoodieDeleteBlock(content, inputStream, readBlockLazily, - Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), + header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java new file mode 100644 index 0000000000000..990af81ca271b --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java @@ -0,0 +1,28 @@ +/* + * 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.table.log.block; + +/** + * A set of feature flags associated with a delete log block format. Versions are changed when the + * log block format changes. TODO(na) - Implement policies around major/minor versions + */ +final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion { + + HoodieDeleteBlockVersion(int version) { + super(version); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index e7735b0dbda46..86b3f26984840 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -36,10 +36,10 @@ public abstract class HoodieLogBlock { /** - * The current version of the log block. Anytime the logBlock format changes - * this version needs to be bumped and corresponding changes need to be made to - * {@link HoodieLogBlockVersion} - * TODO : Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override operations there + * The current version of the log block. Anytime the logBlock format changes this version needs to + * be bumped and corresponding changes need to be made to {@link HoodieLogBlockVersion} TODO : + * Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override + * operations there */ public static int version = 1; // Header for each log block @@ -49,7 +49,7 @@ public abstract class HoodieLogBlock { // Location of a log block on disk private final Optional blockContentLocation; // data for a specific block - private Optional content; + private Optional content; // TODO : change this to just InputStream so this works for any FileSystem // create handlers to return specific type of inputstream based on FS // input stream corresponding to the log file where this logBlock belongs @@ -57,10 +57,10 @@ public abstract class HoodieLogBlock { // Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive) protected boolean readBlockLazily; - public HoodieLogBlock(@Nonnull Map logBlockHeader, + public HoodieLogBlock(@Nonnull Map logBlockHeader, @Nonnull Map logBlockFooter, @Nonnull Optional blockContentLocation, - @Nonnull Optional content, + @Nonnull Optional content, FSDataInputStream inputStream, boolean readBlockLazily) { this.logBlockHeader = logBlockHeader; @@ -76,7 +76,7 @@ public byte[] getContentBytes() throws IOException { throw new HoodieException("No implementation was provided"); } - public byte [] getMagic() { + public byte[] getMagic() { throw new HoodieException("No implementation was provided"); } @@ -116,8 +116,8 @@ public enum HoodieLogBlockType { } /** - * Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. - * Only add new enums at the end. + * Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the + * ordinal. Only add new enums at the end. */ public enum HeaderMetadataType { INSTANT_TIME, @@ -127,17 +127,19 @@ public enum HeaderMetadataType { } /** - * Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. - * Only add new enums at the end. + * Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the + * ordinal. Only add new enums at the end. */ public enum FooterMetadataType { } /** - * This class is used to store the Location of the Content of a Log Block. It's used when a client chooses for a - * IO intensive CompactedScanner, the location helps to lazily read contents from the log file + * This class is used to store the Location of the Content of a Log Block. It's used when a client + * chooses for a IO intensive CompactedScanner, the location helps to lazily read contents from + * the log file */ public static final class HoodieLogBlockContentLocation { + // The logFile that contains this block private final HoodieLogFile logFile; // The filePosition in the logFile for the contents of this block @@ -147,7 +149,8 @@ public static final class HoodieLogBlockContentLocation { // The final position where the complete block ends private final long blockEndPos; - HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, long blockSize, long blockEndPos) { + HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, + long blockSize, long blockEndPos) { this.logFile = logFile; this.contentPositionInLogFile = contentPositionInLogFile; this.blockSize = blockSize; @@ -190,7 +193,8 @@ public static byte[] getLogMetadataBytes(Map metadat } /** - * Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes} + * Convert bytes to LogMetadata, follow the same order as + * {@link HoodieLogBlock#getLogMetadataBytes} */ public static Map getLogMetadata(DataInputStream dis) throws IOException { @@ -216,16 +220,10 @@ public static Map getLogMetadata(DataInputStream dis /** * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in * {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner} - * - * @param inputStream - * @param contentLength - * @param readBlockLazily - * @return - * @throws IOException */ - public static byte [] readOrSkipContent(FSDataInputStream inputStream, + public static byte[] readOrSkipContent(FSDataInputStream inputStream, Integer contentLength, boolean readBlockLazily) throws IOException { - byte [] content = null; + byte[] content = null; if (!readBlockLazily) { // Read the contents in memory content = new byte[contentLength]; @@ -239,7 +237,6 @@ public static Map getLogMetadata(DataInputStream dis /** * When lazyReading of blocks is turned on, inflate the content of a log block from disk - * @throws IOException */ protected void inflate() throws IOException { @@ -248,21 +245,21 @@ protected void inflate() throws IOException { inputStream.seek(this.getBlockContentLocation().get().getContentPositionInLogFile()); inputStream.readFully(content.get(), 0, content.get().length); inputStream.seek(this.getBlockContentLocation().get().getBlockEndPos()); - } catch(IOException e) { + } catch (IOException e) { try { // TODO : fs.open() and return inputstream again, need to pass FS configuration // because the inputstream might close/timeout for large number of log blocks to be merged inflate(); - } catch(IOException io) { + } catch (IOException io) { throw new HoodieIOException("unable to lazily read log block from disk", io); } } } /** - * After the content bytes is converted into the required DataStructure by a logBlock, deflate the content - * to release byte [] and relieve memory pressure when GC kicks in. - * NOTE: This still leaves the heap fragmented + * After the content bytes is converted into the required DataStructure by a logBlock, deflate the + * content to release byte [] and relieve memory pressure when GC kicks in. NOTE: This still + * leaves the heap fragmented */ protected void deflate() { content = Optional.empty(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java new file mode 100644 index 0000000000000..9316099cad554 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java @@ -0,0 +1,33 @@ +/* + * 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.table.log.block; + +abstract class HoodieLogBlockVersion { + + private final int currentVersion; + + public static final int DEFAULT_VERSION = 0; + + HoodieLogBlockVersion(int version) { + this.currentVersion = version; + } + + int getVersion() { + return currentVersion; + } +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java deleted file mode 100644 index 1a984420935dc..0000000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.table.log.block; - -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.version; - -abstract class HoodieLogBlockVersion { - - private final int currentVersion; - - public final static int DEFAULT_VERSION = 0; - - HoodieLogBlockVersion(int version) { - this.currentVersion = version; - } - - int getVersion() { - return currentVersion; - } -} - -/** - * A set of feature flags associated with a data log block format. - * Versions are changed when the log block format changes. - * TODO(na) - Implement policies around major/minor versions - */ -final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion { - - HoodieAvroDataBlockVersion(int version) { - super(version); - } - - public boolean hasRecordCount() { - switch (super.getVersion()) { - case DEFAULT_VERSION: - return true; - default: - return true; - } - } -} - -/** - * A set of feature flags associated with a command log block format. - * Versions are changed when the log block format changes. - * TODO(na) - Implement policies around major/minor versions - */ -final class HoodieCommandBlockVersion extends HoodieLogBlockVersion { - - HoodieCommandBlockVersion(int version) { - super(version); - } -} - -/** - * A set of feature flags associated with a delete log block format. - * Versions are changed when the log block format changes. - * TODO(na) - Implement policies around major/minor versions - */ -final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion { - - HoodieDeleteBlockVersion(int version) { - super(version); - } -} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index 8c34717b23b0f..a8d35411e1558 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -51,9 +51,10 @@ */ public class HoodieActiveTimeline extends HoodieDefaultTimeline { - public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat.getInstance("yyyyMMddHHmmss"); + public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat + .getInstance("yyyyMMddHHmmss"); - private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); + private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class); private HoodieTableMetaClient metaClient; /** @@ -71,12 +72,12 @@ protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includ Arrays.stream( HoodieTableMetaClient .scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> { - // Include only the meta files with extensions that needs to be included - String extension = FSUtils.getFileExtension(path.getName()); - return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); - })).sorted(Comparator.comparing( - // Sort the meta-data by the instant time (first part of the file name) - fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName()))) + // Include only the meta files with extensions that needs to be included + String extension = FSUtils.getFileExtension(path.getName()); + return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); + })).sorted(Comparator.comparing( + // Sort the meta-data by the instant time (first part of the file name) + fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName()))) // create HoodieInstantMarkers from FileStatus, which extracts properties .map(HoodieInstant::new).collect(Collectors.toList()); log.info("Loaded instants " + instants); @@ -84,13 +85,15 @@ protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includ throw new HoodieIOException("Failed to scan metadata", e); } this.metaClient = metaClient; - // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = + (Function> & Serializable) this::getInstantDetails; } public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { this(metaClient, - new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, + new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); } @@ -114,8 +117,7 @@ private void readObject(java.io.ObjectInputStream in) } /** - * Get all instants (commits, delta commits) that produce new data, in the active - * timeline * + * Get all instants (commits, delta commits) that produce new data, in the active timeline * */ public HoodieTimeline getCommitsTimeline() { return getTimelineOfActions( @@ -123,8 +125,8 @@ public HoodieTimeline getCommitsTimeline() { } /** - * Get all instants (commits, delta commits, clean, savepoint, rollback) that result - * in actions, in the active timeline * + * Get all instants (commits, delta commits, clean, savepoint, rollback) that result in actions, + * in the active timeline * */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions( diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java index 793d9d9961778..1da29f66925fd 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java @@ -46,7 +46,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { private HoodieTableMetaClient metaClient; private Map readCommits = new HashMap<>(); - private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); + private static final transient Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { // Read back the commits to make sure @@ -68,8 +68,10 @@ public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { throw new HoodieIOException( "Could not load archived commit timeline from path " + archiveLogPath, e); } - // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = + (Function> & Serializable) this::getInstantDetails; this.metaClient = metaClient; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java index 3a0240239a4df..c855203a3bad5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java @@ -34,7 +34,7 @@ */ public class HoodieDefaultTimeline implements HoodieTimeline { - private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class); + private static final transient Logger log = LogManager.getLogger(HoodieDefaultTimeline.class); protected Function> details; protected List instants; @@ -65,9 +65,9 @@ public HoodieTimeline filterCompletedInstants() { @Override public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) { return new HoodieDefaultTimeline(instants.stream().filter( - s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) && - HoodieTimeline.compareTimestamps( - s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details); + s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) + && HoodieTimeline.compareTimestamps( + s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details); } @Override @@ -133,8 +133,8 @@ public Stream getInstants() { @Override public boolean isBeforeTimelineStarts(String instant) { Optional firstCommit = firstInstant(); - return firstCommit.isPresent() && - HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER); + return firstCommit.isPresent() + && HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java index 1891c9807b2d1..59be65555d75d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java @@ -79,25 +79,20 @@ public String getTimestamp() { */ public String getFileName() { if (HoodieTimeline.COMMIT_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightCommitFileName(timestamp) : - HoodieTimeline.makeCommitFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightCommitFileName(timestamp) + : HoodieTimeline.makeCommitFileName(timestamp); } else if (HoodieTimeline.CLEAN_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightCleanerFileName(timestamp) : - HoodieTimeline.makeCleanerFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightCleanerFileName(timestamp) + : HoodieTimeline.makeCleanerFileName(timestamp); } else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightRollbackFileName(timestamp) : - HoodieTimeline.makeRollbackFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightRollbackFileName(timestamp) + : HoodieTimeline.makeRollbackFileName(timestamp); } else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightSavePointFileName(timestamp) : - HoodieTimeline.makeSavePointFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightSavePointFileName(timestamp) + : HoodieTimeline.makeSavePointFileName(timestamp); } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightDeltaFileName(timestamp) : - HoodieTimeline.makeDeltaFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightDeltaFileName(timestamp) + : HoodieTimeline.makeDeltaFileName(timestamp); } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } @@ -111,9 +106,9 @@ public boolean equals(Object o) { return false; } HoodieInstant that = (HoodieInstant) o; - return isInflight == that.isInflight && - Objects.equals(action, that.action) && - Objects.equals(timestamp, that.timestamp); + return isInflight == that.isInflight + && Objects.equals(action, that.action) + && Objects.equals(timestamp, that.timestamp); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index 60df7f4715b38..8978bf91af2d7 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -55,209 +55,209 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.ReadOptimizedView, TableFileSystemView.RealtimeView, Serializable { - protected HoodieTableMetaClient metaClient; - // This is the commits that will be visible for all views extending this view - protected HoodieTimeline visibleActiveTimeline; + protected HoodieTableMetaClient metaClient; + // This is the commits that will be visible for all views extending this view + protected HoodieTimeline visibleActiveTimeline; - // mapping from partition paths to file groups contained within them - protected HashMap> partitionToFileGroupsMap; - // mapping from file id to the file group. - protected HashMap fileGroupMap; + // mapping from partition paths to file groups contained within them + protected HashMap> partitionToFileGroupsMap; + // mapping from file id to the file group. + protected HashMap fileGroupMap; - /** - * Create a file system view, as of the given timeline - */ - public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, - HoodieTimeline visibleActiveTimeline) { - this.metaClient = metaClient; - this.visibleActiveTimeline = visibleActiveTimeline; - this.fileGroupMap = new HashMap<>(); - this.partitionToFileGroupsMap = new HashMap<>(); - } + /** + * Create a file system view, as of the given timeline + */ + public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline) { + this.metaClient = metaClient; + this.visibleActiveTimeline = visibleActiveTimeline; + this.fileGroupMap = new HashMap<>(); + this.partitionToFileGroupsMap = new HashMap<>(); + } - /** - * Create a file system view, as of the given timeline, with the provided file statuses. - */ - public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, - HoodieTimeline visibleActiveTimeline, - FileStatus[] fileStatuses) { - this(metaClient, visibleActiveTimeline); - addFilesToView(fileStatuses); - } + /** + * Create a file system view, as of the given timeline, with the provided file statuses. + */ + public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline, + FileStatus[] fileStatuses) { + this(metaClient, visibleActiveTimeline); + addFilesToView(fileStatuses); + } - /** - * This method is only used when this object is deserialized in a spark executor. - * - * @deprecated - */ - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - } + /** + * This method is only used when this object is deserialized in a spark executor. + * + * @deprecated + */ + private void readObject(java.io.ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } - private void writeObject(java.io.ObjectOutputStream out) - throws IOException { - out.defaultWriteObject(); - } + private void writeObject(java.io.ObjectOutputStream out) + throws IOException { + out.defaultWriteObject(); + } - /** - * Adds the provided statuses into the file system view, and also caches it inside this object. - */ - private List addFilesToView(FileStatus[] statuses) { - Map, List> dataFiles = convertFileStatusesToDataFiles( - statuses) - .collect(Collectors.groupingBy((dataFile) -> { - String partitionPathStr = FSUtils.getRelativePartitionPath( - new Path(metaClient.getBasePath()), - dataFile.getFileStatus().getPath().getParent()); - return Pair.of(partitionPathStr, dataFile.getFileId()); - })); - Map, List> logFiles = convertFileStatusesToLogFiles( - statuses) - .collect(Collectors.groupingBy((logFile) -> { - String partitionPathStr = FSUtils.getRelativePartitionPath( - new Path(metaClient.getBasePath()), - logFile.getPath().getParent()); - return Pair.of(partitionPathStr, logFile.getFileId()); - })); + /** + * Adds the provided statuses into the file system view, and also caches it inside this object. + */ + private List addFilesToView(FileStatus[] statuses) { + Map, List> dataFiles = convertFileStatusesToDataFiles( + statuses) + .collect(Collectors.groupingBy((dataFile) -> { + String partitionPathStr = FSUtils.getRelativePartitionPath( + new Path(metaClient.getBasePath()), + dataFile.getFileStatus().getPath().getParent()); + return Pair.of(partitionPathStr, dataFile.getFileId()); + })); + Map, List> logFiles = convertFileStatusesToLogFiles( + statuses) + .collect(Collectors.groupingBy((logFile) -> { + String partitionPathStr = FSUtils.getRelativePartitionPath( + new Path(metaClient.getBasePath()), + logFile.getPath().getParent()); + return Pair.of(partitionPathStr, logFile.getFileId()); + })); - Set> fileIdSet = new HashSet<>(dataFiles.keySet()); - fileIdSet.addAll(logFiles.keySet()); + Set> fileIdSet = new HashSet<>(dataFiles.keySet()); + fileIdSet.addAll(logFiles.keySet()); - List fileGroups = new ArrayList<>(); - fileIdSet.forEach(pair -> { - HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), - visibleActiveTimeline); - if (dataFiles.containsKey(pair)) { - dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); - } - if (logFiles.containsKey(pair)) { - logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); - } - fileGroups.add(group); - }); + List fileGroups = new ArrayList<>(); + fileIdSet.forEach(pair -> { + HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), + visibleActiveTimeline); + if (dataFiles.containsKey(pair)) { + dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); + } + if (logFiles.containsKey(pair)) { + logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); + } + fileGroups.add(group); + }); - // add to the cache. - fileGroups.forEach(group -> { - fileGroupMap.put(group.getId(), group); - if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) { - partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>()); - } - partitionToFileGroupsMap.get(group.getPartitionPath()).add(group); - }); + // add to the cache. + fileGroups.forEach(group -> { + fileGroupMap.put(group.getId(), group); + if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) { + partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>()); + } + partitionToFileGroupsMap.get(group.getPartitionPath()).add(group); + }); - return fileGroups; - } + return fileGroups; + } - private Stream convertFileStatusesToDataFiles(FileStatus[] statuses) { - Predicate roFilePredicate = fileStatus -> - fileStatus.getPath().getName() - .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); - return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new); - } + private Stream convertFileStatusesToDataFiles(FileStatus[] statuses) { + Predicate roFilePredicate = fileStatus -> + fileStatus.getPath().getName() + .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); + return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new); + } - private Stream convertFileStatusesToLogFiles(FileStatus[] statuses) { - Predicate rtFilePredicate = fileStatus -> - fileStatus.getPath().getName() - .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); - return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); - } + private Stream convertFileStatusesToLogFiles(FileStatus[] statuses) { + Predicate rtFilePredicate = fileStatus -> + fileStatus.getPath().getName() + .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); + return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); + } - @Override - public Stream getLatestDataFiles(final String partitionPath) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestDataFile()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestDataFiles(final String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestDataFile()) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getLatestDataFiles() { - return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestDataFile()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestDataFiles() { + return fileGroupMap.values().stream() + .map(fileGroup -> fileGroup.getLatestDataFile()) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getLatestDataFilesBeforeOrOn(String partitionPath, - String maxCommitTime) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestDataFilesBeforeOrOn(String partitionPath, + String maxCommitTime) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getLatestDataFilesInRange(List commitsToReturn) { - return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestDataFilesInRange(List commitsToReturn) { + return fileGroupMap.values().stream() + .map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getAllDataFiles(String partitionPath) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getAllDataFiles()) - .flatMap(dataFileList -> dataFileList); - } + @Override + public Stream getAllDataFiles(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getAllDataFiles()) + .flatMap(dataFileList -> dataFileList); + } - @Override - public Stream getLatestFileSlices(String partitionPath) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestFileSlice()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestFileSlices(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestFileSlice()) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, - String maxCommitTime) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, + String maxCommitTime) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getLatestFileSliceInRange(List commitsToReturn) { - return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } + @Override + public Stream getLatestFileSliceInRange(List commitsToReturn) { + return fileGroupMap.values().stream() + .map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } - @Override - public Stream getAllFileSlices(String partitionPath) { - return getAllFileGroups(partitionPath) - .map(group -> group.getAllFileSlices()) - .flatMap(sliceList -> sliceList); - } + @Override + public Stream getAllFileSlices(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(group -> group.getAllFileSlices()) + .flatMap(sliceList -> sliceList); + } - /** - * Given a partition path, obtain all filegroups within that. All methods, that work at the - * partition level go through this. - */ - @Override - public Stream getAllFileGroups(String partitionPathStr) { - // return any previously fetched groups. - if (partitionToFileGroupsMap.containsKey(partitionPathStr)) { - return partitionToFileGroupsMap.get(partitionPathStr).stream(); - } + /** + * Given a partition path, obtain all filegroups within that. All methods, that work at the + * partition level go through this. + */ + @Override + public Stream getAllFileGroups(String partitionPathStr) { + // return any previously fetched groups. + if (partitionToFileGroupsMap.containsKey(partitionPathStr)) { + return partitionToFileGroupsMap.get(partitionPathStr).stream(); + } - try { - // Create the path if it does not exist already - Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); - FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); - FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); - List fileGroups = addFilesToView(statuses); - return fileGroups.stream(); - } catch (IOException e) { - throw new HoodieIOException( - "Failed to list data files in partition " + partitionPathStr, e); - } + try { + // Create the path if it does not exist already + Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); + FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); + FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); + List fileGroups = addFilesToView(statuses); + return fileGroups.stream(); + } catch (IOException e) { + throw new HoodieIOException( + "Failed to list data files in partition " + partitionPathStr, e); } + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index 31fac73558b1f..e837285b1f2e0 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -94,8 +94,10 @@ public static String makeDataFileName(String commitTime, int taskPartitionId, St return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime); } - public static String makeTempDataFileName(String partitionPath, String commitTime, int taskPartitionId, String fileId, int stageId, long taskAttemptId) { - return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId, taskPartitionId, commitTime, stageId, taskAttemptId); + public static String makeTempDataFileName(String partitionPath, String commitTime, + int taskPartitionId, String fileId, int stageId, long taskAttemptId) { + return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId, + taskPartitionId, commitTime, stageId, taskAttemptId); } public static String maskWithoutFileId(String commitTime, int taskPartitionId) { @@ -299,7 +301,8 @@ public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final Optional currentVersion = getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime); // handle potential overflow - return (currentVersion.isPresent()) ? currentVersion.get() + 1 : HoodieLogFile.LOGFILE_BASE_VERSION; + return (currentVersion.isPresent()) ? currentVersion.get() + 1 + : HoodieLogFile.LOGFILE_BASE_VERSION; } public static int getDefaultBufferSize(final FileSystem fs) { @@ -331,7 +334,8 @@ public static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final if (recovered) { break; } - // Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover under default settings + // Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover + // under default settings Thread.sleep(1000); } return recovered; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index 519ce7b60dbec..09d1a2cab8078 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -45,11 +45,11 @@ public class HoodieAvroUtils { // All metadata fields are optional strings. - private final static 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.STRING))); - private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + private static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); /** * Convert a given avro record to bytes diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index ecbaad1c96980..f4215c8045f47 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -50,7 +50,7 @@ public class ParquetUtils { /** * Read the rowKey list from the given parquet file. * - * @param filePath The parquet file path. + * @param filePath The parquet file path. * @param configuration configuration to build fs object */ public static Set readRowKeysFromParquet(Configuration configuration, Path filePath) { @@ -116,8 +116,8 @@ private static List readParquetFooter(Configuration configuration, Path if (metadata.containsKey(footerName)) { footerVals.add(metadata.get(footerName)); } else { - throw new MetadataNotFoundException("Could not find index in Parquet footer. " + - "Looked for key " + footerName + " in " + parquetFilePath); + throw new MetadataNotFoundException("Could not find index in Parquet footer. " + + "Looked for key " + footerName + " in " + parquetFilePath); } } return footerVals; @@ -146,7 +146,7 @@ public static String[] readMinMaxRecordKeys(Configuration configuration, Path pa "Could not read min/max record key out of footer correctly from %s. read) : %s", parquetFilePath, minMaxKeys)); } - return new String[]{minMaxKeys.get(0), minMaxKeys.get(1)}; + return new String[] {minMaxKeys.get(0), minMaxKeys.get(1)}; } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java index 629935ee942de..1d5d3b58f8a98 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java @@ -55,7 +55,8 @@ public static T loadPayload(String recordPayload } return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes) .newInstance(payloadArgs); - } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + } catch (InstantiationException | IllegalAccessException + | InvocationTargetException | NoSuchMethodException e) { throw new HoodieException("Unable to instantiate payload class ", e); } catch (ClassNotFoundException e) { throw new HoodieException("Unable to instantiate payload class ", e); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index bdd5e692141f6..94aa1758da444 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.common.util; import com.uber.hoodie.common.model.HoodieKey; @@ -22,39 +23,28 @@ import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieCorruptedDataException; -import org.apache.avro.generic.GenericRecord; - import java.io.IOException; import java.io.RandomAccessFile; import java.util.Optional; import java.util.zip.CRC32; +import org.apache.avro.generic.GenericRecord; public class SpillableMapUtils { /** * Using the schema and payload class, read and convert the bytes on disk to a HoodieRecord - * - * @param file - * @param valuePosition - * @param valueLength - * @return - * @throws IOException */ - public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) throws IOException { + public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) + throws IOException { DiskBasedMap.FileEntry fileEntry = readInternal(file, valuePosition, valueLength); return fileEntry.getValue(); } /** * |crc|timestamp|sizeOfKey|SizeOfValue|key|value| - * - * @param file - * @param valuePosition - * @param valueLength - * @return - * @throws IOException */ - private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition, int valueLength) throws IOException { + private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition, + int valueLength) throws IOException { file.seek(valuePosition); long crc = file.readLong(); long timestamp = file.readLong(); @@ -69,27 +59,23 @@ private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long v file.read(value, 0, valueSize); long crcOfReadValue = generateChecksum(value); if (!(crc == crcOfReadValue)) { - throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " + - "data may be corrupted"); + throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " + + "data may be corrupted"); } return new DiskBasedMap.FileEntry(crc, keySize, valueSize, key, value, timestamp); } /** - * Write Value and other metadata necessary to disk. Each entry has the following sequence of data - *

+ * Write Value and other metadata necessary to disk. Each entry has the following sequence of data

* |crc|timestamp|sizeOfKey|SizeOfValue|key|value| - * - * @param outputStream - * @param fileEntry - * @return - * @throws IOException */ - public static long spillToDisk(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) throws IOException { + public static long spillToDisk(SizeAwareDataOutputStream outputStream, + DiskBasedMap.FileEntry fileEntry) throws IOException { return spill(outputStream, fileEntry); } - private static long spill(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) + private static long spill(SizeAwareDataOutputStream outputStream, + DiskBasedMap.FileEntry fileEntry) throws IOException { outputStream.writeLong(fileEntry.getCrc()); outputStream.writeLong(fileEntry.getTimestamp()); @@ -102,9 +88,6 @@ private static long spill(SizeAwareDataOutputStream outputStream, DiskBasedMap.F /** * Generate a checksum for a given set of bytes - * - * @param data - * @return */ public static long generateChecksum(byte[] data) { CRC32 crc = new CRC32(); @@ -113,13 +96,8 @@ public static long generateChecksum(byte[] data) { } /** - * Compute a bytes representation of the payload by serializing the contents - * This is used to estimate the size of the payload (either in memory or when written to disk) - * - * @param - * @param value - * @return - * @throws IOException + * Compute a bytes representation of the payload by serializing the contents This is used to estimate the size of the + * payload (either in memory or when written to disk) */ public static long computePayloadSize(R value, Converter valueConverter) throws IOException { return valueConverter.sizeEstimate(value); @@ -127,12 +105,6 @@ public static long computePayloadSize(R value, Converter valueConverter) /** * Utility method to convert bytes to HoodieRecord using schema and payload class - * - * @param rec - * @param payloadClazz - * @param - * @return - * @throws IOException */ public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz) { String recKey = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java index 27ee0fba2be1f..0c3ab8626a409 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -22,9 +22,6 @@ import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -40,18 +37,25 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** - * This class provides a disk spillable only map implementation. All of the data is - * currenly written to one file, without any rollover support. It uses the following : - * 1) An in-memory map that tracks the key-> latest ValueMetadata. - * 2) Current position in the file - * NOTE : Only String.class type supported for Key + * This class provides a disk spillable only map implementation. All of the data is currenly written to one file, + * without any rollover support. It uses the following : 1) An in-memory map that tracks the key-> latest ValueMetadata. + * 2) Current position in the file NOTE : Only String.class type supported for Key */ -final public class DiskBasedMap implements Map { +public final class DiskBasedMap implements Map { + private static final Logger log = LogManager.getLogger(DiskBasedMap.class); + // Default file path prefix to put the spillable file + private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; // Stores the key and corresponding value's latest metadata spilled to disk - final private Map valueMetadataMap; + private final Map valueMetadataMap; + // Key converter to convert key type to bytes + private final Converter keyConverter; + // Value converter to convert value type to bytes + private final Converter valueConverter; // Read only file access to be able to seek to random positions to readFromDisk values private RandomAccessFile readOnlyFileHandle; // Write only OutputStream to be able to ONLY append to the file @@ -63,100 +67,7 @@ final public class DiskBasedMap implements Map { private AtomicLong filePosition; // FilePath to store the spilled data private String filePath; - // Default file path prefix to put the spillable file - private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; - // Key converter to convert key type to bytes - final private Converter keyConverter; - // Value converter to convert value type to bytes - final private Converter valueConverter; - - private static Logger log = LogManager.getLogger(DiskBasedMap.class); - - - public final class ValueMetadata { - - // FilePath to store the spilled data - private String filePath; - // Size (numberOfBytes) of the value written to disk - private Integer sizeOfValue; - // FilePosition of the value written to disk - private Long offsetOfValue; - // Current timestamp when the value was written to disk - private Long timestamp; - - protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) { - this.filePath = filePath; - this.sizeOfValue = sizeOfValue; - this.offsetOfValue = offsetOfValue; - this.timestamp = timestamp; - } - - public String getFilePath() { - return filePath; - } - - public int getSizeOfValue() { - return sizeOfValue; - } - - public Long getOffsetOfValue() { - return offsetOfValue; - } - - public long getTimestamp() { - return timestamp; - } - } - - public static final class FileEntry { - - // Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption - private Long crc; - // Size (numberOfBytes) of the key written to disk - private Integer sizeOfKey; - // Size (numberOfBytes) of the value written to disk - private Integer sizeOfValue; - // Actual key - private byte[] key; - // Actual value - private byte[] value; - // Current timestamp when the value was written to disk - private Long timestamp; - - public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value, - long timestamp) { - this.crc = crc; - this.sizeOfKey = sizeOfKey; - this.sizeOfValue = sizeOfValue; - this.key = key; - this.value = value; - this.timestamp = timestamp; - } - - public long getCrc() { - return crc; - } - - public int getSizeOfKey() { - return sizeOfKey; - } - - public int getSizeOfValue() { - return sizeOfValue; - } - - public byte[] getKey() { - return key; - } - - public byte[] getValue() { - return value; - } - public long getTimestamp() { - return timestamp; - } - } protected DiskBasedMap(Optional baseFilePath, Converter keyConverter, Converter valueConverter) throws IOException { @@ -193,8 +104,8 @@ private void initFile(File writeOnlyFileHandle) throws IOException { } /** - * Register shutdown hook to force flush contents of the data written to FileOutputStream - * from OS page cache (typically 4 KB) to disk + * Register shutdown hook to force flush contents of the data written to FileOutputStream from OS page cache + * (typically 4 KB) to disk */ private void addShutDownHook() { Runtime.getRuntime().addShutdownHook(new Thread() { @@ -324,4 +235,89 @@ public Set> entrySet() { } return entrySet; } + + public static final class FileEntry { + + // Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption + private Long crc; + // Size (numberOfBytes) of the key written to disk + private Integer sizeOfKey; + // Size (numberOfBytes) of the value written to disk + private Integer sizeOfValue; + // Actual key + private byte[] key; + // Actual value + private byte[] value; + // Current timestamp when the value was written to disk + private Long timestamp; + + public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value, + long timestamp) { + this.crc = crc; + this.sizeOfKey = sizeOfKey; + this.sizeOfValue = sizeOfValue; + this.key = key; + this.value = value; + this.timestamp = timestamp; + } + + public long getCrc() { + return crc; + } + + public int getSizeOfKey() { + return sizeOfKey; + } + + public int getSizeOfValue() { + return sizeOfValue; + } + + public byte[] getKey() { + return key; + } + + public byte[] getValue() { + return value; + } + + public long getTimestamp() { + return timestamp; + } + } + + public final class ValueMetadata { + + // FilePath to store the spilled data + private String filePath; + // Size (numberOfBytes) of the value written to disk + private Integer sizeOfValue; + // FilePosition of the value written to disk + private Long offsetOfValue; + // Current timestamp when the value was written to disk + private Long timestamp; + + protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) { + this.filePath = filePath; + this.sizeOfValue = sizeOfValue; + this.offsetOfValue = offsetOfValue; + this.timestamp = timestamp; + } + + public String getFilePath() { + return filePath; + } + + public int getSizeOfValue() { + return sizeOfValue; + } + + public Long getOffsetOfValue() { + return offsetOfValue; + } + + public long getTimestamp() { + return timestamp; + } + } } \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index 3d390690edc06..b3ff515c04457 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -18,11 +18,7 @@ import com.twitter.common.objectsize.ObjectSizeCalculator; import com.uber.hoodie.common.util.collection.converter.Converter; -import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.Collection; import java.util.HashMap; @@ -31,43 +27,43 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** - * An external map that spills content to disk when there is insufficient space for it to grow.

- * This map holds 2 types of data structures :

(1) Key-Value pairs in a in-memory map (2) - * Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk - *

NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed - * from the in-memory key-valueMetadata map but it's values will be lying around in the temp file on - * disk until the file is cleaned.

The setting of the spill threshold faces the following - * trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is - * available, resulting in OOM. However, if the spill threshold is too low, we spill frequently and - * incur unnecessary disk writes. + * An external map that spills content to disk when there is insufficient space for it to grow.

This map holds 2 + * types of data structures :

(1) Key-Value pairs in a in-memory map (2) Key-ValueMetadata pairs in an in-memory map + * which keeps a marker to the values spilled to disk

NOTE : Values are only appended to disk. If a remove() is + * called, the entry is marked removed from the in-memory key-valueMetadata map but it's values will be lying around in + * the temp file on disk until the file is cleaned.

The setting of the spill threshold faces the following + * trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is available, resulting + * in OOM. However, if the spill threshold is too low, we spill frequently and incur unnecessary disk writes. */ public class ExternalSpillableMap implements Map { // Find the actual estimated payload size after inserting N records - final private static int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100; + private static final int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100; + private static final Logger log = LogManager.getLogger(ExternalSpillableMap.class); // maximum space allowed in-memory for this map - final private long maxInMemorySizeInBytes; - // current space occupied by this map in-memory - private Long currentInMemoryMapSize; + private final long maxInMemorySizeInBytes; // Map to store key-values in memory until it hits maxInMemorySizeInBytes - final private Map inMemoryMap; + private final Map inMemoryMap; // Map to store key-valuemetadata important to find the values spilled to disk - final private DiskBasedMap diskBasedMap; - // An estimate of the size of each payload written to this map - private volatile long estimatedPayloadSize = 0; - // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and incorrect payload estimation - final private Double sizingFactorForInMemoryMap = 0.8; + private final DiskBasedMap diskBasedMap; + // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and + // incorrect payload estimation + private final Double sizingFactorForInMemoryMap = 0.8; // Key converter to convert key type to bytes - final private Converter keyConverter; + private final Converter keyConverter; // Value converter to convert value type to bytes - final private Converter valueConverter; + private final Converter valueConverter; + // current space occupied by this map in-memory + private Long currentInMemoryMapSize; + // An estimate of the size of each payload written to this map + private volatile long estimatedPayloadSize = 0; // Flag to determine whether to stop re-estimating payload size private boolean shouldEstimatePayloadSize = true; - private static Logger log = LogManager.getLogger(ExternalSpillableMap.class); - public ExternalSpillableMap(Long maxInMemorySizeInBytes, Optional baseFilePath, Converter keyConverter, Converter valueConverter) throws IOException { this.inMemoryMap = new HashMap<>(); @@ -153,15 +149,14 @@ public R put(T key, R value) { this.estimatedPayloadSize = keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value); log.info("Estimated Payload size => " + estimatedPayloadSize); - } - else if(shouldEstimatePayloadSize && - inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { + } else if (shouldEstimatePayloadSize + && inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { // Re-estimate the size of a record by calculating the size of the entire map containing // N entries and then dividing by the number of entries present (N). This helps to get a // correct estimation of the size of each record in the JVM. long totalMapSize = ObjectSizeCalculator.getObjectSize(inMemoryMap); this.currentInMemoryMapSize = totalMapSize; - this.estimatedPayloadSize = totalMapSize/inMemoryMap.size(); + this.estimatedPayloadSize = totalMapSize / inMemoryMap.size(); shouldEstimatePayloadSize = false; log.info("New Estimated Payload size => " + this.estimatedPayloadSize); } @@ -227,9 +222,8 @@ public Set> entrySet() { } /** - * Iterator that wraps iterating over all the values for this map - * 1) inMemoryIterator - Iterates over all the data in-memory map - * 2) diskLazyFileIterator - Iterates over all the data spilled to disk + * Iterator that wraps iterating over all the values for this map 1) inMemoryIterator - Iterates over all the data + * in-memory map 2) diskLazyFileIterator - Iterates over all the data spilled to disk */ private class IteratorWrapper implements Iterator { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java index 78eaa3f348c48..0d53e2659ffb6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java @@ -20,7 +20,6 @@ import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; - import java.io.IOException; import java.io.RandomAccessFile; import java.util.Iterator; @@ -29,8 +28,7 @@ import java.util.stream.Collectors; /** - * Iterable to lazily fetch values spilled to disk. - * This class uses RandomAccessFile to randomly access the position of + * Iterable to lazily fetch values spilled to disk. This class uses RandomAccessFile to randomly access the position of * the latest value for a key spilled to disk and returns the result. */ public class LazyFileIterable implements Iterable { @@ -63,9 +61,9 @@ public Iterator iterator() { */ public class LazyFileIterator implements Iterator { + private final Converter valueConverter; private RandomAccessFile readOnlyFileHandle; private Iterator> metadataIterator; - private final Converter valueConverter; public LazyFileIterator(RandomAccessFile file, Map map, Converter valueConverter) throws IOException { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java index 73dc0541ad0c3..88bf7fcd06065 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -22,14 +22,9 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.ReflectionUtils; -import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieNotSerializableException; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.Field; -import java.util.Arrays; import java.util.Optional; -import java.util.stream.Stream; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.commons.lang3.SerializationUtils; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java index 8280dda3f594a..df31305de7c46 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java @@ -25,6 +25,7 @@ * Wrapper for DataOutpuStream to keep track of number of bytes written */ public class SizeAwareDataOutputStream { + // Actual outpuStream private DataOutputStream outputStream; // Counter to keep track of number of bytes written @@ -45,12 +46,12 @@ public void writeInt(int v) throws IOException { outputStream.writeInt(v); } - public void write(byte [] v) throws IOException { + public void write(byte[] v) throws IOException { size.addAndGet(v.length); outputStream.write(v); } - public void write(byte [] v, int offset, int len) throws IOException { + public void write(byte[] v, int offset, int len) throws IOException { size.addAndGet(len + offset); outputStream.write(v, offset, len); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java index 3c10a15377d7d..773dbfc923b67 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java @@ -17,7 +17,8 @@ package com.uber.hoodie.exception; /** - *

Exception thrown when any data corruption happens when reading/writing from temporary disk

+ *

Exception thrown when any data corruption happens when reading/writing from temporary disk + *

*/ public class HoodieCorruptedDataException extends HoodieException { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java b/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java index 6c024a897fab5..dbc3567ff8d65 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java @@ -19,9 +19,7 @@ import java.io.IOException; import org.junit.Test; -public class - -TestBloomFilter { +public class TestBloomFilter { @Test public void testAddKey() { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java index a90739f75c4b4..c5d6e1e934e18 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.minicluster; - import com.google.common.base.Preconditions; import com.google.common.io.Files; import com.uber.hoodie.common.model.HoodieTestUtils; @@ -63,25 +62,23 @@ public Configuration getHadoopConf() { } public MiniDFSCluster start(boolean format) throws IOException { - Preconditions - .checkState(workDir != null, "The work dir must be set before starting cluster."); + Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster."); hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); // If clean, then remove the work dir so we can start fresh. String localDFSLocation = getDFSLocation(workDir); if (format) { - logger.info( - "Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh."); + logger.info("Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh."); File file = new File(localDFSLocation); FileUtils.deleteDirectory(file); } // Configure and start the HDFS cluster // boolean format = shouldFormatDFSCluster(localDFSLocation, clean); - hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, - namenodeHttpPort, datanodePort, datanodeIpcPort, datanodeHttpPort); - miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format) - .checkDataNodeAddrConfig(true).checkDataNodeHostConfig(true).build(); + hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, namenodeHttpPort, + datanodePort, datanodeIpcPort, datanodeHttpPort); + miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format).checkDataNodeAddrConfig(true) + .checkDataNodeHostConfig(true).build(); logger.info("HDFS Minicluster service started."); return miniDfsCluster; } @@ -96,8 +93,7 @@ public void stop() throws IOException { /** * Get the location on the local FS where we store the HDFS data. * - * @param baseFsLocation The base location on the local filesystem we have write access to create - * dirs. + * @param baseFsLocation The base location on the local filesystem we have write access to create dirs. * @return The location for HDFS data. */ private static String getDFSLocation(String baseFsLocation) { @@ -105,11 +101,11 @@ private static String getDFSLocation(String baseFsLocation) { } /** - * Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the - * dfsFsLocation does not exist. + * Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the dfsFsLocation does not + * exist. * * @param localDFSLocation The location on the local FS to hold the HDFS metadata and block data - * @param clean Specifies if we want to start a clean cluster + * @param clean Specifies if we want to start a clean cluster * @return Returns true if we should format a DFSCluster, otherwise false */ private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean clean) { @@ -124,14 +120,13 @@ private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean c /** * Configure the DFS Cluster before launching it. * - * @param config The already created Hadoop configuration we'll further configure for HDFS + * @param config The already created Hadoop configuration we'll further configure for HDFS * @param localDFSLocation The location on the local filesystem where cluster data is stored - * @param bindIP An IP address we want to force the datanode and namenode to bind to. + * @param bindIP An IP address we want to force the datanode and namenode to bind to. * @return The updated Configuration object. */ - private static Configuration configureDFSCluster(Configuration config, String localDFSLocation, - String bindIP, int namenodeRpcPort, int namenodeHttpPort, int datanodePort, - int datanodeIpcPort, int datanodeHttpPort) { + private static Configuration configureDFSCluster(Configuration config, String localDFSLocation, String bindIP, + int namenodeRpcPort, int namenodeHttpPort, int datanodePort, int datanodeIpcPort, int datanodeHttpPort) { logger.info("HDFS force binding to ip: " + bindIP); config.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + bindIP + ":" + namenodeRpcPort); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java index 290064e19006a..d47bfa329fc18 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java @@ -43,11 +43,10 @@ *

* 2. It will only attempt to bind to the port specified, and will fail if it can't. *

- * 3. The startup method now takes a bindAddress, which allows us to configure which IP the ZK - * server binds to. This was not configurable in the original class. + * 3. The startup method now takes a bindAddress, which allows us to configure which IP the ZK server binds to. This was + * not configurable in the original class. *

- * 4. The ZK cluster will re-use a data dir on the local filesystem if it already exists instead of - * blowing it away. + * 4. The ZK cluster will re-use a data dir on the local filesystem if it already exists instead of blowing it away. */ public class ZookeeperTestService { @@ -83,8 +82,7 @@ public Configuration getHadoopConf() { } public ZooKeeperServer start() throws IOException, InterruptedException { - Preconditions.checkState(workDir != null, - "The localBaseFsLocation must be set before starting cluster."); + Preconditions.checkState(workDir != null, "The localBaseFsLocation must be set before starting cluster."); setupTestEnv(); stop(); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java index 984fb3a412c2c..bc40108e9840c 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java @@ -47,7 +47,8 @@ public HoodieRecordPayload preCombine(HoodieRecordPayload another) { } @Override - public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + throws IOException { return getInsertValue(schema); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index bd698e7c8ff15..361bd81e7ec54 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -80,19 +80,16 @@ public static Configuration getDefaultHadoopConf() { return new Configuration(); } - public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) - throws IOException { + public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) throws IOException { return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); } public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, - HoodieTableType tableType) - throws IOException { + HoodieTableType tableType) throws IOException { Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); - properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, - HoodieAvroPayload.class.getName()); + properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName()); return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); } @@ -108,63 +105,60 @@ public static String makeNewCommitTime() { return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); } - public static final void createCommitFiles(String basePath, String... commitTimes) - throws IOException { + public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)).createNewFile(); + new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime)) + .createNewFile(); } } - public static final void createInflightCommitFiles(String basePath, String... commitTimes) - throws IOException { + public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeInflightCommitFileName(commitTime)).createNewFile(); + new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName( + commitTime)).createNewFile(); } } - public static final void createInflightCleanFiles(String basePath, String... commitTimes) - throws IOException { + public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + - HoodieTimeline.makeInflightCleanerFileName(commitTime)).createNewFile(); + new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline + .makeInflightCleanerFileName( + commitTime)).createNewFile(); } } - public static final String createNewDataFile(String basePath, String partitionPath, - String commitTime) throws IOException { + public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) + throws IOException { String fileID = UUID.randomUUID().toString(); return createDataFile(basePath, partitionPath, commitTime, fileID); } - public static final String createDataFile(String basePath, String partitionPath, - String commitTime, String fileID) throws IOException { + public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID) + throws IOException { String folderPath = basePath + "/" + partitionPath + "/"; new File(folderPath).mkdirs(); - new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID)) - .createNewFile(); + new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID)).createNewFile(); return fileID; } - public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, - String commitTime, String fileID, Optional version) throws IOException { + public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime, + String fileID, Optional version) throws IOException { String folderPath = basePath + "/" + partitionPath + "/"; boolean makeDir = fs.mkdirs(new Path(folderPath)); if (!makeDir) { throw new IOException("cannot create directory for path " + folderPath); } - boolean createFile = fs.createNewFile(new Path(folderPath + FSUtils - .makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID)))); + boolean createFile = fs.createNewFile(new Path( + folderPath + FSUtils.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID)))); if (!createFile) { - throw new IOException(StringUtils - .format("cannot create data file for commit %s and fileId %s", commitTime, fileID)); + throw new IOException( + StringUtils.format("cannot create data file for commit %s and fileId %s", commitTime, fileID)); } return fileID; } - public static final void createCompactionCommitFiles(FileSystem fs, String basePath, - String... commitTimes) + public static final void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { boolean createFile = fs.createNewFile(new Path( @@ -176,42 +170,42 @@ public static final void createCompactionCommitFiles(FileSystem fs, String baseP } } - public static final String getDataFilePath(String basePath, String partitionPath, - String commitTime, String fileID) throws IOException { + public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) + throws IOException { return basePath + "/" + partitionPath + "/" + FSUtils .makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID); } - public static final String getLogFilePath(String basePath, String partitionPath, - String commitTime, String fileID, Optional version) throws IOException { - return basePath + "/" + partitionPath + "/" + FSUtils - .makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID)); + public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID, + Optional version) throws IOException { + return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime, + version.orElse(DEFAULT_TASK_PARTITIONID)); } - public static final String getCommitFilePath(String basePath, String commitTime) - throws IOException { - return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime - + HoodieTimeline.COMMIT_EXTENSION; + public static final String getCommitFilePath(String basePath, String commitTime) throws IOException { + return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION; } - public static final boolean doesDataFileExist(String basePath, String partitionPath, - String commitTime, String fileID) throws IOException { + public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID) + throws IOException { return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists(); } - public static final boolean doesLogFileExist(String basePath, String partitionPath, - String commitTime, String fileID, Optional version) throws IOException { + public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID, + Optional version) throws IOException { return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists(); } public static final boolean doesCommitExist(String basePath, String commitTime) { - return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime - + HoodieTimeline.COMMIT_EXTENSION).exists(); + return new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION) + .exists(); } public static final boolean doesInflightExist(String basePath, String commitTime) { - return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime - + HoodieTimeline.INFLIGHT_EXTENSION).exists(); + return new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION) + .exists(); } public static String makeInflightTestFileName(String instant) { @@ -219,20 +213,17 @@ public static String makeInflightTestFileName(String instant) { } public static void createCleanFiles(String basePath, String commitTime) throws IOException { - Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCleanerFileName(commitTime)); + Path commitFile = new Path( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime)); FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); FSDataOutputStream os = fs.create(commitFile, true); try { - HoodieCleanStat cleanStats = new HoodieCleanStat( - HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, - DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], - new ArrayList<>(), new ArrayList<>(), + HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, + DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), commitTime); // Create the clean metadata - HoodieCleanMetadata cleanMetadata = - AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), Arrays.asList(cleanStats)); + HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), + Arrays.asList(cleanStats)); // Write empty clean metadata os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get()); } finally { @@ -249,7 +240,8 @@ public static String makeCommitFileName(String instant) { } public static void assertStreamEquals(String message, Stream expected, Stream actual) { - Iterator iter1 = expected.iterator(), iter2 = actual.iterator(); + Iterator iter1 = expected.iterator(); + Iterator iter2 = actual.iterator(); while (iter1.hasNext() && iter2.hasNext()) { assertEquals(message, iter1.next(), iter2.next()); } @@ -275,8 +267,8 @@ public static T serializeDeserialize(T object, Class public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema schema, List updatedRecords) { - Map> groupedUpdated = updatedRecords.stream() - .collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)); + Map> groupedUpdated = updatedRecords.stream().collect( + Collectors.groupingBy(HoodieRecord::getCurrentLocation)); groupedUpdated.entrySet().forEach(s -> { HoodieRecordLocation location = s.getKey(); @@ -284,12 +276,9 @@ public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema Writer logWriter; try { - logWriter = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(basePath, partitionPath)) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION) - .withFileId(location.getFileId()) - .overBaseCommit(location.getCommitTime()) - .withFs(fs).build(); + logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath)) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId()) + .overBaseCommit(location.getCommitTime()).withFs(fs).build(); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getCommitTime()); @@ -297,10 +286,7 @@ public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> { try { GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); - HoodieAvroUtils.addHoodieKeyToRecord(val, - r.getRecordKey(), - r.getPartitionPath(), - ""); + HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); return (IndexedRecord) val; } catch (IOException e) { return null; @@ -313,8 +299,7 @@ public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema }); } - public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) - throws IOException { + public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) throws IOException { RemoteIterator itr = fs.listFiles(new Path(basePath), true); List returns = Lists.newArrayList(); while (itr.hasNext()) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java index 77c7d21def0bc..a462a6da39830 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java @@ -43,10 +43,10 @@ public void testSetPaths() { Path partitionPath = new Path(basePath, partitionPathString); Path tempPath = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME); - Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, - taskPartitionId, fileName)); - Path tempFilePath = new Path(tempPath, FSUtils.makeTempDataFileName(partitionPathString, - commitTime, taskPartitionId, fileName, stageId, taskAttemptId)); + Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)); + Path tempFilePath = new Path(tempPath, FSUtils + .makeTempDataFileName(partitionPathString, commitTime, taskPartitionId, + fileName, stageId, taskAttemptId)); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setPaths(basePath, finalizeFilePath, tempFilePath); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index bd710d1a99e52..414166b99ed7a 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -53,25 +53,22 @@ public void checkMetadata() { assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME, metaClient.getTableConfig().getTableName()); assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath()); - assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie", - metaClient.getMetaPath()); + assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie", metaClient.getMetaPath()); } @Test public void checkSerDe() throws IOException, ClassNotFoundException { // check if this object is serialized and de-serialized, we are able to read from the file system - HoodieTableMetaClient deseralizedMetaClient = - HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class); + HoodieTableMetaClient deseralizedMetaClient = HoodieTestUtils + .serializeDeserialize(metaClient, HoodieTableMetaClient.class); assertNotNull(deseralizedMetaClient); HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline(); - HoodieInstant instant = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); commitTimeline.createInflight(instant); commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); commitTimeline = commitTimeline.reload(); HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant); - assertEquals("Commit should be 1 and completed", completedInstant, - commitTimeline.getInstants().findFirst().get()); + assertEquals("Commit should be 1 and completed", completedInstant, commitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), commitTimeline.getInstantDetails(completedInstant).get()); } @@ -82,8 +79,7 @@ public void checkCommitTimeline() throws IOException { HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); - HoodieInstant instant = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); activeTimeline.createInflight(instant); activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); @@ -96,8 +92,7 @@ public void checkCommitTimeline() throws IOException { activeTimeline = activeTimeline.reload(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty()); - assertEquals("Commit should be 1", completedInstant, - activeCommitTimeline.getInstants().findFirst().get()); + assertEquals("Commit should be 1", completedInstant, activeCommitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), activeCommitTimeline.getInstantDetails(completedInstant).get()); } @@ -118,22 +113,16 @@ public void checkArchiveCommitTimeline() throws IOException { HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); - HoodieInstant instant1 = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2 = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); - HoodieInstant instant3 = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); assertEquals(Lists.newArrayList(instant1, instant2, instant3), archivedTimeline.getInstants().collect(Collectors.toList())); - assertArrayEquals(new Text("data1").getBytes(), - archivedTimeline.getInstantDetails(instant1).get()); - assertArrayEquals(new Text("data2").getBytes(), - archivedTimeline.getInstantDetails(instant2).get()); - assertArrayEquals(new Text("data3").getBytes(), - archivedTimeline.getInstantDetails(instant3).get()); + assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.getInstantDetails(instant1).get()); + assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.getInstantDetails(instant2).get()); + assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.getInstantDetails(instant3).get()); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 209a53cb40ab5..3d37e6816c25d 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -16,6 +16,12 @@ package com.uber.hoodie.common.table.log; +import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import com.google.common.collect.Maps; import com.uber.hoodie.common.minicluster.MiniClusterUtil; import com.uber.hoodie.common.model.HoodieArchivedLogFile; @@ -35,6 +41,17 @@ import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.exception.CorruptedLogFileException; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -51,24 +68,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.IOException; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - @SuppressWarnings("Duplicates") @RunWith(Parameterized.class) public class HoodieLogFormatTest { @@ -85,9 +84,7 @@ public HoodieLogFormatTest(Boolean readBlocksLazily) { @Parameterized.Parameters(name = "LogBlockReadMode") public static Collection data() { - return Arrays.asList(new Boolean[][]{ - {true},{false} - }); + return Arrays.asList(new Boolean[][] {{true}, {false}}); } @BeforeClass @@ -123,10 +120,8 @@ public void testEmptyLog() throws IOException, InterruptedException { .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); assertEquals("Just created this log, size should be 0", 0, writer.getCurrentSize()); - assertTrue("Check all log files should start with a .", - writer.getLogFile().getFileName().startsWith(".")); - assertEquals("Version should be 1 for new log created", 1, - writer.getLogFile().getLogVersion()); + assertTrue("Check all log files should start with a .", writer.getLogFile().getFileName().startsWith(".")); + assertEquals("Version should be 1 for new log created", 1, writer.getLogFile().getLogVersion()); } @Test @@ -142,9 +137,8 @@ public void testBasicAppend() throws IOException, InterruptedException, URISynta writer = writer.appendBlock(dataBlock); long size = writer.getCurrentSize(); assertTrue("We just wrote a block - size should be > 0", size > 0); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); } @@ -166,14 +160,13 @@ public void testRollover() throws IOException, InterruptedException, URISyntaxEx // Create a writer with the size threshold as the size we just wrote - so this has to roll writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).withSizeThreshold(size - 1).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); - assertEquals("This should be a new log file and hence size should be 0", 0, - writer.getCurrentSize()); + assertEquals("This should be a new log file and hence size should be 0", 0, writer.getCurrentSize()); assertEquals("Version should be rolled to 2", 2, writer.getLogFile().getLogVersion()); writer.close(); } @@ -193,32 +186,30 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size2, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size3 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size3, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size3, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); // Cannot get the current size after closing the log @@ -246,27 +237,24 @@ public void testLeaseRecovery() throws IOException, URISyntaxException, Interrup // writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size2, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); } @Test - public void testAppendNotSupported() - throws IOException, URISyntaxException, InterruptedException { + public void testAppendNotSupported() throws IOException, URISyntaxException, InterruptedException { // Use some fs like LocalFileSystem, that does not support appends Path localPartitionPath = new Path("file://" + partitionPath); - FileSystem localFs = FSUtils - .getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); + FileSystem localFs = FSUtils.getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); Path testPath = new Path(localPartitionPath, "append_test"); localFs.mkdirs(testPath); @@ -279,8 +267,8 @@ public void testAppendNotSupported() for (int i = 0; i < 2; i++) { HoodieLogFormat.newWriterBuilder().onParentPath(testPath) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") - .overBaseCommit("").withFs(localFs).build().appendBlock(dataBlock).close(); + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive").overBaseCommit("") + .withFs(localFs).build().appendBlock(dataBlock).close(); } // ensure there are two log file versions, with same data. @@ -290,16 +278,14 @@ public void testAppendNotSupported() @SuppressWarnings("unchecked") @Test - public void testBasicWriteAndScan() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicWriteAndScan() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List records = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords = records.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords = records.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); @@ -307,31 +293,27 @@ public void testBasicWriteAndScan() writer = writer.appendBlock(dataBlock); writer.close(); - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("We wrote a block, we should be able to read it", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); - assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK, - nextBlock.getBlockType()); + assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK, nextBlock.getBlockType()); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords, dataBlockRead.getRecords()); } @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndRead() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); Schema schema = getSimpleSchema(); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); @@ -340,12 +322,11 @@ public void testBasicAppendAndRead() writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); @@ -353,51 +334,48 @@ public void testBasicAppendAndRead() // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords1.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); reader.hasNext(); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords2.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords2.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, dataBlockRead.getRecords()); reader.hasNext(); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords3.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords3.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, dataBlockRead.getRecords()); } @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndScanMultipleFiles() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndScanMultipleFiles() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withSizeThreshold(1024).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withSizeThreshold(1024) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); @@ -406,12 +384,11 @@ public void testBasicAppendAndScanMultipleFiles() Set logFiles = new HashSet<>(); List> allRecords = new ArrayList<>(); // create 4 log files - while(writer.getLogFile().getLogVersion() != 4) { + while (writer.getLogFile().getLogVersion() != 4) { logFiles.add(writer.getLogFile()); List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); allRecords.add(copyOfRecords1); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); @@ -421,24 +398,22 @@ public void testBasicAppendAndScanMultipleFiles() // scan all log blocks (across multiple log files) HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), - schema, "100", 10240L, readBlocksLazily, false); + logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", + 10240L, readBlocksLazily, false); List scannedRecords = new ArrayList<>(); - for(HoodieRecord record: scanner) { + for (HoodieRecord record : scanner) { scannedRecords.add((IndexedRecord) record.getData().getInsertValue(schema).get()); } - assertEquals("Scanner records count should be the same as appended records", - scannedRecords.size(), allRecords.stream().flatMap(records -> records.stream()) - .collect(Collectors.toList()).size()); + assertEquals("Scanner records count should be the same as appended records", scannedRecords.size(), + allRecords.stream().flatMap(records -> records.stream()).collect(Collectors.toList()).size()); } @Test - public void testAppendAndReadOnCorruptedLog() - throws IOException, URISyntaxException, InterruptedException { + public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); @@ -467,14 +442,12 @@ public void testAppendAndReadOnCorruptedLog() outputStream.close(); // First round of reads - we should be able to read the first block and then EOF - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); reader.next(); assertTrue("We should have corrupted block next", reader.hasNext()); HoodieLogBlock block = reader.next(); - assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, - block.getBlockType()); + assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block; //assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes())); assertFalse("There should be no more block left", reader.hasNext()); @@ -496,8 +469,8 @@ public void testAppendAndReadOnCorruptedLog() // Should be able to append a new block writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); @@ -505,16 +478,14 @@ public void testAppendAndReadOnCorruptedLog() writer.close(); // Second round of reads - we should be able to read the first and last block - reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); reader.next(); assertTrue("We should get the 1st corrupted block next", reader.hasNext()); reader.next(); assertTrue("We should get the 2nd corrupted block next", reader.hasNext()); block = reader.next(); - assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, - block.getBlockType()); + assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); corruptBlock = (HoodieCorruptBlock) block; //assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes())); assertTrue("We should get the last block next", reader.hasNext()); @@ -524,8 +495,7 @@ public void testAppendAndReadOnCorruptedLog() @Test - public void testAvroLogRecordReaderBasic() - throws IOException, URISyntaxException, InterruptedException { + public void testAvroLogRecordReaderBasic() throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) @@ -533,9 +503,8 @@ public void testAvroLogRecordReaderBasic() .overBaseCommit("100").withFs(fs).withSizeThreshold(500).build(); // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); @@ -545,22 +514,18 @@ public void testAvroLogRecordReaderBasic() // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "100", 10240L, readBlocksLazily, false); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -568,10 +533,8 @@ public void testAvroLogRecordReaderBasic() copyOfRecords1.addAll(copyOfRecords2); Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toSet()); - assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, - readKeys); + .collect(Collectors.toSet()); + assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, readKeys); } @Test @@ -585,9 +548,8 @@ public void testAvroLogRecordReaderWithRollbackTombstone() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); @@ -612,34 +574,27 @@ public void testAvroLogRecordReaderWithRollbackTombstone() // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "102", 10240L, readBlocksLazily, false); - assertEquals("We read 200 records from 2 write batches", 200, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "102", 10240L, readBlocksLazily, false); + assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); copyOfRecords1.addAll(copyOfRecords3); Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toSet()); - assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, - readKeys); + .collect(Collectors.toSet()); + assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, readKeys); } @Test @@ -653,9 +608,8 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -690,42 +644,35 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); writer = writer.appendBlock(commandBlock); // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "103", 10240L, true, false); - assertEquals("We would read 200 records", 200, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "103", 10240L, true, false); + assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); copyOfRecords1.addAll(copyOfRecords3); Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toSet()); - assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, - readKeys); + .collect(Collectors.toSet()); + assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, readKeys); } @Test @@ -739,9 +686,8 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -751,44 +697,35 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() // Write 2 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); copyOfRecords1.addAll(copyOfRecords2); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "102", 10240L, readBlocksLazily, false); - assertEquals("We still would read 200 records", 200, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "102", 10240L, readBlocksLazily, false); + assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 150, readKeys.size()); originalKeys.removeAll(deletedKeys); Collections.sort(originalKeys); Collections.sort(readKeys); - assertEquals("CompositeAvroLogReader should return 150 records from 2 versions", originalKeys, - readKeys); + assertEquals("CompositeAvroLogReader should return 150 records from 2 versions", originalKeys, readKeys); // Rollback the last block header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); @@ -802,8 +739,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback() scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, false); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals("Stream collect should return all 200 records after rollback of delete", 200, - readKeys.size()); + assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); } @Test @@ -819,8 +755,8 @@ public void testAvroLogRecordReaderWithFailedRollbacks() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); @@ -835,15 +771,12 @@ public void testAvroLogRecordReaderWithFailedRollbacks() dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); // Attempt 1 : Write rollback block for a failed write @@ -860,14 +793,12 @@ public void testAvroLogRecordReaderWithFailedRollbacks() // Attempt 2 : Write another rollback blocks for a failed write writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); // all data must be rolled back before merge - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L, readBlocksLazily, false); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(); @@ -888,9 +819,8 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); @@ -898,15 +828,12 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write @@ -916,15 +843,12 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback() writer = writer.appendBlock(commandBlock); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L, readBlocksLazily, false); - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @Test @@ -951,15 +875,12 @@ public void testAvroLogRecordReaderWithInvalidRollback() HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L, readBlocksLazily, false); - assertEquals("We still would read 100 records", 100, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); + assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 100, readKeys.size()); @@ -978,9 +899,8 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); @@ -990,15 +910,12 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); // Write 1 rollback block for a failed write @@ -1008,15 +925,12 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback() HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101", 10240L, readBlocksLazily, false); - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "101", 10240L, readBlocksLazily, false); + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @Test @@ -1069,8 +983,8 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() outputStream.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); writer = writer.appendBlock(dataBlock); writer.close(); @@ -1089,8 +1003,8 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() outputStream.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); // Write 1 rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, @@ -1098,15 +1012,12 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101", 10240L, readBlocksLazily, false); - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "101", 10240L, readBlocksLazily, false); + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @Test @@ -1133,7 +1044,7 @@ public void testMagicAndLogVersionsBackwardsCompatibility() records = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, new String(HoodieAvroUtils.compress(schema.toString()))); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema); - byte [] content = dataBlock.getBytes(schema); + byte[] content = dataBlock.getBytes(schema); outputStream.writeInt(content.length); // Write out some content outputStream.write(content); @@ -1142,8 +1053,8 @@ public void testMagicAndLogVersionsBackwardsCompatibility() outputStream.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); // Write 2 with MAGIC and latest log format version records = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -1153,16 +1064,15 @@ public void testMagicAndLogVersionsBackwardsCompatibility() // Write 3 with MAGIC and latest log format version writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), schema); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), schema); // Read the first block written with latest version and magic reader.hasNext(); @@ -1189,16 +1099,14 @@ public void testMagicAndLogVersionsBackwardsCompatibility() @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndReadInReverse() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndReadInReverse() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -1207,54 +1115,51 @@ public void testBasicAppendAndReadInReverse() writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily, - true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + readBlocksLazily, true); assertTrue("Last block should be available", reader.hasPrev()); HoodieLogBlock prevBlock = reader.prev(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Third records size should be equal to the written records size", - copyOfRecords3.size(), dataBlockRead.getRecords().size()); + assertEquals("Third records size should be equal to the written records size", copyOfRecords3.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, dataBlockRead.getRecords()); assertTrue("Second block should be available", reader.hasPrev()); prevBlock = reader.prev(); dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords2.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords2.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, dataBlockRead.getRecords()); assertTrue("First block should be available", reader.hasPrev()); prevBlock = reader.prev(); dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords1.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); @@ -1262,8 +1167,7 @@ public void testBasicAppendAndReadInReverse() } @Test - public void testAppendAndReadOnCorruptedLogInReverse() - throws IOException, URISyntaxException, InterruptedException { + public void testAppendAndReadOnCorruptedLogInReverse() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); @@ -1295,16 +1199,15 @@ public void testAppendAndReadOnCorruptedLogInReverse() // Should be able to append a new block writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); // First round of reads - we should be able to read the first block and then EOF - HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true); assertTrue("Last block should be available", reader.hasPrev()); HoodieLogBlock block = reader.prev(); @@ -1313,7 +1216,7 @@ public void testAppendAndReadOnCorruptedLogInReverse() assertTrue("Last block should be available", reader.hasPrev()); try { reader.prev(); - } catch(CorruptedLogFileException e) { + } catch (CorruptedLogFileException e) { e.printStackTrace(); // We should have corrupted block } @@ -1321,16 +1224,14 @@ public void testAppendAndReadOnCorruptedLogInReverse() @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndTraverseInReverse() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndTraverseInReverse() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -1339,31 +1240,28 @@ public void testBasicAppendAndTraverseInReverse() writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily, - true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + readBlocksLazily, true); assertTrue("Third block should be available", reader.hasPrev()); reader.moveToPrev(); @@ -1375,8 +1273,8 @@ public void testBasicAppendAndTraverseInReverse() assertTrue("First block should be available", reader.hasPrev()); HoodieLogBlock prevBlock = reader.prev(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords1.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 096c75d779320..25a6a00f486bd 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -54,25 +54,16 @@ public void tearDown() throws Exception { @Test public void testLoadingInstantsFromFiles() throws IOException { - HoodieInstant instant1 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant3 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant4 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8"); - HoodieInstant instant1_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant3_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant4_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8"); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant4 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8"); + HoodieInstant instant1Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant4Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8"); - HoodieInstant instant5 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); + HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); timeline = new HoodieActiveTimeline(metaClient); timeline.saveAsComplete(instant1, Optional.empty()); @@ -83,14 +74,14 @@ public void testLoadingInstantsFromFiles() throws IOException { timeline = timeline.reload(); assertEquals("Total instants should be 5", 5, timeline.countInstants()); - HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream - .of(instant1_complete, instant2_complete, instant3_complete, instant4_complete, - instant5), timeline.getInstants()); - HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream - .of(instant1_complete, instant2_complete, instant3_complete, instant4_complete, - instant5), timeline.getCommitTimeline().getInstants()); HoodieTestUtils.assertStreamEquals("Check the instants stream", - Stream.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete), + Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5), + timeline.getInstants()); + HoodieTestUtils.assertStreamEquals("Check the instants stream", + Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5), + timeline.getCommitTimeline().getInstants()); + HoodieTestUtils.assertStreamEquals("Check the instants stream", + Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete), timeline.getCommitTimeline().filterCompletedInstants().getInstants()); HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5), timeline.getCommitTimeline().filterInflights().getInstants()); @@ -105,21 +96,19 @@ public void testTimelineOperationsBasic() throws Exception { assertEquals("", Optional.empty(), timeline.nthInstant(5)); assertEquals("", Optional.empty(), timeline.nthInstant(-1)); assertEquals("", Optional.empty(), timeline.lastInstant()); - assertFalse("", timeline.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01"))); + assertFalse("", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01"))); } @Test public void testTimelineOperations() throws Exception { - timeline = new MockHoodieTimeline( - Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"), + timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"), Stream.of("21", "23")); HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"), - timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11") - .getInstants().map(HoodieInstant::getTimestamp)); + timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11").getInstants() + .map(HoodieInstant::getTimestamp)); HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"), - timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2) - .getInstants().map(HoodieInstant::getTimestamp)); + timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2).getInstants() + .map(HoodieInstant::getTimestamp)); assertFalse(timeline.empty()); assertFalse(timeline.getCommitTimeline().filterInflights().empty()); assertEquals("", 12, timeline.countInstants()); @@ -130,8 +119,7 @@ public void testTimelineOperations() throws Exception { assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp()); assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp()); assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp()); - assertTrue("", activeCommitTimeline.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09"))); + assertTrue("", activeCommitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09"))); assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02")); assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00")); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java index 5e3b4884462fa..83360e416b951 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java @@ -27,13 +27,10 @@ public class MockHoodieTimeline extends HoodieActiveTimeline { - public MockHoodieTimeline(Stream completed, Stream inflights) - throws IOException { + public MockHoodieTimeline(Stream completed, Stream inflights) throws IOException { super(); - this.instants = Stream.concat(completed - .map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)), - inflights.map( - s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))) + this.instants = Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)), + inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))) .sorted(Comparator.comparing(new Function() { @Override public String apply(HoodieInstant hoodieInstant) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index fab8e371459f4..f26f6d3cd8d74 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -69,8 +69,7 @@ private void refreshFsView(FileStatus[] statuses) { metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); if (statuses != null) { fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), - statuses); + metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), statuses); } else { fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); @@ -86,8 +85,8 @@ public void testGetLatestDataFilesForFileId() throws IOException { String fileId = UUID.randomUUID().toString(); assertFalse("No commit, should not find any data file", - roView.getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent()); + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst() + .isPresent()); // Only one commit, but is not safe String commitTime1 = "1"; @@ -95,43 +94,34 @@ public void testGetLatestDataFilesForFileId() throws IOException { new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); refreshFsView(null); assertFalse("No commit, should not find any data file", - roView.getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().isPresent()); + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst() + .isPresent()); // Make this commit safe HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); commitTimeline.saveAsComplete(instant1, Optional.empty()); refreshFsView(null); - assertEquals("", fileName1, roView - .getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().get() - .getFileName()); + assertEquals("", fileName1, + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get() + .getFileName()); // Do another commit, but not safe String commitTime2 = "2"; String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); refreshFsView(null); - assertEquals("", fileName1, roView - .getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().get() - .getFileName()); + assertEquals("", fileName1, + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get() + .getFileName()); // Make it safe - HoodieInstant instant2 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); commitTimeline.saveAsComplete(instant2, Optional.empty()); refreshFsView(null); - assertEquals("", fileName2, roView - .getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().get() - .getFileName()); + assertEquals("", fileName2, + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get() + .getFileName()); } @Test @@ -148,31 +138,20 @@ public void testStreamLatestVersionInPartition() throws IOException { String fileId3 = UUID.randomUUID().toString(); String fileId4 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)) .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); @@ -188,16 +167,15 @@ public void testStreamLatestVersionInPartition() throws IOException { // Check files as of lastest commit. List allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList()); assertEquals(8, allSlices.size()); - Map fileSliceMap = allSlices.stream().collect(Collectors.groupingBy( - slice -> slice.getFileId(), Collectors.counting())); + Map fileSliceMap = allSlices.stream().collect( + Collectors.groupingBy(slice -> slice.getFileId(), Collectors.counting())); assertEquals(2, fileSliceMap.get(fileId1).longValue()); assertEquals(3, fileSliceMap.get(fileId2).longValue()); assertEquals(2, fileSliceMap.get(fileId3).longValue()); assertEquals(1, fileSliceMap.get(fileId4).longValue()); - List dataFileList = - roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4) - .collect(Collectors.toList()); + List dataFileList = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4) + .collect(Collectors.toList()); assertEquals(3, dataFileList.size()); Set filenames = Sets.newHashSet(); for (HoodieDataFile status : dataFileList) { @@ -208,28 +186,21 @@ public void testStreamLatestVersionInPartition() throws IOException { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3))); filenames = Sets.newHashSet(); - List logFilesList = - rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4) - .map(slice -> slice.getLogFiles()) - .flatMap(logFileList -> logFileList) - .collect(Collectors.toList()); + List logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4) + .map(slice -> slice.getLogFiles()).flatMap(logFileList -> logFileList) + .collect(Collectors.toList()); assertEquals(logFilesList.size(), 4); for (HoodieLogFile logFile : logFilesList) { filenames.add(logFile.getFileName()); } - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))); - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))); - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); // Reset the max commit time - List dataFiles = - roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3) - .collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3) + .collect(Collectors.toList()); assertEquals(dataFiles.size(), 3); filenames = Sets.newHashSet(); for (HoodieDataFile status : dataFiles) { @@ -239,10 +210,8 @@ public void testStreamLatestVersionInPartition() throws IOException { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3))); - logFilesList = - rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3) - .map(slice -> slice.getLogFiles()) - .flatMap(logFileList -> logFileList).collect(Collectors.toList()); + logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles()) + .flatMap(logFileList -> logFileList).collect(Collectors.toList()); assertEquals(logFilesList.size(), 1); assertTrue(logFilesList.get(0).getFileName() .equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))); @@ -261,20 +230,13 @@ public void testStreamEveryVersionInPartition() throws IOException { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -286,8 +248,7 @@ public void testStreamEveryVersionInPartition() throws IOException { assertEquals(7, statuses.length); refreshFsView(null); - List fileGroups = - fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList()); assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { @@ -298,18 +259,14 @@ public void testStreamEveryVersionInPartition() throws IOException { filenames.add(dataFile.getFileName()); }); if (fileId.equals(fileId1)) { - assertEquals(filenames, - Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1), - FSUtils.makeDataFileName(commitTime4, 1, fileId1))); + assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1), + FSUtils.makeDataFileName(commitTime4, 1, fileId1))); } else if (fileId.equals(fileId2)) { - assertEquals(filenames, - Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2), - FSUtils.makeDataFileName(commitTime2, 1, fileId2), - FSUtils.makeDataFileName(commitTime3, 1, fileId2))); + assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2), + FSUtils.makeDataFileName(commitTime2, 1, fileId2), FSUtils.makeDataFileName(commitTime3, 1, fileId2))); } else { - assertEquals(filenames, - Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3), - FSUtils.makeDataFileName(commitTime4, 1, fileId3))); + assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3), + FSUtils.makeDataFileName(commitTime4, 1, fileId3))); } } } @@ -327,28 +284,19 @@ public void streamLatestVersionInRange() throws IOException { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -360,8 +308,7 @@ public void streamLatestVersionInRange() throws IOException { assertEquals(9, statuses.length); refreshFsView(statuses); - List dataFiles = roView - .getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3)) + List dataFiles = roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3)) .collect(Collectors.toList()); assertEquals(3, dataFiles.size()); Set filenames = Sets.newHashSet(); @@ -371,8 +318,7 @@ public void streamLatestVersionInRange() throws IOException { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3))); - List slices = rtView - .getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4)) + List slices = rtView.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4)) .collect(Collectors.toList()); assertEquals(3, slices.size()); for (FileSlice slice : slices) { @@ -406,20 +352,13 @@ public void streamLatestVersionsBefore() throws IOException { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -431,9 +370,8 @@ public void streamLatestVersionsBefore() throws IOException { assertEquals(7, statuses.length); refreshFsView(null); - List dataFiles = - roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2) - .collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2) + .collect(Collectors.toList()); assertEquals(2, dataFiles.size()); Set filenames = Sets.newHashSet(); for (HoodieDataFile status : dataFiles) { @@ -457,31 +395,21 @@ public void streamLatestVersions() throws IOException { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0)) .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -494,9 +422,7 @@ public void streamLatestVersions() throws IOException { refreshFsView(statuses); - List fileGroups = fsView - .getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { List slices = fileGroup.getAllFileSlices().collect(Collectors.toList()); @@ -516,8 +442,7 @@ public void streamLatestVersions() throws IOException { } } - List statuses1 = - roView.getLatestDataFiles().collect(Collectors.toList()); + List statuses1 = roView.getLatestDataFiles().collect(Collectors.toList()); assertEquals(3, statuses1.size()); Set filenames = Sets.newHashSet(); for (HoodieDataFile status : statuses1) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java index a736da8e5f40a..02c8933cbefd2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java @@ -20,10 +20,8 @@ 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.exception.HoodieIOException; - import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -40,9 +38,7 @@ import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -51,19 +47,16 @@ public class SchemaTestUtil { public static Schema getSimpleSchema() throws IOException { - return new Schema.Parser() - .parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro")); } - public static List generateTestRecords(int from, int limit) - throws IOException, URISyntaxException { + public static List generateTestRecords(int from, int limit) throws IOException, URISyntaxException { return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit); } - private static List toRecords(Schema writerSchema, Schema readerSchema, int from, - int limit) throws IOException, URISyntaxException { - GenericDatumReader reader = - new GenericDatumReader<>(writerSchema, readerSchema); + private static List toRecords(Schema writerSchema, Schema readerSchema, int from, int limit) + throws IOException, URISyntaxException { + GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema); // Required to register the necessary JAR:// file system URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI(); Path dataPath; @@ -103,15 +96,12 @@ public static List generateHoodieTestRecords(int from, int limit) List records = generateTestRecords(from, limit); String commitTime = HoodieActiveTimeline.createNewCommitTime(); Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); - return records.stream() - .map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)) - .map(p -> { - p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString()); - p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); - p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); - return p; - }).collect( - Collectors.toList()); + return records.stream().map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)).map(p -> { + p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString()); + p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); + p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); + return p; + }).collect(Collectors.toList()); } @@ -131,8 +121,7 @@ private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String } public static List updateHoodieTestRecords(List oldRecordKeys, List newRecords, - String commitTime) - throws IOException, URISyntaxException { + String commitTime) throws IOException, URISyntaxException { return newRecords.stream() .map(p -> { @@ -155,8 +144,9 @@ public static List generateHoodieTestRecordsWithoutHoodieMetadata( new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList()); } - public static List updateHoodieTestRecordsWithoutHoodieMetadata(List oldRecords, Schema schema, - String fieldNameToUpdate, String newValue) + public static List updateHoodieTestRecordsWithoutHoodieMetadata(List oldRecords, + Schema schema, + String fieldNameToUpdate, String newValue) throws IOException, URISyntaxException { return oldRecords .stream() @@ -173,8 +163,7 @@ public static List updateHoodieTestRecordsWithoutHoodieMetadata(Li } public static Schema getEvolvedSchema() throws IOException { - return new Schema.Parser() - .parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro")); } public static List generateEvolvedTestRecords(int from, int limit) @@ -183,12 +172,11 @@ public static List generateEvolvedTestRecords(int from, int limit } public static Schema getComplexEvolvedSchema() throws IOException { - return new Schema.Parser() - .parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro")); } - public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, - String commitTime, String fileId) throws IOException { + public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, String commitTime, + String fileId) throws IOException { TestRecord record = new TestRecord(commitTime, recordNumber, fileId); MercifulJsonConverter converter = new MercifulJsonConverter(schema); return converter.convert(record.toJsonString()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java index f8360376ef644..42549afcf56fa 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java @@ -20,18 +20,17 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; - import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; public class SpillableMapTestUtils { public static List upsertRecords(List iRecords, - Map> records) { + Map> records) { List recordKeys = new ArrayList<>(); iRecords .stream() diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java index 3e05158b2c8b5..fff22c7e1af6a 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java @@ -31,8 +31,7 @@ public class TestFSUtils { @Rule - public final EnvironmentVariables environmentVariables - = new EnvironmentVariables(); + public final EnvironmentVariables environmentVariables = new EnvironmentVariables(); @Test public void testMakeDataFileName() { @@ -51,8 +50,10 @@ public void testMakeTempDataFileName() { int stageId = Integer.MAX_VALUE; long taskAttemptId = Long.MAX_VALUE; String fileName = UUID.randomUUID().toString(); - assertTrue(FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId) - .equals(partitionPath.replace("/", "-") + "_" + fileName + "_" + taskPartitionId + "_" + commitTime + "_" + stageId + "_" + taskAttemptId + ".parquet")); + assertTrue( + FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId) + .equals(partitionPath.replace("/", "-") + "_" + fileName + "_" + taskPartitionId + "_" + commitTime + "_" + + stageId + "_" + taskAttemptId + ".parquet")); } @Test diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java index fd0a4475a13b1..636df99e3d602 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java @@ -63,13 +63,12 @@ public void testHoodieWriteSupport() throws Exception { // Write out a parquet file Schema schema = HoodieAvroUtils.getRecordKeySchema(); BloomFilter filter = new BloomFilter(1000, 0.0001); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(schema), schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, + filter); String filePath = basePath + "/test.parquet"; - ParquetWriter writer = new ParquetWriter(new Path(filePath), - writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, - ParquetWriter.DEFAULT_PAGE_SIZE); + ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, + 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); for (String rowKey : rowKeys) { GenericRecord rec = new GenericData.Record(schema); rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey); @@ -80,15 +79,13 @@ public void testHoodieWriteSupport() throws Exception { // Read and verify List rowKeysInFile = new ArrayList<>( - ParquetUtils - .readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); + ParquetUtils.readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); Collections.sort(rowKeysInFile); Collections.sort(rowKeys); assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile); - BloomFilter filterInFile = ParquetUtils - .readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), - new Path(filePath)); + BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), + new Path(filePath)); for (String rowKey : rowKeys) { assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index 5eee986186fbf..ee2c77b5ee41b 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -20,18 +20,18 @@ 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.model.AvroBinaryTestPayload; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import java.io.IOException; import java.io.UncheckedIOException; import java.net.URISyntaxException; @@ -46,6 +46,7 @@ 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 { @@ -58,8 +59,7 @@ public void testSimpleInsert() throws IOException, URISyntaxException { DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - String commitTime = ((GenericRecord) iRecords.get(0)) - .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); // make sure records have spilled to disk @@ -108,8 +108,7 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - String commitTime = ((GenericRecord) iRecords.get(0)) - .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + // perform some inserts List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -125,9 +124,6 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { String newCommitTime = ((GenericRecord) updatedRecords.get(0)) .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); - // new commit time should be different - assertEquals(commitTime, newCommitTime); - // perform upserts recordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, records); @@ -194,6 +190,12 @@ public void testSizeEstimator() throws IOException, URISyntaxException { assertTrue(payloadSize > 0); } + /** + * vb - Disabled this test after talking to Nishanth as this relies on timing and sometimes fails in my laptop. + * This specific test sometime takes more than 100 ms (In one case, saw 122 ms), + * @na: TODO: Please check if this can be removed. + */ + @Ignore @Test public void testSizeEstimatorPerformance() throws IOException, URISyntaxException { // Test sizeEstimatorPerformance with simpleSchema @@ -205,6 +207,7 @@ public void testSizeEstimatorPerformance() throws IOException, URISyntaxExceptio long startTime = System.currentTimeMillis(); SpillableMapUtils.computePayloadSize(record, converter); long timeTaken = System.currentTimeMillis() - startTime; + System.out.println("Time taken :" + timeTaken); assertTrue(timeTaken < 100); } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index 1c56b6cb85e9f..c62aa1cc45ced 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -28,8 +28,8 @@ import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -55,9 +55,8 @@ public void simpleInsertTest() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.empty(), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.empty(), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -78,9 +77,8 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -90,9 +88,8 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { HoodieRecord rec = itr.next(); assert recordKeys.contains(rec.getRecordKey()); } - List updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100), - HoodieActiveTimeline.createNewCommitTime()); + List updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys, + SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime()); // update records already inserted SpillableMapTestUtils.upsertRecords(updatedRecords, records); @@ -118,9 +115,8 @@ public void testAllMapOperations() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.empty(), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.empty(), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // insert a bunch of records so that values spill to disk too @@ -174,9 +170,8 @@ public void simpleTestWithException() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -200,9 +195,8 @@ public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOExcept String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk @@ -221,15 +215,14 @@ public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOExcept List keysToBeUpdated = new ArrayList<>(); keysToBeUpdated.add(key); // Update the commitTime for this record - List updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); + List updatedRecords = SchemaTestUtil + .updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); // Upsert this updated record SpillableMapTestUtils.upsertRecords(updatedRecords, records); GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); // The record returned for this key should have the updated commitTime assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); - // Get a record from the disk based map key = recordKeys.get(recordKeys.size() - 1); record = records.get(key); @@ -240,8 +233,7 @@ record = records.get(key); keysToBeUpdated = new ArrayList<>(); keysToBeUpdated.add(key); // Update the commitTime for this record - updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); + updatedRecords = SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); // Upsert this updated record SpillableMapTestUtils.upsertRecords(updatedRecords, records); gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); @@ -256,9 +248,8 @@ public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISy String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java index 12b4abf40d194..70acdf73b124f 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java @@ -23,8 +23,7 @@ public class HoodieHiveUtil { - public static final Logger LOG = - LogManager.getLogger(HoodieHiveUtil.class); + public static final Logger LOG = LogManager.getLogger(HoodieHiveUtil.class); public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode"; public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp"; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index 302373e1beb92..84d7da39c1bb7 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -71,8 +71,7 @@ * Hoodie/Non-Hoodie datasets */ @UseFileSplitsFromInputFormat -public class HoodieInputFormat extends MapredParquetInputFormat - implements Configurable { +public class HoodieInputFormat extends MapredParquetInputFormat implements Configurable { public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class); @@ -99,7 +98,8 @@ public FileStatus[] listStatus(JobConf job) throws IOException { } String tableName = metadata.getTableConfig().getTableName(); String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName); - // Get all commits, delta commits, compactions, as all of them produce a base parquet file today + // Get all commits, delta commits, compactions, as all of them produce a base parquet file + // today HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata, @@ -112,19 +112,16 @@ public FileStatus[] listStatus(JobConf job) throws IOException { // Total number of commits to return in this batch. Set this to -1 to get all the commits. Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName); LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs); - List commitsToReturn = - timeline.findInstantsAfter(lastIncrementalTs, maxCommits).getInstants() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - List filteredFiles = roView - .getLatestDataFilesInRange(commitsToReturn) + List commitsToReturn = timeline.findInstantsAfter(lastIncrementalTs, maxCommits) + .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List filteredFiles = roView.getLatestDataFilesInRange(commitsToReturn) .collect(Collectors.toList()); for (HoodieDataFile filteredFile : filteredFiles) { LOG.info("Processing incremental hoodie file - " + filteredFile.getPath()); filteredFile = checkFileStatus(filteredFile); returns.add(filteredFile.getFileStatus()); } - LOG.info( - "Total paths to process after hoodie incremental filter " + filteredFiles.size()); + LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size()); } else { // filter files on the latest commit found List filteredFiles = roView.getLatestDataFiles() @@ -171,13 +168,13 @@ private Map> groupFileStatus(FileStatus[ String nonHoodieBasePath = null; for (FileStatus status : fileStatuses) { if (!status.getPath().getName().endsWith(".parquet")) { - //FIXME(vc): skip non parquet files for now. This wont be needed once log file name start with "." + //FIXME(vc): skip non parquet files for now. This wont be needed once log file name start + // with "." continue; } if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath() - .toString() - .contains(nonHoodieBasePath)) || (metadata != null && !status.getPath().toString() - .contains(metadata.getBasePath()))) { + .toString().contains(nonHoodieBasePath)) || (metadata != null && !status.getPath() + .toString().contains(metadata.getBasePath()))) { try { metadata = getTableMetaClient(status.getPath().getFileSystem(conf), status.getPath().getParent()); @@ -185,8 +182,7 @@ private Map> groupFileStatus(FileStatus[ } catch (InvalidDatasetException e) { LOG.info("Handling a non-hoodie path " + status.getPath()); metadata = null; - nonHoodieBasePath = - status.getPath().getParent().toString(); + nonHoodieBasePath = status.getPath().getParent().toString(); } if (!grouped.containsKey(metadata)) { grouped.put(metadata, new ArrayList<>()); @@ -209,17 +205,17 @@ public Configuration getConf() { public RecordReader getRecordReader(final InputSplit split, final JobConf job, final Reporter reporter) throws IOException { // TODO enable automatic predicate pushdown after fixing issues -// FileSplit fileSplit = (FileSplit) split; -// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent()); -// String tableName = metadata.getTableName(); -// String mode = HoodieHiveUtil.readMode(job, tableName); + // FileSplit fileSplit = (FileSplit) split; + // HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent()); + // String tableName = metadata.getTableName(); + // String mode = HoodieHiveUtil.readMode(job, tableName); -// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) { -// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split); -// LOG.info("Setting parquet predicate push down as " + predicate); -// ParquetInputFormat.setFilterPredicate(job, predicate); + // if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) { + // FilterPredicate predicate = constructHoodiePredicate(job, tableName, split); + // LOG.info("Setting parquet predicate push down as " + predicate); + // ParquetInputFormat.setFilterPredicate(job, predicate); //clearOutExistingPredicate(job); -// } + // } return super.getRecordReader(split, job, reporter); } @@ -236,9 +232,8 @@ private void clearOutExistingPredicate(JobConf job) { * `hoodie_commit_time` > 'start_commit_time' and ANDs with the existing predicate if one is * present already. */ - private FilterPredicate constructHoodiePredicate(JobConf job, - String tableName, - InputSplit split) throws IOException { + private FilterPredicate constructHoodiePredicate(JobConf job, String tableName, InputSplit split) + throws IOException { FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName); LOG.info("Commit time predicate - " + commitTimePushdown.toString()); FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split); @@ -262,21 +257,19 @@ private FilterPredicate constructHQLPushdownPredicate(JobConf job, InputSplit sp || columnNamesString.isEmpty()) { return null; } else { - SearchArgument sarg = - SearchArgumentFactory.create(Utilities.deserializeExpression(serializedPushdown)); + SearchArgument sarg = SearchArgumentFactory + .create(Utilities.deserializeExpression(serializedPushdown)); final Path finalPath = ((FileSplit) split).getPath(); final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(job, finalPath); final FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); - return ParquetFilterPredicateConverter - .toFilterPredicate(sarg, fileMetaData.getSchema()); + return ParquetFilterPredicateConverter.toFilterPredicate(sarg, fileMetaData.getSchema()); } } private FilterPredicate constructCommitTimePushdownPredicate(JobConf job, String tableName) throws IOException { String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName); - Operators.BinaryColumn sequenceColumn = - binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD); + Operators.BinaryColumn sequenceColumn = binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD); FilterPredicate p = gt(sequenceColumn, Binary.fromString(lastIncrementalTs)); LOG.info("Setting predicate in InputFormat " + p.toString()); return p; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index 2c01ed666a220..44672ec4b6452 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.hadoop; import com.uber.hoodie.common.model.HoodieDataFile; @@ -37,12 +38,12 @@ /** * Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path - * Non-Hoodie dataset = then always accept - * + *

* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then * you should be able to query both hoodie and non-hoodie datasets as you would normally do. - * - * hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop.HoodieROTablePathFilter.class, - * org.apache.hadoop.fs.PathFilter.class) + *

+ * hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop + * .HoodieROTablePathFilter.class, org.apache.hadoop.fs.PathFilter.class) */ public class HoodieROTablePathFilter implements PathFilter, Serializable { @@ -104,8 +105,7 @@ public boolean accept(Path path) { if (hoodiePathCache.containsKey(folder.toString())) { if (LOG.isDebugEnabled()) { - LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n", - path, + LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n", path, hoodiePathCache.get(folder.toString()).contains(path))); } return hoodiePathCache.get(folder.toString()).contains(path); @@ -123,37 +123,33 @@ public boolean accept(Path path) { if (baseDir != null) { try { - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), + baseDir.toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline() - .filterCompletedInstants(), + metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), fs.listStatus(folder)); - List latestFiles = fsView - .getLatestDataFiles() + List latestFiles = fsView.getLatestDataFiles() .collect(Collectors.toList()); // populate the cache if (!hoodiePathCache.containsKey(folder.toString())) { hoodiePathCache.put(folder.toString(), new HashSet<>()); } - LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + - ", caching " + latestFiles.size() + " files under " + folder); + LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching " + + latestFiles.size() + " files under " + folder); for (HoodieDataFile lfile : latestFiles) { hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath())); } // accept the path, if its among the latest files. if (LOG.isDebugEnabled()) { - LOG.debug(String.format("%s checked after cache population, accept => %s \n", - path, + LOG.debug(String.format("%s checked after cache population, accept => %s \n", path, hoodiePathCache.get(folder.toString()).contains(path))); } return hoodiePathCache.get(folder.toString()).contains(path); } catch (DatasetNotFoundException e) { // Non-hoodie path, accept it. if (LOG.isDebugEnabled()) { - LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", - folder.toString())); + LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString())); } nonHoodiePathCache.add(folder.toString()); return true; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index 939f582976ec3..a6f12b8f9d996 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -79,8 +79,7 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { // TODO(vc): Should we handle also non-hoodie splits here? Map metaClientMap = new HashMap<>(); Map partitionsToMetaClient = partitionsToParquetSplits.keySet() - .stream() - .collect(Collectors.toMap(Function.identity(), p -> { + .stream().collect(Collectors.toMap(Function.identity(), p -> { // find if we have a metaclient already for this partition. Optional matchingBasePath = metaClientMap.keySet().stream() .filter(basePath -> p.toString().startsWith(basePath)).findFirst(); @@ -97,7 +96,8 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { } })); - // for all unique split parents, obtain all delta files based on delta commit timeline, grouped on file id + // for all unique split parents, obtain all delta files based on delta commit timeline, + // grouped on file id List rtSplits = new ArrayList<>(); partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> { // for each partition path obtain the data & log file groupings, then map back to inputsplits @@ -119,14 +119,13 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { dataFileSplits.forEach(split -> { try { List logFilePaths = fileSlice.getLogFiles() - .map(logFile -> logFile.getPath().toString()) - .collect(Collectors.toList()); - // Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table - String maxCommitTime = metaClient.getActiveTimeline() - .getTimelineOfActions( - Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, - HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()); + // Get the maxCommit from the last delta or compaction or commit - when + // bootstrapped from COW table + String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions( + Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, + HoodieTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants().lastInstant() + .get().getTimestamp(); rtSplits.add( new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime)); @@ -147,7 +146,8 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { @Override public FileStatus[] listStatus(JobConf job) throws IOException { - // Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit timeline. + // Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit + // timeline. return super.listStatus(job); } @@ -170,12 +170,11 @@ private static Configuration addProjectionField(Configuration conf, String field if (!readColNames.contains(fieldName)) { // If not already in the list - then add it - conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, - readColNamesPrefix + fieldName); + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName); conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex); if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Adding extra column " + fieldName - + ", to enable log merging cols (%s) ids (%s) ", + LOG.debug(String.format( + "Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ", conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR))); } @@ -189,15 +188,14 @@ private static Configuration addRequiredProjectionFields(Configuration configura HOODIE_RECORD_KEY_COL_POS); configuration = addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HOODIE_COMMIT_TIME_COL_POS); - configuration = addProjectionField(configuration, - HoodieRecord.PARTITION_PATH_METADATA_FIELD, HOODIE_PARTITION_PATH_COL_POS); + configuration = addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD, + HOODIE_PARTITION_PATH_COL_POS); return configuration; } @Override public RecordReader getRecordReader(final InputSplit split, - final JobConf job, - final Reporter reporter) throws IOException { + final JobConf job, final Reporter reporter) throws IOException { LOG.info("Creating record reader with readCols :" + job .get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); // sanity check diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 81cae359ba30e..357023b4bded0 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -73,8 +73,10 @@ public class HoodieRealtimeRecordReader implements RecordReader deltaRecordMap; private final MessageType baseFileSchema; - public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, - JobConf job, + public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job, RecordReader realReader) { this.split = split; this.jobConf = job; @@ -106,11 +107,9 @@ public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, */ private static MessageType readSchema(Configuration conf, Path parquetFilePath) { try { - return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData() - .getSchema(); + return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema(); } catch (IOException e) { - throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, - e); + throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); } } @@ -125,27 +124,27 @@ private void readAndCompactLog(JobConf jobConf) throws IOException { jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), jobConf.get("partition_columns", "")); - // TODO(vc): In the future, the reader schema should be updated based on log files & be able to null out fields not present before + // TODO(vc): In the future, the reader schema should be updated based on log files & be able + // to null out fields not present before Schema readerSchema = generateProjectionSchema(writerSchema, projectionFields); - LOG.info( - String.format("About to read compacted logs %s for base split %s, projecting cols %s", - split.getDeltaFilePaths(), split.getPath(), projectionFields)); - HoodieCompactedLogRecordScanner compactedLogRecordScanner = - new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), - split.getBasePath(), - split.getDeltaFilePaths(), - readerSchema, split.getMaxCommitTime(), - (long) Math.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) - *jobConf.getMemoryForMapTask()), - Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), false); + LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s", + split.getDeltaFilePaths(), split.getPath(), projectionFields)); + HoodieCompactedLogRecordScanner compactedLogRecordScanner = new HoodieCompactedLogRecordScanner( + FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), + split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime(), (long) Math.ceil(Double + .valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) + * jobConf.getMemoryForMapTask()), Boolean.valueOf(jobConf + .get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), + false); // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit - // but can return records for completed commits > the commit we are trying to read (if using readCommit() API) + // but can return records for completed commits > the commit we are trying to read (if using + // readCommit() API) for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { - GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema) - .get(); + GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema).get(); String key = hoodieRecord.getRecordKey(); - // we assume, a later safe record in the log, is newer than what we have in the map & replace it. + // we assume, a later safe record in the log, is newer than what we have in the map & + // replace it. // TODO : handle deletes here ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema); deltaRecordMap.put(key, aWritable); @@ -180,14 +179,13 @@ public static List orderFields(String fieldNameCsv, String fieldOrderCsv Set partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) .collect(Collectors.toSet()); List fieldNames = Arrays.stream(fieldNameCsv.split(",")) - .filter(fn -> !partitioningFields.contains(fn)).collect( - Collectors.toList()); + .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); // Hive does not provide ids for partitioning fields, so check for lengths excluding that. if (fieldNames.size() != fieldOrders.length) { - throw new HoodieException(String.format( - "Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", - fieldNames.size(), fieldOrders.length)); + throw new HoodieException(String + .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", + fieldNames.size(), fieldOrders.length)); } TreeMap orderedFieldMap = new TreeMap<>(); for (int ox = 0; ox < fieldOrders.length; ox++) { @@ -287,26 +285,28 @@ public static Writable avroToArrayWritable(Object value, Schema schema) { } case FIXED: return new BytesWritable(((GenericFixed) value).bytes()); + default: + return null; } - return null; } @Override public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException { - // Call the underlying parquetReader.next - which may replace the passed in ArrayWritable with a new block of values + // Call the underlying parquetReader.next - which may replace the passed in ArrayWritable + // with a new block of values boolean result = this.parquetReader.next(aVoid, arrayWritable); if (!result) { // if the result is false, then there are no more records return false; } else { - // TODO(VC): Right now, we assume all records in log, have a matching base record. (which would be true until we have a way to index logs too) + // TODO(VC): Right now, we assume all records in log, have a matching base record. (which + // would be true until we have a way to index logs too) // return from delta records map if we have some match. String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS] .toString(); if (LOG.isDebugEnabled()) { - LOG.debug(String.format("key %s, base values: %s, log values: %s", - key, arrayWritableToString(arrayWritable), - arrayWritableToString(deltaRecordMap.get(key)))); + LOG.debug(String.format("key %s, base values: %s, log values: %s", key, + arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); } if (deltaRecordMap.containsKey(key)) { // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/AnnotationTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/AnnotationTest.java index ebe3fd805380f..c08cdc6dd48f2 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/AnnotationTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/AnnotationTest.java @@ -16,7 +16,6 @@ package com.uber.hoodie.hadoop; - import static org.junit.Assert.assertTrue; import java.lang.annotation.Annotation; diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java index 5bedb30230492..779184e0a2caa 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java @@ -82,17 +82,17 @@ public void testInputFormatUpdates() throws IOException { files = inputFormat.listStatus(jobConf); assertEquals(10, files.length); ensureFilesInCommit( - "Commit 200 has not been committed. We should not see files from this commit", files, - "200", 0); + "Commit 200 has not been committed. We should not see files from this commit", files, "200", + 0); InputFormatTestUtil.commit(basePath, "200"); files = inputFormat.listStatus(jobConf); assertEquals(10, files.length); ensureFilesInCommit( - "5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 files from 100 commit", - files, "200", 5); + "5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 " + + "files from 100 commit", files, "200", 5); ensureFilesInCommit( - "5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 files from 200 commit", - files, "100", 5); + "5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 " + + "files from 200 commit", files, "100", 5); } @Test @@ -108,8 +108,8 @@ public void testIncrementalSimple() throws IOException { FileStatus[] files = inputFormat.listStatus(jobConf); assertEquals( - "We should exclude commit 100 when returning incremental pull with start commit time as 100", - 0, files.length); + "We should exclude commit 100 when returning incremental pull with start commit time as " + + "100", 0, files.length); } @Test @@ -146,8 +146,8 @@ public void testIncrementalWithMultipleCommits() throws IOException { files = inputFormat.listStatus(jobConf); assertEquals( - "Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 commit and 1 file from 200 commit", - 5, files.length); + "Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 " + + "commit and 1 file from 200 commit", 5, files.length); ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit", files, "400", 3); ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit", @@ -159,23 +159,18 @@ public void testIncrementalWithMultipleCommits() throws IOException { files = inputFormat.listStatus(jobConf); assertEquals( - "Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 commits", - 5, files.length); - ensureFilesInCommit( - "Pulling all commits from 100, should get us the 1 files from 600 commit", files, "600", - 1); - ensureFilesInCommit( - "Pulling all commits from 100, should get us the 1 files from 500 commit", files, "500", - 1); - ensureFilesInCommit( - "Pulling all commits from 100, should get us the 1 files from 400 commit", files, "400", - 1); - ensureFilesInCommit( - "Pulling all commits from 100, should get us the 1 files from 300 commit", files, "300", - 1); - ensureFilesInCommit( - "Pulling all commits from 100, should get us the 1 files from 200 commit", files, "200", - 1); + "Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 " + + "commits", 5, files.length); + ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit", + files, "600", 1); + ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit", + files, "500", 1); + ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit", + files, "400", 1); + ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit", + files, "300", 1); + ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit", + files, "200", 1); } //TODO enable this after enabling predicate pushdown @@ -183,8 +178,7 @@ public void testPredicatePushDown() throws IOException { // initial commit Schema schema = InputFormatTestUtil.readSchema("/sample1.avro"); String commit1 = "20160628071126"; - File partitionDir = - InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1); + File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1); InputFormatTestUtil.commit(basePath, commit1); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); @@ -212,19 +206,20 @@ public void testPredicatePushDown() throws IOException { commit2, 2, 10); } - private void ensureRecordsInCommit(String msg, String commit, - int expectedNumberOfRecordsInCommit, int totalExpected) throws IOException { + private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit, + int totalExpected) throws IOException { int actualCount = 0; int totalCount = 0; InputSplit[] splits = inputFormat.getSplits(jobConf, 1); for (InputSplit split : splits) { - RecordReader - recordReader = inputFormat.getRecordReader(split, jobConf, null); + RecordReader recordReader = inputFormat + .getRecordReader(split, jobConf, null); Void key = recordReader.createKey(); ArrayWritable writable = recordReader.createValue(); while (recordReader.next(key, writable)) { - // writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno] + // writable returns an array with [field1, field2, _hoodie_commit_time, + // _hoodie_commit_seqno] // Take the commit time and compare with the one we are interested in if (commit.equals((writable.get()[2]).toString())) { actualCount++; diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java index c8305e3a3f4aa..4088afdd05285 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java @@ -42,16 +42,15 @@ public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); for (int i = 0; i < numberOfFiles; i++) { - File dataFile = - new File(partitionPath, FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i)); + File dataFile = new File(partitionPath, + FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i)); dataFile.createNewFile(); } return partitionPath; } public static void simulateUpdates(File directory, final String originalCommit, - int numberOfFilesUpdated, - String newCommit, boolean randomize) throws IOException { + int numberOfFilesUpdated, String newCommit, boolean randomize) throws IOException { List dataFiles = Arrays.asList(directory.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -62,8 +61,8 @@ public boolean accept(File dir, String name) { if (randomize) { Collections.shuffle(dataFiles); } - List toUpdateList = - dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size())); + List toUpdateList = dataFiles + .subList(0, Math.min(numberOfFilesUpdated, dataFiles.size())); for (File file : toUpdateList) { String fileId = FSUtils.getFileId(file.getName()); File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, 1, fileId)); @@ -78,8 +77,8 @@ public static void commit(TemporaryFolder basePath, String commitNumber) throws public static void setupIncremental(JobConf jobConf, String startCommit, int numberOfCommitsToPull) { - String modePropertyName = String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, - HoodieTestUtils.RAW_TRIPS_TEST_NAME); + String modePropertyName = String + .format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME); jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE); String startCommitTimestampName = String @@ -96,19 +95,16 @@ public static Schema readSchema(String location) throws IOException { } public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema, - int numberOfFiles, int numberOfRecords, - String commitNumber) throws IOException { + int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { basePath.create(); HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { String fileId = FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i); - File dataFile = - new File(partitionPath, fileId); + File dataFile = new File(partitionPath, fileId); // dataFile.createNewFile(); - parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), - schema); + parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema); try { for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) { @@ -132,8 +128,7 @@ private static Iterable generateAvroRecords(Schema sche } public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit, - int totalNumberOfRecords, int numberOfRecordsToUpdate, - String newCommit) throws IOException { + int totalNumberOfRecords, int numberOfRecordsToUpdate, String newCommit) throws IOException { File fileToUpdate = directory.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -145,8 +140,8 @@ public boolean accept(File dir, String name) { AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema); try { - for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, - originalCommit, fileId)) { + for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, + fileId)) { if (numberOfRecordsToUpdate > 0) { // update this record record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit); diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java index e64f918d40894..c81f9fccbf98f 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.hadoop; import static org.junit.Assert.assertFalse; diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 653e122d305d1..8e43aa25e056a 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -18,7 +18,6 @@ package com.uber.hoodie.hadoop.realtime; - import static org.junit.Assert.assertTrue; import com.google.common.collect.Maps; @@ -85,9 +84,7 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) - .overBaseCommit(baseCommit) - .withFs(fs) - .build(); + .overBaseCommit(baseCommit).withFs(fs).build(); List records = new ArrayList<>(); for (int i = 0; i < numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); @@ -106,8 +103,8 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St public void testReader() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); - HoodieTestUtils - .initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), + HoodieTableType.MERGE_ON_READ); String commitTime = "100"; File partitionDir = InputFormatTestUtil .prepareParquetDataset(basePath, schema, 1, 100, commitTime); @@ -125,15 +122,15 @@ public void testReader() throws Exception { //create a split with baseFile (parquet file written earlier) and new log file(s) String logFilePath = writer.getLogFile().getPath().toString(); - HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir - + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(), - Arrays.asList(logFilePath), newCommitTime); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, + jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime); //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = - new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), - (String[]) null), jobConf, null); + new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), + jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); @@ -164,8 +161,8 @@ public void testReader() throws Exception { public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema()); - HoodieTestUtils - .initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), + HoodieTableType.MERGE_ON_READ); String commitTime = "100"; int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; @@ -185,15 +182,15 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { //create a split with baseFile (parquet file written earlier) and new log file(s) String logFilePath = writer.getLogFile().getPath().toString(); - HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir - + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(), - Arrays.asList(logFilePath), newCommitTime); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, + jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime); //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = - new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), - (String[]) null), jobConf, null); + new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), + jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java index 5379580ef6052..dd9e701493ba8 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java @@ -48,15 +48,23 @@ public class HiveSyncConfig implements Serializable { "--base-path"}, description = "Basepath of hoodie dataset to sync", required = true) public String basePath; - @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by", required = true) + @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by", + required = true) public List partitionFields = new ArrayList<>(); - @Parameter(names = "-partition-value-extractor", description = "Class which implements PartitionValueExtractor to extract the partition values from HDFS path") + @Parameter(names = "-partition-value-extractor", description = "Class which implements " + + "PartitionValueExtractor " + + "to extract the partition " + + "values from HDFS path") public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class .getName(); @Parameter(names = { - "--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") + "--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" + + " exists to support " + + "backward compatibility. If" + + " you use hoodie 0.3.x, do " + + "not set this parameter") public Boolean assumeDatePartitioning = false; @Parameter(names = {"--help", "-h"}, help = true) diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java index 472001ede9e69..164aa7a6c28c5 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java @@ -45,16 +45,16 @@ * Tool to sync a hoodie HDFS dataset with a hive metastore table. Either use it as a api * HiveSyncTool.syncHoodieTable(HiveSyncConfig) or as a command line java -cp hoodie-hive.jar * HiveSyncTool [args] - * + *

* This utility will get the schema from the latest commit and will sync hive table schema Also this * will sync the partitions incrementally (all the partitions modified since the last commit) */ @SuppressWarnings("WeakerAccess") public class HiveSyncTool { - private static Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class); + private static final Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class); private final HoodieHiveClient hoodieHiveClient; - public final static String SUFFIX_REALTIME_TABLE = "_rt"; + public static final String SUFFIX_REALTIME_TABLE = "_rt"; private final HiveSyncConfig cfg; public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { @@ -85,9 +85,8 @@ public void syncHoodieTable() { } private void syncHoodieTable(boolean isRealTime) { - LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + hoodieHiveClient - .getBasePath() + " of type " + hoodieHiveClient - .getTableType()); + LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + + hoodieHiveClient.getBasePath() + " of type " + hoodieHiveClient.getTableType()); // Check if the necessary table exists boolean tableExists = hoodieHiveClient.doesTableExist(); @@ -118,27 +117,29 @@ private void syncHoodieTable(boolean isRealTime) { * If not, evolves the table schema. * * @param tableExists - does table exist - * @param schema - extracted schema + * @param schema - extracted schema */ private void syncSchema(boolean tableExists, boolean isRealTime, MessageType schema) { // Check and sync schema if (!tableExists) { LOG.info("Table " + cfg.tableName + " is not found. Creating it"); if (!isRealTime) { - // TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default for now) + // TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default + // for now) hoodieHiveClient.createTable(schema, HoodieInputFormat.class.getName(), MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName()); } else { // Custom serde will not work with ALTER TABLE REPLACE COLUMNS - // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java#L3488 + // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive + // /ql/exec/DDLTask.java#L3488 hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(), MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName()); } } else { // Check if the dataset schema has evolved Map tableSchema = hoodieHiveClient.getTableSchema(); - SchemaDifference schemaDiff = SchemaUtil - .getSchemaDifference(schema, tableSchema, cfg.partitionFields); + SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema, + cfg.partitionFields); if (!schemaDiff.isEmpty()) { LOG.info("Schema difference found for " + cfg.tableName); hoodieHiveClient.updateTableDefinition(schema); @@ -156,8 +157,8 @@ private void syncSchema(boolean tableExists, boolean isRealTime, MessageType sch private void syncPartitions(List writtenPartitionsSince) { try { List hivePartitions = hoodieHiveClient.scanTablePartitions(); - List partitionEvents = hoodieHiveClient - .getPartitionEvents(hivePartitions, writtenPartitionsSince); + List partitionEvents = hoodieHiveClient.getPartitionEvents(hivePartitions, + writtenPartitionsSince); List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); LOG.info("New Partitions " + newPartitions); hoodieHiveClient.addPartitionsToTable(newPartitions); @@ -165,15 +166,13 @@ private void syncPartitions(List writtenPartitionsSince) { LOG.info("Changed Partitions " + updatePartitions); hoodieHiveClient.updatePartitionsToTable(updatePartitions); } catch (Exception e) { - throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName, - e); + throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName, e); } } private List filterPartitions(List events, PartitionEventType eventType) { - return events.stream() - .filter(s -> s.eventType == eventType).map(s -> s.storagePartition).collect( - Collectors.toList()); + return events.stream().filter(s -> s.eventType == eventType).map(s -> s.storagePartition) + .collect(Collectors.toList()); } public static void main(String[] args) throws Exception { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 1ac16ad431004..c3d202b5bdd37 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -15,6 +15,7 @@ * * */ + package com.uber.hoodie.hive; import com.google.common.base.Preconditions; @@ -25,15 +26,22 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; -import com.uber.hoodie.common.table.log.HoodieLogFormat; -import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader; -import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; -import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidDatasetException; import com.uber.hoodie.hive.util.SchemaUtil; +import java.io.IOException; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; import org.apache.commons.dbcp.BasicDataSource; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -51,18 +59,6 @@ import parquet.hadoop.metadata.ParquetMetadata; import parquet.schema.MessageType; -import java.io.IOException; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - @SuppressWarnings("ConstantConditions") public class HoodieHiveClient { @@ -103,16 +99,15 @@ public class HoodieHiveClient { } try { - this.partitionValueExtractor = (PartitionValueExtractor) Class - .forName(cfg.partitionValueExtractorClass).newInstance(); + this.partitionValueExtractor = (PartitionValueExtractor) Class.forName( + cfg.partitionValueExtractorClass).newInstance(); } catch (Exception e) { throw new HoodieHiveSyncException( "Failed to initialize PartitionValueExtractor class " + cfg.partitionValueExtractorClass, e); } - activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); } public HoodieTimeline getActiveTimeline() { @@ -207,9 +202,8 @@ List getPartitionEvents(List tablePartitions, for (Partition tablePartition : tablePartitions) { List hivePartitionValues = tablePartition.getValues(); Collections.sort(hivePartitionValues); - String fullTablePartitionPath = Path - .getPathWithoutSchemeAndAuthority(new Path(tablePartition.getSd().getLocation())).toUri() - .getPath(); + String fullTablePartitionPath = Path.getPathWithoutSchemeAndAuthority( + new Path(tablePartition.getSd().getLocation())).toUri().getPath(); paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath); } @@ -235,8 +229,7 @@ List getPartitionEvents(List tablePartitions, * Scan table partitions */ List scanTablePartitions() throws TException { - return client - .listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); + return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); } void updateTableDefinition(MessageType newSchema) { @@ -245,9 +238,10 @@ void updateTableDefinition(MessageType newSchema) { // Cascade clause should not be present for non-partitioned tables String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : ""; StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append("`") - .append(syncConfig.databaseName).append(".").append(syncConfig.tableName).append("`") - .append(" REPLACE COLUMNS(") - .append(newSchemaStr).append(" )").append(cascadeClause); + .append(syncConfig.databaseName).append(".") + .append(syncConfig.tableName).append("`") + .append(" REPLACE COLUMNS(").append(newSchemaStr).append(" )") + .append(cascadeClause); LOG.info("Creating table with " + sqlBuilder); updateHiveSQL(sqlBuilder.toString()); } catch (IOException e) { @@ -255,8 +249,8 @@ void updateTableDefinition(MessageType newSchema) { } } - void createTable(MessageType storageSchema, - String inputFormatClass, String outputFormatClass, String serdeClass) { + void createTable(MessageType storageSchema, String inputFormatClass, String outputFormatClass, + String serdeClass) { try { String createSQLQuery = SchemaUtil .generateCreateDDL(storageSchema, syncConfig, inputFormatClass, @@ -289,8 +283,8 @@ Map getTableSchema() { } return schema; } catch (SQLException e) { - throw new HoodieHiveSyncException( - "Failed to get table schema for " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to get table schema for " + syncConfig.tableName, + e); } finally { closeQuietly(result, null); } @@ -308,49 +302,47 @@ public MessageType getDataSchema() { try { switch (tableType) { case COPY_ON_WRITE: - // If this is COW, get the last commit and read the schema from a file written in the last commit - HoodieInstant lastCommit = activeTimeline.lastInstant() - .orElseThrow(() -> new InvalidDatasetException(syncConfig.basePath)); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastCommit).get()); + // If this is COW, get the last commit and read the schema from a file written in the + // last commit + HoodieInstant lastCommit = activeTimeline.lastInstant().orElseThrow( + () -> new InvalidDatasetException(syncConfig.basePath)); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastCommit).get()); String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().findAny() - .orElseThrow(() -> new IllegalArgumentException( + .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for commit " + lastCommit + ", could not get schema for dataset " + metaClient.getBasePath())); return readSchemaFromDataFile(new Path(filePath)); case MERGE_ON_READ: - // If this is MOR, depending on whether the latest commit is a delta commit or compaction commit + // If this is MOR, depending on whether the latest commit is a delta commit or + // compaction commit // Get a datafile written and get the schema from that file Optional lastCompactionCommit = metaClient.getActiveTimeline() - .getCommitTimeline().filterCompletedInstants().lastInstant(); + .getCommitTimeline() + .filterCompletedInstants() + .lastInstant(); LOG.info("Found the last compaction commit as " + lastCompactionCommit); Optional lastDeltaCommit; if (lastCompactionCommit.isPresent()) { - lastDeltaCommit = metaClient.getActiveTimeline() - .getDeltaCommitTimeline() + lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline() .filterCompletedInstants() - .findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE) - .lastInstant(); + .findInstantsAfter(lastCompactionCommit.get().getTimestamp(), + Integer.MAX_VALUE).lastInstant(); } else { - lastDeltaCommit = metaClient.getActiveTimeline() - .getDeltaCommitTimeline() - .filterCompletedInstants() - .lastInstant(); + lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline() + .filterCompletedInstants().lastInstant(); } - LOG.info("Found the last delta commit " - + lastDeltaCommit); + LOG.info("Found the last delta commit " + lastDeltaCommit); if (lastDeltaCommit.isPresent()) { HoodieInstant lastDeltaInstant = lastDeltaCommit.get(); // read from the log file wrote - commitMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastDeltaInstant).get()); + commitMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastDeltaInstant).get()); filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().filter(s -> s.contains( - HoodieLogFile.DELTA_EXTENSION)).findAny() - .orElseThrow(() -> new IllegalArgumentException( + .stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION)) + .findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for commit " + lastDeltaInstant + ", could not get schema for dataset " + metaClient.getBasePath())); return readSchemaFromLogFile(lastCompactionCommit, new Path(filePath)); @@ -362,8 +354,8 @@ public MessageType getDataSchema() { throw new InvalidDatasetException(syncConfig.basePath); } } catch (IOException e) { - throw new HoodieHiveSyncException( - "Failed to get dataset schema for " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to get dataset schema for " + syncConfig.tableName, + e); } } @@ -379,11 +371,10 @@ private MessageType readSchemaFromLastCompaction(Optional lastCom + syncConfig.basePath)); // Read from the compacted file wrote - HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get()); + HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastCompactionCommit).get()); String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().findAny() - .orElseThrow(() -> new IllegalArgumentException( + .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for compaction " + lastCompactionCommit + ", could not get schema for dataset " + metaClient.getBasePath())); return readSchemaFromDataFile(new Path(filePath)); @@ -411,12 +402,10 @@ private MessageType readSchemaFromDataFile(Path parquetFilePath) throws IOExcept LOG.info("Reading schema from " + parquetFilePath); if (!fs.exists(parquetFilePath)) { throw new IllegalArgumentException( - "Failed to read schema from data file " + parquetFilePath - + ". File does not exist."); + "Failed to read schema from data file " + parquetFilePath + ". File does not exist."); } - ParquetMetadata fileFooter = - ParquetFileReader - .readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER); + ParquetMetadata fileFooter = ParquetFileReader.readFooter(fs.getConf(), parquetFilePath, + ParquetMetadataConverter.NO_FILTER); return fileFooter.getFileMetaData().getSchema(); } @@ -427,8 +416,8 @@ boolean doesTableExist() { try { return client.tableExists(syncConfig.databaseName, syncConfig.tableName); } catch (TException e) { - throw new HoodieHiveSyncException( - "Failed to check if table exists " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to check if table exists " + syncConfig.tableName, + e); } } @@ -511,8 +500,8 @@ Optional getLastCommitTimeSynced() { // Get the last commit time from the TBLproperties try { Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName); - return Optional - .ofNullable(database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); + return Optional.ofNullable( + database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); } catch (Exception e) { throw new HoodieHiveSyncException( "Failed to get the last commit time synced from the database", e); @@ -537,8 +526,8 @@ List getPartitionsWrittenToSince(Optional lastCommitTimeSynced) if (!lastCommitTimeSynced.isPresent()) { LOG.info("Last commit time synced is not known, listing all partitions"); try { - return FSUtils - .getAllPartitionPaths(fs, syncConfig.basePath, syncConfig.assumeDatePartitioning); + return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath, + syncConfig.assumeDatePartitioning); } catch (IOException e) { throw new HoodieIOException("Failed to list all partitions in " + syncConfig.basePath, e); } @@ -546,8 +535,8 @@ List getPartitionsWrittenToSince(Optional lastCommitTimeSynced) LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); - HoodieTimeline timelineToSync = activeTimeline - .findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE); + HoodieTimeline timelineToSync = activeTimeline.findInstantsAfter(lastCommitTimeSynced.get(), + Integer.MAX_VALUE); return timelineToSync.getInstants().map(s -> { try { return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get()); @@ -579,13 +568,15 @@ void updateLastCommitTimeSynced() { */ static class PartitionEvent { - public enum PartitionEventType {ADD, UPDATE} + public enum PartitionEventType { + ADD, + UPDATE + } PartitionEventType eventType; String storagePartition; - PartitionEvent( - PartitionEventType eventType, String storagePartition) { + PartitionEvent(PartitionEventType eventType, String storagePartition) { this.eventType = eventType; this.storagePartition = storagePartition; } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java index 794c262e3ab63..73776c4616c76 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java @@ -24,8 +24,9 @@ * HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not * straight forward and requires a pluggable implementation to extract the partition value from HDFS * path. - * - * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] + *

+ * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path + * /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] */ public interface PartitionValueExtractor { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java index 956bbb8b22cde..893b61e4cef1e 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java @@ -28,7 +28,7 @@ * HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not * straight forward and requires a pluggable implementation to extract the partition value from HDFS * path. - * + *

* This implementation extracts datestr=yyyy-mm-dd from path of type /yyyy/mm/dd */ public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java index a06494fe227c0..cc7039a386eb0 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java @@ -26,9 +26,9 @@ public class ColumnNameXLator { public static String translateNestedColumn(String colName) { Map.Entry entry; - for (Iterator i$ = xformMap.entrySet().iterator(); i$.hasNext(); + for (Iterator ic = xformMap.entrySet().iterator(); ic.hasNext(); colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) { - entry = (Map.Entry) i$.next(); + entry = (Map.Entry) ic.next(); } return colName; diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index 9eda294d9afde..098c01368fd1f 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -23,14 +23,12 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.hive.HiveSyncConfig; import com.uber.hoodie.hive.HoodieHiveSyncException; import com.uber.hoodie.hive.SchemaDifference; import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -48,7 +46,7 @@ */ public class SchemaUtil { - private static Logger LOG = LoggerFactory.getLogger(SchemaUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(SchemaUtil.class); /** * Get the schema difference between the storage schema and hive table schema @@ -59,19 +57,19 @@ public static SchemaDifference getSchemaDifference(MessageType storageSchema, try { newTableSchema = convertParquetSchemaToHiveSchema(storageSchema); } catch (IOException e) { - throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema", - e); + throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema", e); } LOG.info("Getting schema difference for " + tableSchema + "\r\n\r\n" + newTableSchema); - SchemaDifference.Builder schemaDiffBuilder = - SchemaDifference.newBuilder(storageSchema, tableSchema); + SchemaDifference.Builder schemaDiffBuilder = SchemaDifference + .newBuilder(storageSchema, tableSchema); Set tableColumns = Sets.newHashSet(); for (Map.Entry field : tableSchema.entrySet()) { String fieldName = field.getKey().toLowerCase(); String tickSurroundedFieldName = tickSurround(fieldName); if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName) && !partitionKeys - .contains(fieldName)) { + .contains( + fieldName)) { schemaDiffBuilder.deleteTableColumn(fieldName); } else { // check type @@ -82,8 +80,8 @@ public static SchemaDifference getSchemaDifference(MessageType storageSchema, continue; } // We will log this and continue. Hive schema is a superset of all parquet schemas - LOG.warn("Ignoring table column " + fieldName - + " as its not present in the parquet schema"); + LOG.warn( + "Ignoring table column " + fieldName + " as its not present in the parquet schema"); continue; } tableColumnType = tableColumnType.replaceAll("\\s+", ""); @@ -93,11 +91,12 @@ public static SchemaDifference getSchemaDifference(MessageType storageSchema, expectedType = expectedType.replaceAll("`", ""); if (!tableColumnType.equalsIgnoreCase(expectedType)) { - // check for incremental datasets, the schema type change is allowed as per evolution rules + // check for incremental datasets, the schema type change is allowed as per evolution + // rules if (!isSchemaTypeUpdateAllowed(tableColumnType, expectedType)) { throw new HoodieHiveSyncException( - "Could not convert field Type from " + tableColumnType + " to " - + expectedType + " for field " + fieldName); + "Could not convert field Type from " + tableColumnType + " to " + expectedType + + " for field " + fieldName); } schemaDiffBuilder.updateTableColumn(fieldName, getExpectedType(newTableSchema, tickSurroundedFieldName)); @@ -169,21 +168,19 @@ public static Map convertParquetSchemaToHiveSchema(MessageType m private static String convertField(final Type parquetType) { StringBuilder field = new StringBuilder(); if (parquetType.isPrimitive()) { - final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName = - parquetType.asPrimitiveType().getPrimitiveTypeName(); + final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName = parquetType.asPrimitiveType() + .getPrimitiveTypeName(); final OriginalType originalType = parquetType.getOriginalType(); if (originalType == OriginalType.DECIMAL) { - final DecimalMetadata decimalMetadata = - parquetType.asPrimitiveType().getDecimalMetadata(); - return field.append("DECIMAL(").append(decimalMetadata.getPrecision()). - append(" , ").append(decimalMetadata.getScale()).append(")").toString(); + final DecimalMetadata decimalMetadata = parquetType.asPrimitiveType().getDecimalMetadata(); + return field.append("DECIMAL(").append(decimalMetadata.getPrecision()).append(" , ") + .append(decimalMetadata.getScale()).append(")").toString(); } // TODO - fix the method naming here return parquetPrimitiveTypeName .convert(new PrimitiveType.PrimitiveTypeNameConverter() { @Override - public String convertBOOLEAN( - PrimitiveType.PrimitiveTypeName primitiveTypeName) { + public String convertBOOLEAN(PrimitiveType.PrimitiveTypeName primitiveTypeName) { return "boolean"; } @@ -220,8 +217,7 @@ public String convertFIXED_LEN_BYTE_ARRAY( @Override public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) { - if (originalType == OriginalType.UTF8 - || originalType == OriginalType.ENUM) { + if (originalType == OriginalType.UTF8 || originalType == OriginalType.ENUM) { return "string"; } else { return "binary"; @@ -235,33 +231,28 @@ public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) { switch (originalType) { case LIST: if (parquetGroupType.getFieldCount() != 1) { - throw new UnsupportedOperationException( - "Invalid list type " + parquetGroupType); + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); } Type elementType = parquetGroupType.getType(0); if (!elementType.isRepetition(Type.Repetition.REPEATED)) { - throw new UnsupportedOperationException( - "Invalid list type " + parquetGroupType); + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); } return createHiveArray(elementType, parquetGroupType.getName()); case MAP: if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0) .isPrimitive()) { - throw new UnsupportedOperationException( - "Invalid map type " + parquetGroupType); + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); } GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType(); - if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) || - !mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) || - mapKeyValType.getFieldCount() != 2) { - throw new UnsupportedOperationException( - "Invalid map type " + parquetGroupType); + if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) + || !mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) + || mapKeyValType.getFieldCount() != 2) { + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); } Type keyType = mapKeyValType.getType(0); - if (!keyType.isPrimitive() || - !keyType.asPrimitiveType().getPrimitiveTypeName() - .equals(PrimitiveType.PrimitiveTypeName.BINARY) || - !keyType.getOriginalType().equals(OriginalType.UTF8)) { + if (!keyType.isPrimitive() || !keyType.asPrimitiveType().getPrimitiveTypeName() + .equals(PrimitiveType.PrimitiveTypeName.BINARY) + || !keyType.getOriginalType().equals(OriginalType.UTF8)) { throw new UnsupportedOperationException( "Map key type must be binary (UTF8): " + keyType); } @@ -277,8 +268,7 @@ public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) { // MAP. Hence, PARQUET-113 // dropped the requirement for having MAP_KEY_VALUE. default: - throw new UnsupportedOperationException( - "Cannot convert Parquet type " + parquetType); + throw new UnsupportedOperationException("Cannot convert Parquet type " + parquetType); } } else { // if no original type then it's a record @@ -306,7 +296,8 @@ private static String createHiveStruct(List parquetFields) { // ", " struct.append(">"); String finalStr = struct.toString(); - // Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the schema. + // Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the + // schema. // HDrone sync should not fail because of this. finalStr = finalStr.replaceAll("-", "_"); return finalStr; @@ -362,8 +353,8 @@ private static String createHiveArray(Type elementType, String elementName) { } public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) { - if (prevType == null || prevType.trim().isEmpty() || - newType == null || newType.trim().isEmpty()) { + if (prevType == null || prevType.trim().isEmpty() || newType == null || newType.trim() + .isEmpty()) { return false; } prevType = prevType.toLowerCase(); @@ -392,9 +383,8 @@ public static String generateSchemaString(MessageType storageSchema) throws IOEx return columns.toString(); } - public static String generateCreateDDL(MessageType storageSchema, - HiveSyncConfig config, String inputFormatClass, - String outputFormatClass, String serdeClass) throws IOException { + public static String generateCreateDDL(MessageType storageSchema, HiveSyncConfig config, + String inputFormatClass, String outputFormatClass, String serdeClass) throws IOException { Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema); String columns = generateSchemaString(storageSchema); @@ -422,7 +412,8 @@ private static String getPartitionKeyType(Map hiveSchema, String return hiveSchema.get(partitionKey); } // Default the unknown partition fields to be String - // TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that + // TODO - all partition fields should be part of the schema. datestr is treated as special. + // Dont do that return "String"; } @@ -430,8 +421,7 @@ private static String getPartitionKeyType(Map hiveSchema, String * Read the schema from the log file on path */ @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - public static MessageType readSchemaFromLogFile(FileSystem fs, - Path path) throws IOException { + public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws IOException { Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); HoodieAvroDataBlock lastBlock = null; while (reader.hasNext()) { diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java index 5250a660d9ebf..844b6c3641659 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java @@ -60,39 +60,39 @@ public void teardown() throws IOException, InterruptedException { @Test public void testSchemaConvertArray() throws IOException { // Testing the 3-level annotation structure - MessageType schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().optional(PrimitiveType.PrimitiveTypeName.INT32).named("element") - .named("list").named("int_list").named("ArrayOfInts"); + MessageType schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .optional(PrimitiveType.PrimitiveTypeName.INT32).named("element") + .named("list").named("int_list").named("ArrayOfInts"); String schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`int_list` ARRAY< int>", schemaString); // A array of arrays - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().requiredGroup().as(OriginalType.LIST).repeatedGroup() - .required(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list") - .named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup().requiredGroup() + .as(OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list") + .named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString); // A list of integers - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeated(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("int_list") - .named("ArrayOfInts"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST) + .repeated(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("int_list") + .named("ArrayOfInts"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`int_list` ARRAY< int>", schemaString); // A list of structs with two fields - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .required(PrimitiveType.PrimitiveTypeName.INT32).named("num").named("element") - .named("tuple_list").named("ArrayOfTuples"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") + .required(PrimitiveType.PrimitiveTypeName.INT32).named("num").named("element") + .named("tuple_list").named("ArrayOfTuples"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString); @@ -100,10 +100,10 @@ public void testSchemaConvertArray() throws IOException { // A list of structs with a single field // For this case, since the inner group name is "array", we treat the // element type as a one-element struct. - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .named("array").named("one_tuple_list").named("ArrayOfOneTuples"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array") + .named("one_tuple_list").named("ArrayOfOneTuples"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); @@ -111,10 +111,10 @@ public void testSchemaConvertArray() throws IOException { // A list of structs with a single field // For this case, since the inner group name ends with "_tuple", we also treat the // element type as a one-element struct. - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .named("one_tuple_list_tuple").named("one_tuple_list").named("ArrayOfOneTuples2"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") + .named("one_tuple_list_tuple").named("one_tuple_list").named("ArrayOfOneTuples2"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); @@ -122,22 +122,22 @@ public void testSchemaConvertArray() throws IOException { // A list of structs with a single field // Unlike the above two cases, for this the element type is the type of the // only field in the struct. - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .named("one_tuple_list").named("one_tuple_list").named("ArrayOfOneTuples3"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") + .named("one_tuple_list").named("one_tuple_list").named("ArrayOfOneTuples3"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`one_tuple_list` ARRAY< binary>", schemaString); // A list of maps - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().as(OriginalType.MAP).repeatedGroup().as(OriginalType.MAP_KEY_VALUE) - .required(PrimitiveType.PrimitiveTypeName.BINARY).as(OriginalType.UTF8) - .named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32) - .named("int_value").named("key_value").named("array").named("map_list") - .named("ArrayOfMaps"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup().as(OriginalType.MAP) + .repeatedGroup().as(OriginalType.MAP_KEY_VALUE) + .required(PrimitiveType.PrimitiveTypeName.BINARY).as(OriginalType.UTF8) + .named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32) + .named("int_value").named("key_value").named("array").named("map_list") + .named("ArrayOfMaps"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString); @@ -146,7 +146,8 @@ public void testSchemaConvertArray() throws IOException { @Test public void testBasicSync() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime = "100"; TestUtil.createCOWDataset(commitTime, 5); HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, @@ -160,18 +161,17 @@ public void testBasicSync() assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes", hiveClient.doesTableExist()); assertEquals("Hive Schema should match the dataset schema + partition field", - hiveClient.getTableSchema().size(), - hiveClient.getDataSchema().getColumns().size() + 1); + hiveClient.getTableSchema().size(), hiveClient.getDataSchema().getColumns().size() + 1); assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClient.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - commitTime, - hiveClient.getLastCommitTimeSynced().get()); + commitTime, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncIncremental() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime1 = "100"; TestUtil.createCOWDataset(commitTime1, 5); HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, @@ -183,8 +183,7 @@ public void testSyncIncremental() assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClient.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - commitTime1, - hiveClient.getLastCommitTimeSynced().get()); + commitTime1, hiveClient.getLastCommitTimeSynced().get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -192,33 +191,32 @@ public void testSyncIncremental() TestUtil.addCOWPartitions(1, true, dateTime, commitTime2); // Lets do the sync - hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, - TestUtil.getHiveConf(), TestUtil.fileSystem); - List writtenPartitionsSince = hiveClient - .getPartitionsWrittenToSince(Optional.of(commitTime1)); + hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), + TestUtil.fileSystem); + List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince( + Optional.of(commitTime1)); assertEquals("We should have one partition written after 100 commit", 1, writtenPartitionsSince.size()); List hivePartitions = hiveClient.scanTablePartitions(); - List partitionEvents = hiveClient - .getPartitionEvents(hivePartitions, writtenPartitionsSince); + List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, + writtenPartitionsSince); assertEquals("There should be only one paritition event", 1, partitionEvents.size()); assertEquals("The one partition event must of type ADD", PartitionEventType.ADD, partitionEvents.iterator().next().eventType); - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); // Sync should add the one partition assertEquals("The one partition we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 101", - commitTime2, + assertEquals("The last commit that was sycned should be 101", commitTime2, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncIncrementalWithSchemaEvolution() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime1 = "100"; TestUtil.createCOWDataset(commitTime1, 5); HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, @@ -236,30 +234,27 @@ public void testSyncIncrementalWithSchemaEvolution() TestUtil.addCOWPartitions(1, false, dateTime, commitTime2); // Lets do the sync - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - assertEquals("Hive Schema has evolved and should not be 3 more field", - fields + 3, + assertEquals("Hive Schema has evolved and should not be 3 more field", fields + 3, hiveClient.getTableSchema().size()); assertEquals("Hive Schema has evolved - Field favorite_number has evolved from int to long", - "BIGINT", - hiveClient.getTableSchema().get("favorite_number")); + "BIGINT", hiveClient.getTableSchema().get("favorite_number")); assertTrue("Hive Schema has evolved - Field favorite_movie was added", hiveClient.getTableSchema().containsKey("favorite_movie")); // Sync should add the one partition assertEquals("The one partition we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 101", - commitTime2, + assertEquals("The last commit that was sycned should be 101", commitTime2, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncMergeOnRead() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime = "100"; String deltaCommitTime = "101"; TestUtil.createMORDataset(commitTime, deltaCommitTime, 5); @@ -280,8 +275,7 @@ public void testSyncMergeOnRead() assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClient.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - deltaCommitTime, - hiveClient.getLastCommitTimeSynced().get()); + deltaCommitTime, hiveClient.getLastCommitTimeSynced().get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -291,11 +285,10 @@ public void testSyncMergeOnRead() TestUtil.addCOWPartitions(1, true, dateTime, commitTime2); TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, - TestUtil.getHiveConf(), TestUtil.fileSystem); + hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), + TestUtil.fileSystem); assertEquals("Hive Schema should match the evolved dataset schema + partition field", hiveClient.getTableSchema().size(), @@ -303,14 +296,14 @@ public void testSyncMergeOnRead() // Sync should add the one partition assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 103", - deltaCommitTime2, + assertEquals("The last commit that was sycned should be 103", deltaCommitTime2, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncMergeOnReadRT() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime = "100"; String deltaCommitTime = "101"; String roTablename = TestUtil.hiveSyncConfig.tableName; @@ -321,8 +314,7 @@ public void testSyncMergeOnReadRT() TestUtil.getHiveConf(), TestUtil.fileSystem); assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE - + " should not exist initially", - hiveClientRT.doesTableExist()); + + " should not exist initially", hiveClientRT.doesTableExist()); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), @@ -330,8 +322,7 @@ public void testSyncMergeOnReadRT() tool.syncHoodieTable(); assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE - + " should exist after sync completes", - hiveClientRT.doesTableExist()); + + " should exist after sync completes", hiveClientRT.doesTableExist()); assertEquals("Hive Schema should match the dataset schema + partition field", hiveClientRT.getTableSchema().size(), @@ -339,8 +330,7 @@ public void testSyncMergeOnReadRT() assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClientRT.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - deltaCommitTime, - hiveClientRT.getLastCommitTimeSynced().get()); + deltaCommitTime, hiveClientRT.getLastCommitTimeSynced().get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -350,11 +340,10 @@ public void testSyncMergeOnReadRT() TestUtil.addCOWPartitions(1, true, dateTime, commitTime2); TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, - TestUtil.getHiveConf(), TestUtil.fileSystem); + hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), + TestUtil.fileSystem); assertEquals("Hive Schema should match the evolved dataset schema + partition field", hiveClientRT.getTableSchema().size(), @@ -362,8 +351,7 @@ public void testSyncMergeOnReadRT() // Sync should add the one partition assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClientRT.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 103", - deltaCommitTime2, + assertEquals("The last commit that was sycned should be 103", deltaCommitTime2, hiveClientRT.getLastCommitTimeSynced().get()); TestUtil.hiveSyncConfig.tableName = roTablename; } diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java index ff260f893e714..a2a3ab2bb0a50 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java @@ -16,6 +16,9 @@ package com.uber.hoodie.hive; +import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID; +import static org.junit.Assert.fail; + import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -39,6 +42,15 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.hive.util.HiveTestService; +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.commons.io.FileUtils; @@ -58,19 +70,6 @@ import org.joda.time.format.DateTimeFormatter; import org.junit.runners.model.InitializationError; -import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.UUID; - -import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID; -import static org.junit.Assert.fail; - @SuppressWarnings("SameParameterValue") public class TestUtil { @@ -127,8 +126,7 @@ static void clear() throws IOException { client.updateHiveSQL("drop table if exists " + tableName); } createdTablesSet.clear(); - client.updateHiveSQL( - "drop database if exists " + hiveSyncConfig.databaseName); + client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName); client.updateHiveSQL("create database " + hiveSyncConfig.databaseName); } @@ -182,9 +180,8 @@ static void createMORDataset(String commitTime, String deltaCommitTime, int numb createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); - commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream() - .forEach(l -> compactionMetadata.addWriteStat(key, l))); + commitMetadata.getPartitionToWriteStats().forEach( + (key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); // Write a delta commit HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), @@ -202,8 +199,7 @@ static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimp } static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, - boolean isLogSchemaSimple, DateTime startFrom, - String commitTime, String deltaCommitTime) + boolean isLogSchemaSimple, DateTime startFrom, String commitTime, String deltaCommitTime) throws IOException, URISyntaxException, InterruptedException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, startFrom, commitTime); @@ -211,9 +207,8 @@ static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimp createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); - commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream() - .forEach(l -> compactionMetadata.addWriteStat(key, l))); + commitMetadata.getPartitionToWriteStats().forEach( + (key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), isLogSchemaSimple); @@ -264,8 +259,8 @@ private static List createTestData(Path partPath, boolean isPar for (int i = 0; i < 5; i++) { // Create 5 files String fileId = UUID.randomUUID().toString(); - Path filePath = new Path(partPath.toString() + "/" + FSUtils - .makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileId)); + Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(commitTime, + DEFAULT_TASK_PARTITIONID, fileId)); generateParquetData(filePath, isParquetSchemaSimple); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); @@ -283,11 +278,10 @@ private static void generateParquetData(Path filePath, boolean isParquetSchemaSi org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = new BloomFilter(1000, 0.0001); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); - ParquetWriter writer = new ParquetWriter(filePath, - writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, - ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, - ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, - fileSystem.getConf()); + ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, + 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, + ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, + ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); List testRecords = (isParquetSchemaSimple ? SchemaTestUtil .generateTestRecords(0, 100) @@ -309,10 +303,10 @@ private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLog HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(parquetFilePath)); // Write a log file for this parquet file Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(parquetFilePath.getParent()) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(dataFile.getFileId()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId(dataFile.getFileId()) .overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build(); - List records = (isLogSchemaSimple ? SchemaTestUtil - .generateTestRecords(0, 100) + List records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100) : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime()); @@ -329,37 +323,37 @@ private static void checkResult(boolean result) throws InitializationError { } } - private static void createCommitFile( - HoodieCommitMetadata commitMetadata, String commitTime) + private static void createCommitFile(HoodieCommitMetadata commitMetadata, String commitTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)); + .makeCommitFileName( + commitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); } - private static void createCompactionCommitFile( - HoodieCommitMetadata commitMetadata, String commitTime) - throws IOException { + private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata, + String commitTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)); + .makeCommitFileName( + commitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); } - private static void createDeltaCommitFile( - HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime) - throws IOException { + private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, + String deltaCommitTime) throws IOException { byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeDeltaFileName(deltaCommitTime)); + .makeDeltaFileName( + deltaCommitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java index 7b9172ead70bd..ec9cd3ca26363 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java @@ -16,7 +16,6 @@ package com.uber.hoodie.hive.util; - import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.io.Files; @@ -84,8 +83,7 @@ public Configuration getHadoopConf() { } public HiveServer2 start() throws IOException { - Preconditions - .checkState(workDir != null, "The work dir must be set before starting cluster."); + Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster."); if (hadoopConf == null) { hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); @@ -93,8 +91,7 @@ public HiveServer2 start() throws IOException { String localHiveLocation = getHiveLocation(workDir); if (clean) { - LOG.info( - "Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh."); + LOG.info("Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh."); File file = new File(localHiveLocation); FileUtils.deleteDirectory(file); } @@ -134,11 +131,9 @@ public void stop() throws IOException { hadoopConf = null; } - private HiveConf configureHive(Configuration conf, String localHiveLocation) - throws IOException { + private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { conf.set("hive.metastore.local", "false"); - conf.set(HiveConf.ConfVars.METASTOREURIS.varname, - "thrift://" + bindIP + ":" + metastorePort); + conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort); conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP); conf.setInt(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, serverPort); // The following line to turn of SASL has no effect since HiveAuthFactory calls @@ -154,8 +149,7 @@ private HiveConf configureHive(Configuration conf, String localHiveLocation) File derbyLogFile = new File(localHiveDir, "derby.log"); derbyLogFile.createNewFile(); setSystemProperty("derby.stream.error.file", derbyLogFile.getPath()); - conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, - Files.createTempDir().getAbsolutePath()); + conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, Files.createTempDir().getAbsolutePath()); return new HiveConf(conf, this.getClass()); } @@ -269,8 +263,8 @@ public TServer startMetaStore(String forceBindIP, int port, HiveConf conf) throw int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS); int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS); boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE); - boolean useFramedTransport = - conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT); + boolean useFramedTransport = conf.getBoolVar( + HiveConf.ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT); // don't support SASL yet //boolean useSasl = conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL); @@ -282,21 +276,20 @@ public TServer startMetaStore(String forceBindIP, int port, HiveConf conf) throw tcpKeepAlive ? new TServerSocketKeepAlive(address) : new TServerSocket(address); } else { - serverTransport = - tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port); + serverTransport = tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port); } TProcessor processor; TTransportFactory transFactory; IHMSHandler handler = (IHMSHandler) HiveMetaStore - .newRetryingHMSHandler("new db based metaserver", conf, true); + .newRetryingHMSHandler("new db based metaserver", + conf, true); if (conf.getBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI)) { - transFactory = useFramedTransport ? - new ChainedTTransportFactory(new TFramedTransport.Factory(), - new TUGIContainingTransport.Factory()) : - new TUGIContainingTransport.Factory(); + transFactory = + useFramedTransport ? new ChainedTTransportFactory(new TFramedTransport.Factory(), + new TUGIContainingTransport.Factory()) : new TUGIContainingTransport.Factory(); processor = new TUGIBasedProcessor(handler); LOG.info("Starting DB backed MetaStore Server with SetUGI enabled"); @@ -307,10 +300,11 @@ public TServer startMetaStore(String forceBindIP, int port, HiveConf conf) throw LOG.info("Starting DB backed MetaStore Server"); } - TThreadPoolServer.Args args = - new TThreadPoolServer.Args(serverTransport).processor(processor) - .transportFactory(transFactory).protocolFactory(new TBinaryProtocol.Factory()) - .minWorkerThreads(minWorkerThreads).maxWorkerThreads(maxWorkerThreads); + TThreadPoolServer.Args args = new TThreadPoolServer.Args(serverTransport).processor(processor) + .transportFactory(transFactory) + .protocolFactory(new TBinaryProtocol.Factory()) + .minWorkerThreads(minWorkerThreads) + .maxWorkerThreads(maxWorkerThreads); final TServer tServer = new TThreadPoolServer(args); executorService.submit(new Runnable() { diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java index 26e9cd31e55dc..8e452c8dc11d1 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java @@ -38,7 +38,6 @@ public abstract class BaseAvroPayload implements Serializable { protected final Comparable orderingVal; /** - * * @param record * @param orderingVal */ diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index b16202cffab98..19226af91b168 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -87,8 +87,8 @@ public static KeyGenerator createKeyGenerator(String keyGeneratorClass, public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) throws IOException { try { - return (HoodieRecordPayload) ConstructorUtils - .invokeConstructor(Class.forName(payloadClass), (Object) record, (Object) orderingVal); + return (HoodieRecordPayload) ConstructorUtils.invokeConstructor(Class.forName(payloadClass), + (Object) record, (Object) orderingVal); } catch (Throwable e) { throw new IOException("Could not create payload for class: " + payloadClass, e); } @@ -103,36 +103,26 @@ public static void checkRequiredProperties(PropertiesConfiguration configuration }); } - public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, - String schemaStr, - String basePath, - String tblName, - Map parameters) throws Exception { - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() - .combineInput(true, true) - .withPath(basePath) - .withAutoCommit(false) - .withSchema(schemaStr) - .forTable(tblName) - .withIndexConfig( - HoodieIndexConfig.newBuilder() - .withIndexType(HoodieIndex.IndexType.BLOOM) - .build()) + public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, + String basePath, String tblName, Map parameters) throws Exception { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().combineInput(true, true) + .withPath(basePath).withAutoCommit(false) + .withSchema(schemaStr).forTable(tblName).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY())) + .withPayloadClass(parameters.get( + DataSourceWriteOptions + .PAYLOAD_CLASS_OPT_KEY())) .build()) // override above with Hoodie configs specified as options. - .withProps(parameters) - .build(); + .withProps(parameters).build(); return new HoodieWriteClient<>(jssc, writeConfig); } public static JavaRDD doWriteOperation(HoodieWriteClient client, - JavaRDD hoodieRecords, - String commitTime, - String operation) { + JavaRDD hoodieRecords, String commitTime, String operation) { if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) { return client.bulkInsert(hoodieRecords, commitTime); } else if (operation.equals(DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())) { @@ -143,14 +133,9 @@ public static JavaRDD doWriteOperation(HoodieWriteClient client, } } - public static HoodieRecord createHoodieRecord(GenericRecord gr, - Comparable orderingVal, - HoodieKey hKey, - String payloadClass) throws IOException { - HoodieRecordPayload payload = DataSourceUtils.createPayload( - payloadClass, - gr, - orderingVal); + public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, + HoodieKey hKey, String payloadClass) throws IOException { + HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal); return new HoodieRecord<>(hKey, payload); } } diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java index 996786a7d4bed..f3fe941000f75 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -67,11 +67,12 @@ public static String latestCommit(FileSystem fs, String basePath) { */ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); + .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), + null); if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { return table.getActiveTimeline().getTimelineOfActions( Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION)); + HoodieActiveTimeline.DELTA_COMMIT_ACTION)); } else { return table.getCommitTimeline().filterCompletedInstants(); } diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java b/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java index f23148ad3b0d4..74aa74560c6bb 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java @@ -24,7 +24,8 @@ import org.apache.commons.configuration.PropertiesConfiguration; /** - * Abstract class to extend for plugging in extraction of {@link com.uber.hoodie.common.model.HoodieKey} + * Abstract class to extend for plugging in extraction of + * {@link com.uber.hoodie.common.model.HoodieKey} * from an Avro record */ public abstract class KeyGenerator implements Serializable { diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java index 6b12b1f97242b..48396f08c9486 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java @@ -28,7 +28,7 @@ /** * Default payload used for delta streamer. - * + *

* 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2. * combineAndGetUpdateValue/getInsertValue - Simply overwrites storage with latest delta record */ @@ -36,7 +36,6 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements HoodieRecordPayload { /** - * * @param record * @param orderingVal */ diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index 6840248878884..4312636fa2586 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -47,10 +47,11 @@ object AvroConversionUtils { structName: String, recordNamespace: String): (Any) => Any = { dataType match { - case BinaryType => (item: Any) => item match { - case null => null - case bytes: Array[Byte] => ByteBuffer.wrap(bytes) - } + case BinaryType => (item: Any) => + item match { + case null => null + case bytes: Array[Byte] => ByteBuffer.wrap(bytes) + } case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | StringType | BooleanType => identity case _: DecimalType => (item: Any) => if (item == null) null else item.toString diff --git a/hoodie-spark/src/test/java/DataSourceTestUtils.java b/hoodie-spark/src/test/java/DataSourceTestUtils.java index a39c42ac35849..e3d4c65ee7706 100644 --- a/hoodie-spark/src/test/java/DataSourceTestUtils.java +++ b/hoodie-spark/src/test/java/DataSourceTestUtils.java @@ -32,17 +32,15 @@ public static Optional convertToString(HoodieRecord record) { try { String str = ((TestRawTripPayload) record.getData()).getJsonData(); str = "{" + str.substring(str.indexOf("\"timestamp\":")); - return Optional - .of(str.replaceAll("}", ", \"partition\": \"" + record.getPartitionPath() + "\"}")); + return Optional.of(str.replaceAll("}", + ", \"partition\": \"" + record.getPartitionPath() + "\"}")); } catch (IOException e) { return Optional.empty(); } } public static List convertToStringList(List records) { - return records.stream().map(hr -> convertToString(hr)) - .filter(os -> os.isPresent()) - .map(os -> os.get()) - .collect(Collectors.toList()); + return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent()) + .map(os -> os.get()).collect(Collectors.toList()); } } diff --git a/hoodie-spark/src/test/java/HoodieJavaApp.java b/hoodie-spark/src/test/java/HoodieJavaApp.java index ff2a0341696bb..bef67cfc82c9e 100644 --- a/hoodie-spark/src/test/java/HoodieJavaApp.java +++ b/hoodie-spark/src/test/java/HoodieJavaApp.java @@ -1,12 +1,12 @@ /* * 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. @@ -16,7 +16,6 @@ * */ - import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.uber.hoodie.DataSourceReadOptions; @@ -68,10 +67,9 @@ public static void main(String[] args) throws Exception { public void run() throws Exception { // Spark session setup.. - SparkSession spark = SparkSession.builder() - .appName("Hoodie Spark APP") - .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .master("local[1]") + SparkSession spark = SparkSession.builder().appName("Hoodie Spark APP") + .config("spark.serializer", + "org.apache.spark.serializer.KryoSerializer").master("local[1]") .getOrCreate(); JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext()); FileSystem fs = FileSystem.get(jssc.hadoopConfiguration()); @@ -83,13 +81,12 @@ public void run() throws Exception { * Commit with only inserts */ // Generate some input.. - List records1 = DataSourceTestUtils - .convertToStringList(dataGen.generateInserts("001"/* ignore */, 100)); + List records1 = DataSourceTestUtils.convertToStringList( + dataGen.generateInserts("001"/* ignore */, 100)); Dataset inputDF1 = spark.read().json(jssc.parallelize(records1, 2)); // Save as hoodie dataset (copy on write) - inputDF1.write() - .format("com.uber.hoodie") // specify the hoodie source + inputDF1.write().format("com.uber.hoodie") // specify the hoodie source .option("hoodie.insert.shuffle.parallelism", "2") // any hoodie client config can be passed like this .option("hoodie.upsert.shuffle.parallelism", @@ -104,7 +101,8 @@ public void run() throws Exception { "timestamp") // use to combine duplicate records in input/with disk val .option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries .mode( - SaveMode.Overwrite) // This will remove any existing data at path below, and create a new dataset if needed + SaveMode.Overwrite) // This will remove any existing data at path below, and create a + // new dataset if needed .save(tablePath); // ultimately where the dataset will be placed String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); logger.info("First commit at instant time :" + commitInstantTime1); @@ -112,28 +110,24 @@ public void run() throws Exception { /** * Commit that updates records */ - List records2 = DataSourceTestUtils - .convertToStringList(dataGen.generateUpdates("002"/* ignore */, 100)); + List records2 = DataSourceTestUtils.convertToStringList( + dataGen.generateUpdates("002"/* ignore */, 100)); Dataset inputDF2 = spark.read().json(jssc.parallelize(records2, 2)); - inputDF2.write() - .format("com.uber.hoodie") - .option("hoodie.insert.shuffle.parallelism", "2") + inputDF2.write().format("com.uber.hoodie").option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.upsert.shuffle.parallelism", "2") .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") - .option(HoodieWriteConfig.TABLE_NAME, tableName) - .mode(SaveMode.Append) - .save(tablePath); + .option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append).save(tablePath); String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); logger.info("Second commit at instant time :" + commitInstantTime1); /** * Read & do some queries */ - Dataset hoodieROViewDF = spark.read() - .format("com.uber.hoodie") - // pass any path glob, can include hoodie & non-hoodie datasets + Dataset hoodieROViewDF = spark.read().format("com.uber.hoodie") + // pass any path glob, can include hoodie & non-hoodie + // datasets .load(tablePath + "/*/*/*/*"); hoodieROViewDF.registerTempTable("hoodie_ro"); spark.sql("describe hoodie_ro").show(); @@ -149,7 +143,8 @@ public void run() throws Exception { DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1) // Only changes in write 2 above - .load(tablePath); // For incremental view, pass in the root/base path of dataset + .load( + tablePath); // For incremental view, pass in the root/base path of dataset logger.info("You will only see records from : " + commitInstantTime2); hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 77184971ce061..70871b30a4b84 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -65,8 +65,7 @@ public class HDFSParquetImporter implements Serializable { private transient FileSystem fs; public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd"); - public HDFSParquetImporter( - Config cfg) throws IOException { + public HDFSParquetImporter(Config cfg) throws IOException { this.cfg = cfg; } @@ -77,9 +76,8 @@ public static class FormatValidator implements IValueValidator { @Override public void validate(String name, String value) throws ParameterException { if (value == null || !validFormats.contains(value)) { - throw new ParameterException(String - .format("Invalid format type: value:%s: supported formats:%s", value, - validFormats)); + throw new ParameterException(String.format( + "Invalid format type: value:%s: supported formats:%s", value, validFormats)); } } } @@ -91,9 +89,8 @@ public static class SourceTypeValidator implements IValueValidator { @Override public void validate(String name, String value) throws ParameterException { if (value == null || !validSourceTypes.contains(value)) { - throw new ParameterException(String - .format("Invalid source type: value:%s: supported source types:%s", value, - validSourceTypes)); + throw new ParameterException(String.format( + "Invalid source type: value:%s: supported source types:%s", value, validSourceTypes)); } } } @@ -127,23 +124,21 @@ public static class Config implements Serializable { "-sf"}, description = "path for Avro schema file", required = true) public String schemaFile = null; @Parameter(names = {"--format", - "-f"}, description = "Format for the input data.", required = false, - validateValueWith = FormatValidator.class) + "-f"}, description = "Format for the input data.", required = false, validateValueWith = + FormatValidator.class) public String format = null; - @Parameter(names = {"--spark-master", - "-ms"}, description = "Spark master", required = false) + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) public String sparkMaster = null; @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) public String sparkMemory = null; - @Parameter(names = {"--retry", - "-rt"}, description = "number of retries", required = false) + @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false) public int retry = 0; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; } - public static void main(String args[]) throws Exception { + public static void main(String[] args) throws Exception { final HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); JCommander cmd = new JCommander(cfg, args); if (cfg.help || args.length == 0) { @@ -182,8 +177,7 @@ private String getSchema() throws Exception { // Read schema file. Path p = new Path(cfg.schemaFile); if (!fs.exists(p)) { - throw new Exception( - String.format("Could not find - %s - schema file.", cfg.schemaFile)); + throw new Exception(String.format("Could not find - %s - schema file.", cfg.schemaFile)); } long len = fs.getFileStatus(p).getLen(); ByteBuffer buf = ByteBuffer.allocate((int) len); @@ -205,8 +199,7 @@ public int dataImport(JavaSparkContext jsc, int retry) throws Exception { try { // Verify that targetPath is not present. if (fs.exists(new Path(cfg.targetPath))) { - throw new HoodieIOException( - String.format("Make sure %s is not present.", cfg.targetPath)); + throw new HoodieIOException(String.format("Make sure %s is not present.", cfg.targetPath)); } do { ret = dataImport(jsc); @@ -232,7 +225,8 @@ protected int dataImport(JavaSparkContext jsc) throws IOException { Properties properties = new Properties(); properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType); - HoodieTableMetaClient.initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); + HoodieTableMetaClient + .initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism); @@ -240,50 +234,54 @@ protected int dataImport(JavaSparkContext jsc) throws IOException { Job job = Job.getInstance(jsc.hadoopConfiguration()); // To parallelize reading file status. job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024"); - AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(), - (new Schema.Parser().parse(schemaStr))); + AvroReadSupport + .setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr))); ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class)); - JavaRDD> hoodieRecords = jsc - .newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, - GenericRecord.class, job.getConfiguration()) - // To reduce large number of tasks. + JavaRDD> hoodieRecords = jsc.newAPIHadoopFile(cfg.srcPath, + ParquetInputFormat.class, Void.class, GenericRecord.class, job.getConfiguration()) + // To reduce large number of + // tasks. .coalesce(16 * cfg.parallelism) .map(entry -> { - GenericRecord genericRecord = ((Tuple2) entry)._2(); - Object partitionField = genericRecord.get(cfg.partitionKey); - if (partitionField == null) { - throw new HoodieIOException( - "partition key is missing. :" + cfg.partitionKey); - } - Object rowField = genericRecord.get(cfg.rowKey); - if (rowField == null) { - throw new HoodieIOException( - "row field is missing. :" + cfg.rowKey); - } - long ts = (long) ((Double) partitionField * 1000l); - String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); - return new HoodieRecord<>( - new HoodieKey((String) rowField, partitionPath), - new HoodieJsonPayload(genericRecord.toString())); - } - ); + GenericRecord genericRecord + = ((Tuple2) entry)._2(); + Object partitionField = + genericRecord.get(cfg.partitionKey); + if (partitionField == null) { + throw new HoodieIOException( + "partition key is missing. :" + + cfg.partitionKey); + } + Object rowField = genericRecord.get(cfg.rowKey); + if (rowField == null) { + throw new HoodieIOException( + "row field is missing. :" + cfg.rowKey); + } + long ts = (long) ((Double) partitionField * 1000L); + String partitionPath = + PARTITION_FORMATTER.format(new Date(ts)); + return new HoodieRecord<>( + new HoodieKey( + (String) rowField, partitionPath), + new HoodieJsonPayload( + genericRecord.toString())); + }); // Get commit time. String commitTime = client.startCommit(); JavaRDD writeResponse = client.bulkInsert(hoodieRecords, commitTime); Accumulator errors = jsc.accumulator(0); writeResponse.foreach(writeStatus -> { - if (writeStatus.hasErrors()) { - errors.add(1); - logger.error(String.format("Error processing records :writeStatus:%s", - writeStatus.getStat().toString())); - } + if (writeStatus.hasErrors()) { + errors.add(1); + logger.error(String.format("Error processing records :writeStatus:%s", + writeStatus.getStat().toString())); + } }); if (errors.value() == 0) { - logger.info(String - .format("Dataset imported into hoodie dataset with %s commit time.", - commitTime)); + logger.info( + String.format("Dataset imported into hoodie dataset with %s commit time.", commitTime)); return 0; } logger.error(String.format("Import failed with %d errors.", errors.value())); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index a6fbc3f363cfb..07d3ecbe9d3e8 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -52,9 +52,9 @@ /** * Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as * another hive temporary table. - * + *

* Current Limitations: - * + *

* - Only the source table can be incrementally pulled (usually the largest table) - The * incrementally pulled table can't be referenced more than once. */ @@ -66,8 +66,7 @@ public class HiveIncrementalPuller { public static class Config implements Serializable { @Parameter(names = {"--hiveUrl"}) - public String hiveJDBCUrl = - "jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2"; + public String hiveJDBCUrl = "jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2"; @Parameter(names = {"--hiveUser"}) public String hiveUsername = "hive"; @Parameter(names = {"--hivePass"}) @@ -113,8 +112,8 @@ public static class Config implements Serializable { public HiveIncrementalPuller(Config config) throws IOException { this.config = config; validateConfig(config); - String templateContent = IOUtils - .toString(this.getClass().getResourceAsStream("IncrementalPull.sqltemplate")); + String templateContent = IOUtils.toString( + this.getClass().getResourceAsStream("IncrementalPull.sqltemplate")); incrementalPullSQLtemplate = new ST(templateContent); } @@ -182,19 +181,22 @@ private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, S String storedAsClause = getStoredAsClause(); incrementalPullSQLtemplate.add("storedAsClause", storedAsClause); - String incrementalSQL = - new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z").next(); + String incrementalSQL = new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z") + .next(); if (!incrementalSQL.contains(config.sourceDb + "." + config.sourceTable)) { log.info("Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable - + ", which means its pulling from a different table. Fencing this from happening."); + + ", which means its pulling from a different table. Fencing this from " + + "happening."); throw new HoodieIncrementalPullSQLException( "Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable); } if (!incrementalSQL.contains("`_hoodie_commit_time` > '%targetBasePath'")) { log.info("Incremental SQL : " + incrementalSQL - + " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add this clause for incremental to work properly."); + + " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add " + + "this clause for incremental to work properly."); throw new HoodieIncrementalPullSQLException( - "Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which means its not pulling incrementally"); + "Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which " + + "means its not pulling incrementally"); } incrementalPullSQLtemplate @@ -227,11 +229,12 @@ private void initHiveBeelineProperties(Statement stmt) throws SQLException { // Set the hoodie modie executeStatement("set hoodie." + config.sourceTable + ".consume.mode=INCREMENTAL", stmt); // Set the from commit time - executeStatement("set hoodie." + config.sourceTable + ".consume.start.timestamp=" - + config.fromCommitTime, stmt); + executeStatement( + "set hoodie." + config.sourceTable + ".consume.start.timestamp=" + config.fromCommitTime, + stmt); // Set number of commits to pull - executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String - .valueOf(config.maxCommits), stmt); + executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String.valueOf( + config.maxCommits), stmt); } private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException { @@ -245,8 +248,9 @@ private void executeStatement(String sql, Statement stmt) throws SQLException { } private String inferCommitTime(FileSystem fs) throws SQLException, IOException { - log.info("FromCommitTime not specified. Trying to infer it from Hoodie dataset " - + config.targetDb + "." + config.targetTable); + log.info( + "FromCommitTime not specified. Trying to infer it from Hoodie dataset " + config.targetDb + + "." + config.targetTable); String targetDataLocation = getTableLocation(config.targetDb, config.targetTable); return scanForCommitTime(fs, targetDataLocation); } @@ -260,8 +264,8 @@ private String getTableLocation(String db, String table) throws SQLException { resultSet = stmt.executeQuery("describe formatted `" + db + "." + table + "`"); while (resultSet.next()) { if (resultSet.getString(1).trim().equals("Location:")) { - log.info("Inferred table location for " + db + "." + table + " as " + resultSet - .getString(2)); + log.info( + "Inferred table location for " + db + "." + table + " as " + resultSet.getString(2)); return resultSet.getString(2); } } @@ -293,8 +297,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO } HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath); - Optional - lastCommit = metadata.getActiveTimeline().getCommitsTimeline() + Optional lastCommit = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants().lastInstant(); if (lastCommit.isPresent()) { return lastCommit.get().getTimestamp(); @@ -302,8 +305,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO return "0"; } - private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) - throws IOException { + private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) throws IOException { Path targetBaseDirPath = new Path(config.hoodieTmpDir, config.targetTable + "__" + config.sourceTable); if (!fs.exists(targetBaseDirPath)) { @@ -320,8 +322,7 @@ private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) if (fs.exists(targetPath)) { boolean result = fs.delete(targetPath, true); if (!result) { - throw new HoodieException( - "Could not delete existing " + targetPath); + throw new HoodieException("Could not delete existing " + targetPath); } } log.info("Creating " + targetPath + " with permission drwxrwxrwx"); @@ -334,15 +335,14 @@ private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants() - .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants() - .map(HoodieInstant::getTimestamp) + .findInstantsAfter(config.fromCommitTime, config.maxCommits) + .getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (commitsToSync.isEmpty()) { - log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata - .getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .getInstants() - .collect(Collectors.toList()) + " and from commit time is " - + config.fromCommitTime); + log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + + metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .getInstants().collect(Collectors.toList()) + + " and from commit time is " + config.fromCommitTime); return null; } log.info("Syncing commits " + commitsToSync); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index 41ef772769288..a91e8f396bd55 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -77,19 +77,18 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView( tableMetadata, - tableMetadata.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants()); + tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); // Get the latest commit - Optional latestCommit = tableMetadata.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().lastInstant(); + Optional latestCommit = tableMetadata.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); if (!latestCommit.isPresent()) { logger.warn("No commits present. Nothing to snapshot"); return; } final String latestCommitTimestamp = latestCommit.get().getTimestamp(); - logger.info(String - .format("Starting to snapshot latest version files which are also no-late-than %s.", - latestCommitTimestamp)); + logger.info(String.format( + "Starting to snapshot latest version files which are also no-late-than %s.", + latestCommitTimestamp)); List partitions = FSUtils .getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning); @@ -104,25 +103,24 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi fs.delete(new Path(outputDir), true); } - jsc.parallelize(partitions, partitions.size()) - .flatMap(partition -> { - // Only take latest version files <= latestCommit. - FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get()); - List> filePaths = new ArrayList<>(); - Stream dataFiles = fsView - .getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp); - dataFiles.forEach( - hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()))); - - // also need to copy over partition metadata - Path partitionMetaFile = new Path(new Path(baseDir, partition), - HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); - if (fs1.exists(partitionMetaFile)) { - filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString())); - } + jsc.parallelize(partitions, partitions.size()).flatMap(partition -> { + // Only take latest version files <= latestCommit. + FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get()); + List> filePaths = new ArrayList<>(); + Stream dataFiles = fsView.getLatestDataFilesBeforeOrOn(partition, + latestCommitTimestamp); + dataFiles.forEach( + hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()))); + + // also need to copy over partition metadata + Path partitionMetaFile = new Path(new Path(baseDir, partition), + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); + if (fs1.exists(partitionMetaFile)) { + filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString())); + } - return filePaths.iterator(); - }).foreach(tuple -> { + return filePaths.iterator(); + }).foreach(tuple -> { String partition = tuple._1(); Path sourceFilePath = new Path(tuple._2()); Path toPartitionPath = new Path(outputDir, partition); @@ -131,8 +129,8 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi if (!ifs.exists(toPartitionPath)) { ifs.mkdirs(toPartitionPath); } - FileUtil.copy(ifs, sourceFilePath, ifs, - new Path(toPartitionPath, sourceFilePath.getName()), false, ifs.getConf()); + FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()), + false, ifs.getConf()); }); // Also copy the .commit files @@ -143,23 +141,21 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) { return true; } else { - String commitTime = - FSUtils.getCommitFromCommitFile(commitFilePath.getName()); + String commitTime = FSUtils.getCommitFromCommitFile(commitFilePath.getName()); return HoodieTimeline.compareTimestamps(commitTime, latestCommitTimestamp, HoodieTimeline.LESSER_OR_EQUAL); } }); for (FileStatus commitStatus : commitFilesToCopy) { Path targetFilePath = new Path( - outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus - .getPath().getName()); + outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus.getPath() + .getName()); if (!fs.exists(targetFilePath.getParent())) { fs.mkdirs(targetFilePath.getParent()); } if (fs.exists(targetFilePath)) { - logger.error(String - .format("The target output commit file (%targetBasePath) already exists.", - targetFilePath)); + logger.error(String.format( + "The target output commit file (%targetBasePath) already exists.", targetFilePath)); } FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf()); } @@ -179,9 +175,8 @@ public static void main(String[] args) throws IOException { // Take input configs final Config cfg = new Config(); new JCommander(cfg, args); - logger.info(String - .format("Snapshot hoodie table from %targetBasePath to %targetBasePath", cfg.basePath, - cfg.outputPath)); + logger.info(String.format("Snapshot hoodie table from %targetBasePath to %targetBasePath", + cfg.basePath, cfg.outputPath)); // Create a spark job to do the snapshot copy SparkConf sparkConf = new SparkConf().setAppName("Hoodie-snapshot-copier"); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index 502e36e838053..d64829ea6bf72 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -41,9 +41,8 @@ public static Source createSource(String sourceClass, PropertiesConfiguration cf JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider) throws IOException { try { - return (Source) ConstructorUtils - .invokeConstructor(Class.forName(sourceClass), (Object) cfg, (Object) jssc, - (Object) dataFormat, (Object) schemaProvider); + return (Source) ConstructorUtils.invokeConstructor(Class.forName(sourceClass), (Object) cfg, + (Object) jssc, (Object) dataFormat, (Object) schemaProvider); } catch (Throwable e) { throw new IOException("Could not load source class " + sourceClass, e); } @@ -52,8 +51,8 @@ public static Source createSource(String sourceClass, PropertiesConfiguration cf public static SchemaProvider createSchemaProvider(String schemaProviderClass, PropertiesConfiguration cfg) throws IOException { try { - return (SchemaProvider) ConstructorUtils - .invokeConstructor(Class.forName(schemaProviderClass), (Object) cfg); + return (SchemaProvider) ConstructorUtils.invokeConstructor(Class.forName(schemaProviderClass), + (Object) cfg); } catch (Throwable e) { throw new IOException("Could not load schema provider class " + schemaProviderClass, e); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index eadb5cd491ff9..f4619465cae3f 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -120,9 +120,8 @@ public HoodieDeltaStreamer(Config cfg) throws IOException { if (fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), cfg.targetBasePath); - this.commitTimelineOpt = Optional - .of(meta.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants()); + this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants()); } else { this.commitTimelineOpt = Optional.empty(); } @@ -137,13 +136,13 @@ private void initSource() throws IOException { // Create the source & schema providers PropertiesConfiguration sourceCfg = UtilHelpers.readConfig(fs, new Path(cfg.sourceConfigProps)); log.info("Creating source " + cfg.sourceClassName + " with configs : " + sourceCfg.toString()); - this.source = UtilHelpers - .createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, schemaProvider); + this.source = UtilHelpers.createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, + schemaProvider); } private void initSchemaProvider() throws IOException { - PropertiesConfiguration schemaCfg = UtilHelpers - .readConfig(fs, new Path(cfg.schemaProviderConfigProps)); + PropertiesConfiguration schemaCfg = UtilHelpers.readConfig(fs, + new Path(cfg.schemaProviderConfigProps)); log.info( "Creating schema provider " + cfg.schemaProviderClassName + " with configs : " + schemaCfg .toString()); @@ -175,8 +174,8 @@ private JavaSparkContext getSparkContext() { sparkConf = HoodieWriteClient.registerClasses(sparkConf); // register the schemas, so that shuffle does not serialize the full schemas - List schemas = Arrays - .asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema()); + List schemas = Arrays.asList(schemaProvider.getSourceSchema(), + schemaProvider.getTargetSchema()); sparkConf.registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList()); return new JavaSparkContext(sparkConf); } @@ -187,15 +186,14 @@ private void sync() throws Exception { if (commitTimelineOpt.isPresent()) { Optional lastCommit = commitTimelineOpt.get().lastInstant(); if (lastCommit.isPresent()) { - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata - .fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get()); if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) { resumeCheckpointStr = Optional.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); } else { throw new HoodieDeltaStreamerException( - "Unable to find previous checkpoint. Please double check if this table " + - "was indeed built via delta streamer "); + "Unable to find previous checkpoint. Please double check if this table " + + "was indeed built via delta streamer "); } } } else { @@ -208,8 +206,8 @@ private void sync() throws Exception { log.info("Checkpoint to resume from : " + resumeCheckpointStr); // Pull the data from the source & prepare the write - Pair>, String> dataAndCheckpoint = source - .fetchNewData(resumeCheckpointStr, cfg.maxInputBytes); + Pair>, String> dataAndCheckpoint = source.fetchNewData( + resumeCheckpointStr, cfg.maxInputBytes); if (!dataAndCheckpoint.getKey().isPresent()) { log.info("No new data, nothing to commit.. "); @@ -217,14 +215,11 @@ private void sync() throws Exception { } JavaRDD avroRDD = dataAndCheckpoint.getKey().get(); - JavaRDD records = avroRDD - .map(gr -> { - HoodieRecordPayload payload = DataSourceUtils.createPayload( - cfg.payloadClassName, - gr, - (Comparable) gr.get(cfg.sourceOrderingField)); - return new HoodieRecord<>(keyGenerator.getKey(gr), payload); - }); + JavaRDD records = avroRDD.map(gr -> { + HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr, + (Comparable) gr.get(cfg.sourceOrderingField)); + return new HoodieRecord<>(keyGenerator.getKey(gr), payload); + }); // Perform the write HoodieWriteConfig hoodieCfg = getHoodieClientConfig(cfg.hoodieClientProps); @@ -245,8 +240,8 @@ private void sync() throws Exception { HashMap checkpointCommitMetadata = new HashMap<>(); checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue()); - boolean success = client - .commit(commitTime, writeStatusRDD, Optional.of(checkpointCommitMetadata)); + boolean success = client.commit(commitTime, writeStatusRDD, + Optional.of(checkpointCommitMetadata)); if (success) { log.info("Commit " + commitTime + " successful!"); // TODO(vc): Kick off hive sync from here. @@ -258,23 +253,20 @@ private void sync() throws Exception { } private HoodieWriteConfig getHoodieClientConfig(String hoodieClientCfgPath) throws Exception { - return HoodieWriteConfig.newBuilder() - .combineInput(true, true) - .withPath(cfg.targetBasePath) - .withAutoCommit(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass(OverwriteWithLatestAvroPayload.class.getName()).build()) + return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) + .withAutoCommit(false).withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withPayloadClass( + OverwriteWithLatestAvroPayload + .class + .getName()).build()) .withSchema(schemaProvider.getTargetSchema().toString()) - .forTable(cfg.targetTableName) - .withIndexConfig( + .forTable(cfg.targetTableName).withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .fromInputStream(fs.open(new Path(hoodieClientCfgPath))) - .build(); + .fromInputStream(fs.open(new Path(hoodieClientCfgPath))).build(); } private enum Operation { - UPSERT, - INSERT + UPSERT, INSERT } private class OperationConvertor implements IStringConverter { @@ -308,58 +300,86 @@ public static class Config implements Serializable { public String targetTableName; @Parameter(names = {"--hoodie-client-config"}, description = - "path to properties file on localfs or dfs, with hoodie client config. Sane defaults" + - "are used, but recommend use to provide basic things like metrics endpoints, hive configs etc") + "path to properties file on localfs or " + + "dfs, with hoodie client config. " + + "Sane defaults" + + "are used, but recommend use to " + + "provide basic things like metrics " + + "endpoints, hive configs etc") public String hoodieClientProps = null; /** * SOURCE CONFIGS **/ @Parameter(names = {"--source-class"}, description = - "subclass of com.uber.hoodie.utilities.sources.Source to use to read data. " + - "built-in options: com.uber.hoodie.utilities.common.{DFSSource (default), KafkaSource, HiveIncrPullSource}") + "subclass of com.uber.hoodie.utilities.sources" + + ".Source to use to read data. " + + "built-in options: com.uber.hoodie.utilities" + + ".common.{DFSSource (default), KafkaSource, " + + "HiveIncrPullSource}") public String sourceClassName = DFSSource.class.getName(); @Parameter(names = {"--source-config"}, description = - "path to properties file on localfs or dfs, with source configs. " + - "For list of acceptable properties, refer the source class", required = true) + "path to properties file on localfs or dfs, with " + + "source configs. " + + "For list of acceptable properties, refer " + + "the source class", required = true) public String sourceConfigProps = null; @Parameter(names = {"--source-format"}, description = - "Format of data in source, JSON (default), Avro. All source data is " + - "converted to Avro using the provided schema in any case", converter = SourceFormatConvertor.class) + "Format of data in source, JSON (default), Avro. " + + "All source data is " + + "converted to Avro using the provided " + + "schema in any case", converter = SourceFormatConvertor.class) public SourceDataFormat sourceFormat = SourceDataFormat.JSON; @Parameter(names = {"--source-ordering-field"}, description = - "Field within source record to decide how to break ties between " + - " records with same key in input data. Default: 'ts' holding unix timestamp of record") + "Field within source record to decide how" + + " to break ties between " + + " records with same key in input " + + "data. Default: 'ts' holding unix " + + "timestamp of record") public String sourceOrderingField = "ts"; @Parameter(names = {"--key-generator-class"}, description = - "Subclass of com.uber.hoodie.utilities.common.KeyExtractor to generate" + - "a HoodieKey from the given avro record. Built in: SimpleKeyGenerator (Uses provided field names as recordkey & partitionpath. " - + - "Nested fields specified via dot notation, e.g: a.b.c)") + "Subclass of com.uber.hoodie.utilities" + + ".common.KeyExtractor to generate" + + "a HoodieKey from the given avro " + + "record. Built in: SimpleKeyGenerator" + + " (Uses provided field names as " + + "recordkey & partitionpath. " + + "Nested fields specified via dot " + + "notation, e.g: a.b.c)") public String keyGeneratorClass = SimpleKeyGenerator.class.getName(); @Parameter(names = {"--key-generator-config"}, description = - "Path to properties file on localfs or dfs, with KeyGenerator configs. " + - "For list of acceptable properites, refer the KeyGenerator class", required = true) + "Path to properties file on localfs or " + + "dfs, with KeyGenerator configs. " + + "For list of acceptable properites, " + + "refer the KeyGenerator class", + required = true) public String keyGeneratorProps = null; @Parameter(names = {"--payload-class"}, description = - "subclass of HoodieRecordPayload, that works off a GenericRecord. " + - "Default: SourceWrapperPayload. Implement your own, if you want to do something other than overwriting existing value") + "subclass of HoodieRecordPayload, that works off " + + "a GenericRecord. " + + "Default: SourceWrapperPayload. Implement " + + "your own, if you want to do something " + + "other than overwriting existing value") public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); @Parameter(names = {"--schemaprovider-class"}, description = - "subclass of com.uber.hoodie.utilities.schema.SchemaProvider " + - "to attach schemas to input & target table data, built in options: FilebasedSchemaProvider") + "subclass of com.uber.hoodie.utilities" + + ".schema.SchemaProvider " + + "to attach schemas to input & target" + + " table data, built in options: " + + "FilebasedSchemaProvider") public String schemaProviderClassName = FilebasedSchemaProvider.class.getName(); @Parameter(names = {"--schemaprovider-config"}, description = - "path to properties file on localfs or dfs, with schema configs. " + - "For list of acceptable properties, refer the schema provider class", required = true) + "path to properties file on localfs or dfs, with schema " + + "configs. For list of acceptable properties, refer " + + "the schema provider class", required = true) public String schemaProviderConfigProps = null; @@ -371,8 +391,9 @@ public static class Config implements Serializable { public long maxInputBytes = 1L * 1024 * 1024 * 1024 * 1024; @Parameter(names = {"--op"}, description = - "Takes one of these values : UPSERT (default), INSERT (use when input " + - "is purely new data/inserts to gain speed)", converter = OperationConvertor.class) + "Takes one of these values : UPSERT (default), INSERT (use when input " + + "is purely new data/inserts to gain speed)", + converter = OperationConvertor.class) public Operation operation = Operation.UPSERT; diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java index d9da949b561d9..a0c8e4a62c845 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java @@ -38,9 +38,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator { enum TimestampType implements Serializable { - UNIX_TIMESTAMP, - DATE_STRING, - MIXED + UNIX_TIMESTAMP, DATE_STRING, MIXED } private final TimestampType timestampType; @@ -56,9 +54,14 @@ enum TimestampType implements Serializable { static class Config { // One value from TimestampType above - private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type"; - private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformat"; - private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat"; + private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen" + + ".timebased.timestamp.type"; + private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen" + + ".timebased.input" + + ".dateformat"; + private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen" + + ".timebased.output" + + ".dateformat"; } public TimestampBasedKeyGenerator(PropertiesConfiguration config) { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java index 44745b093fd2e..aa38bf8b96adc 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java @@ -39,8 +39,12 @@ public class FilebasedSchemaProvider extends SchemaProvider { */ static class Config { - private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.source.schema.file"; - private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.target.schema.file"; + private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased" + + ".schemaprovider.source.schema" + + ".file"; + private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased" + + ".schemaprovider.target.schema" + + ".file"; } private final FileSystem fs; @@ -56,10 +60,10 @@ public FilebasedSchemaProvider(PropertiesConfiguration config) { DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP)); try { - this.sourceSchema = new Schema.Parser() - .parse(fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); - this.targetSchema = new Schema.Parser() - .parse(fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP)))); + this.sourceSchema = new Schema.Parser().parse( + fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); + this.targetSchema = new Schema.Parser().parse( + fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP)))); } catch (IOException ioe) { throw new HoodieIOException("Error reading schema", ioe); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java index f9c9a9a1dbd63..a6bd623f6ee86 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java @@ -55,10 +55,10 @@ public class DFSSource extends Source { */ static class Config { - private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; + private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; } - private final static List IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_"); + private static final List IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_"); private final transient FileSystem fs; @@ -73,9 +73,7 @@ public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, public static JavaRDD fromAvroFiles(final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) { JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, - AvroKeyInputFormat.class, - AvroKey.class, - NullWritable.class, + AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, sparkContext.hadoopConfiguration()); return avroRDD.keys().map(r -> ((GenericRecord) r.datum())); } @@ -106,28 +104,28 @@ public Pair>, String> fetchNewData( try { // obtain all eligible files under root folder. List eligibleFiles = new ArrayList<>(); - RemoteIterator fitr = fs - .listFiles(new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true); + RemoteIterator fitr = fs.listFiles( + new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true); while (fitr.hasNext()) { LocatedFileStatus fileStatus = fitr.next(); - if (fileStatus.isDirectory() || - IGNORE_FILEPREFIX_LIST.stream() - .filter(pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) { + if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream().filter( + pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) { continue; } eligibleFiles.add(fileStatus); } // sort them by modification time. eligibleFiles.sort((FileStatus f1, FileStatus f2) -> Long.valueOf(f1.getModificationTime()) - .compareTo(Long.valueOf(f2.getModificationTime()))); + .compareTo(Long.valueOf( + f2.getModificationTime()))); // Filter based on checkpoint & input size, if needed long currentBytes = 0; long maxModificationTime = Long.MIN_VALUE; List filteredFiles = new ArrayList<>(); for (FileStatus f : eligibleFiles) { - if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long - .valueOf(lastCheckpointStr.get())) { + if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long.valueOf( + lastCheckpointStr.get())) { // skip processed files continue; } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index aeecb9db014ca..2b4de59eee845 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -44,12 +44,12 @@ /** * Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit * by commit and apply to the target table - * + *

* The general idea here is to have commits sync across the data pipeline. - * + *

* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable * {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...} - * + *

* This produces beautiful causality, that makes data issues in ETLs very easy to debug */ public class HiveIncrPullSource extends Source { @@ -66,7 +66,7 @@ public class HiveIncrPullSource extends Source { */ static class Config { - private final static 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(PropertiesConfiguration config, JavaSparkContext sparkContext, @@ -121,8 +121,8 @@ public Pair>, String> fetchNewData( } // read the files out. - List commitDeltaFiles = Arrays - .asList(fs.listStatus(new Path(incrPullRootPath, commitToPull.get()))); + List commitDeltaFiles = Arrays.asList( + fs.listStatus(new Path(incrPullRootPath, commitToPull.get()))); String pathStr = commitDeltaFiles.stream().map(f -> f.getPath().toString()) .collect(Collectors.joining(",")); String schemaStr = schemaProvider.getSourceSchema().toString(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java index 36c43c5968988..f470848f1e0b1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java @@ -84,7 +84,7 @@ public static HashMap strToOffsets /** * String representation of checkpoint - * + *

* Format: topic1,0:offset0,1:offset1,2:offset2, ..... */ public static String offsetsToStr( @@ -132,9 +132,7 @@ public static long totalNewMessages(OffsetRange[] ranges) { static class ScalaHelpers { public static Map toScalaMap(HashMap m) { - return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap( - Predef.conforms() - ); + return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(Predef.conforms()); } public static Set toScalaSet(HashSet s) { @@ -152,8 +150,8 @@ public static java.util.Map toJavaMap(Map m) { */ static class Config { - private final static String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; - private final static String DEFAULT_AUTO_RESET_OFFSET = "largest"; + private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; + private static final String DEFAULT_AUTO_RESET_OFFSET = "largest"; } @@ -166,8 +164,8 @@ public KafkaSource(PropertiesConfiguration config, JavaSparkContext sparkContext super(config, sparkContext, dataFormat, schemaProvider); kafkaParams = new HashMap<>(); - Stream keys = StreamSupport - .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false); + Stream keys = StreamSupport.stream( + Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false); keys.forEach(k -> kafkaParams.put(k, config.getString(k))); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME)); @@ -180,8 +178,8 @@ public Pair>, String> fetchNewData( // Obtain current metadata for the topic KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); - Either, Set> either = cluster - .getPartitions(ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName)))); + Either, Set> either = cluster.getPartitions( + ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName)))); if (either.isLeft()) { // log errors. and bail out. throw new HoodieDeltaStreamerException("Error obtaining partition metadata", @@ -197,8 +195,8 @@ public Pair>, String> fetchNewData( String autoResetValue = config .getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET); if (autoResetValue.equals("smallest")) { - fromOffsets = new HashMap(ScalaHelpers - .toJavaMap(cluster.getEarliestLeaderOffsets(topicPartitions).right().get())); + fromOffsets = new HashMap(ScalaHelpers.toJavaMap( + cluster.getEarliestLeaderOffsets(topicPartitions).right().get())); } else if (autoResetValue.equals("largest")) { fromOffsets = new HashMap( ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); @@ -213,7 +211,8 @@ public Pair>, String> fetchNewData( ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); // Come up with final set of OffsetRanges to read (account for new partitions) - // TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from partition size + // TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from + // partition size OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); if (totalNewMsgs <= 0) { @@ -225,14 +224,8 @@ public Pair>, String> fetchNewData( } // Perform the actual read from Kafka - JavaRDD kafkaRDD = KafkaUtils.createRDD( - sparkContext, - byte[].class, - byte[].class, - DefaultDecoder.class, - DefaultDecoder.class, - kafkaParams, - offsetRanges).values(); + JavaRDD kafkaRDD = KafkaUtils.createRDD(sparkContext, byte[].class, byte[].class, + DefaultDecoder.class, DefaultDecoder.class, kafkaParams, offsetRanges).values(); // Produce a RDD[GenericRecord] final AvroConvertor avroConvertor = new AvroConvertor( @@ -241,8 +234,8 @@ public Pair>, String> fetchNewData( if (dataFormat == SourceDataFormat.AVRO) { newDataRDD = kafkaRDD.map(bytes -> avroConvertor.fromAvroBinary(bytes)); } else if (dataFormat == SourceDataFormat.JSON) { - newDataRDD = kafkaRDD - .map(bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8")))); + newDataRDD = kafkaRDD.map( + bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8")))); } else { throw new HoodieNotSupportedException("Unsupport data format :" + dataFormat); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java index d8ff58e89d561..fe7a805d4c2c1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java @@ -54,8 +54,7 @@ protected Source(PropertiesConfiguration config, JavaSparkContext sparkContext, * data, as well as the checkpoint to be written as a result of that. */ public abstract Pair>, String> fetchNewData( - Optional lastCheckpointStr, - long maxInputBytes); + Optional lastCheckpointStr, long maxInputBytes); public PropertiesConfiguration getConfig() { diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java index c1e58153fb4c5..65a5d972cb94c 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java @@ -16,7 +16,6 @@ package com.uber.hoodie.utilities; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -103,9 +102,8 @@ public void testDatasetImportWithRetries() throws Exception { createRecords(srcFolder); HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), - hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", - 1, schemaFile); + hoodieFolder.toString(), "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1, + schemaFile); AtomicInteger retry = new AtomicInteger(3); AtomicInteger fileCreated = new AtomicInteger(0); HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg) { @@ -168,8 +166,7 @@ private void createRecords(Path srcFolder) throws ParseException, IOException { .generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum, "driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } - ParquetWriter writer = AvroParquetWriter - .builder(srcFile) + ParquetWriter writer = AvroParquetWriter.builder(srcFile) .withSchema(HoodieTestDataGenerator.avroSchema) .withConf(HoodieTestUtils.getDefaultHadoopConf()) .build(); @@ -202,9 +199,8 @@ public void testSchemaFile() throws Exception { Path srcFolder = new Path(basePath.toString(), "srcTest"); Path schemaFile = new Path(basePath.toString(), "missingFile.schema"); HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), - hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", - 1, schemaFile.toString()); + hoodieFolder.toString(), "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1, + schemaFile.toString()); HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg); // Should fail - return : -1. assertEquals(-1, dataImporter.dataImport(jsc, 0)); @@ -247,16 +243,14 @@ public void testRowAndPartitionKey() throws Exception { HDFSParquetImporter.Config cfg; // Check for invalid row key. - cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "invalidRowKey", "timestamp", - 1, schemaFile.toString()); + cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable", + "COPY_ON_WRITE", "invalidRowKey", "timestamp", 1, schemaFile.toString()); dataImporter = new HDFSParquetImporter(cfg); assertEquals(-1, dataImporter.dataImport(jsc, 0)); // Check for invalid partition key. - cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "_row_key", "invalidTimeStamp", - 1, schemaFile.toString()); + cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable", + "COPY_ON_WRITE", "_row_key", "invalidTimeStamp", 1, schemaFile.toString()); dataImporter = new HDFSParquetImporter(cfg); assertEquals(-1, dataImporter.dataImport(jsc, 0)); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java index f522333393640..45344da0d2230 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java @@ -97,9 +97,9 @@ public void testSnapshotCopy() throws Exception { new File(basePath + "/2016/05/01/").mkdirs(); new File(basePath + "/2016/05/02/").mkdirs(); new File(basePath + "/2016/05/06/").mkdirs(); - HoodieTestDataGenerator.writePartitionMetadata(fs, - new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + HoodieTestDataGenerator + .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, + basePath); // Make commit1 File file11 = new File( basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, 1, "id11")); diff --git a/pom.xml b/pom.xml index 7ed46636b6e5e..f8240c4829289 100644 --- a/pom.xml +++ b/pom.xml @@ -151,6 +151,40 @@ + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.0.0 + + + com.puppycrawl.tools + checkstyle + 8.8 + + + + true + UTF-8 + style/checkstyle.xml + style/checkstyle-suppressions.xml + checkstyle.suppressions.file + true + warning + true + + ${project.build.sourceDirectory} + + **\/generated-sources\/ + + + + compile + + check + + + + org.apache.maven.plugins maven-compiler-plugin @@ -287,6 +321,7 @@ **/test/resources/*.csv **/main/avro/*.avsc **/target/* + **/style/* diff --git a/style/checkstyle-suppressions.xml b/style/checkstyle-suppressions.xml new file mode 100644 index 0000000000000..02a7a4a0f0f2d --- /dev/null +++ b/style/checkstyle-suppressions.xml @@ -0,0 +1,12 @@ + + + + + + + + + + diff --git a/style/checkstyle.xml b/style/checkstyle.xml new file mode 100644 index 0000000000000..c0667b274a5c3 --- /dev/null +++ b/style/checkstyle.xml @@ -0,0 +1,246 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/style/eclipse-java-google-style.xml b/style/eclipse-java-google-style.xml new file mode 100644 index 0000000000000..d1af2890be3be --- /dev/null +++ b/style/eclipse-java-google-style.xml @@ -0,0 +1,337 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/style/intellij-java-google-style.xml b/style/intellij-java-google-style.xml new file mode 100644 index 0000000000000..658087ef6ca4d --- /dev/null +++ b/style/intellij-java-google-style.xml @@ -0,0 +1,598 @@ + + + + + +

+ + + + xmlns:android + + ^$ + + + +
+
+ + + + xmlns:.* + + ^$ + + + BY_NAME + +
+
+ + + + .*:id + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + style + + ^$ + + + +
+
+ + + + .* + + ^$ + + + BY_NAME + +
+
+ + + + .*:.*Style + + http://schemas.android.com/apk/res/android + + + BY_NAME + +
+
+ + + + .*:layout_width + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_height + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_weight + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_margin + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginTop + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginBottom + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginStart + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginEnd + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginLeft + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginRight + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_.* + + http://schemas.android.com/apk/res/android + + + BY_NAME + +
+
+ + + + .*:padding + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingTop + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingBottom + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingStart + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingEnd + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingLeft + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingRight + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .* + http://schemas.android.com/apk/res/android + + + BY_NAME + +
+
+ + + + .* + http://schemas.android.com/apk/res-auto + + + BY_NAME + +
+
+ + + + .* + http://schemas.android.com/tools + + + BY_NAME + +
+
+ + + + .* + .* + + + BY_NAME + +
+ + + + + + + + + + From 69bd01e2588b25e4c3fa9af928cf707af8787f1b Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Mon, 2 Apr 2018 09:29:15 -0700 Subject: [PATCH 042/374] Update release notes for 0.4.1 (post) --- RELEASE_NOTES.md | 38 +++++++++++++++++++++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index f67d0951f77e8..da9bce008ed01 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -1,4 +1,40 @@ -Release 0.4.0 +Release 0.4.1 +------------------------------------ + +### Highlights + + * Good enhancements for merge-on-read write path : spillable map for merging, evolvable log format, rollback support + * Cloud file systems should now work out-of-box for copy-on-write tables, with configs picked up from SparkContext + * Compaction action is no more, multiple delta commits now lead to a commit upon compaction + * API level changes include : compaction api, new prepped APIs for higher plugability for advanced clients + + +### Full PR List + + * **@n3nash** - Separated rollback as a table operation, implement rollback for MOR #247 + * **@n3nash** - Implementing custom payload/merge hooks abstractions for application #275 + * **@vinothchandar** - Reformat project & tighten code style guidelines #280 + * **@n3nash** - Separating out compaction() API #282 + * **@n3nash** - Enable hive sync even if there is no compaction commit #286 + * **@n3nash** - Partition compaction strategy #281 + * **@n3nash** - Removing compaction action type and associated compaction timeline operations, replace with commit action type #288 + * **@vinothchandar** - Multi/Cloud FS Support for Copy-On-Write tables #293 + * **@vinothchandar** - Update Gemfile.lock #298 + * **@n3nash** - Reducing memory footprint required in HoodieAvroDataBlock and HoodieAppendHandle #290 + * **@jianxu** - Add FinalizeWrite in HoodieCreateHandle for COW tables #285 + * **@n3nash** - Adding global indexing to HbaseIndex implementation #318 + * **@n3nash** - Small File Size correction handling for MOR table type #299 + * **@jianxu** - Use FastDateFormat for thread safety #320 + * **@vinothchandar** - Fix formatting in HoodieWriteClient #322 + * **@n3nash** - Write smaller sized multiple blocks to log file instead of a large one #317 + * **@n3nash** - Added support for Disk Spillable Compaction to prevent OOM issues #289 + * **@jianxu** - Add new APIs in HoodieReadClient and HoodieWriteClient #327 + * **@jianxu** - Handle inflight clean instants during Hoodie instants archiving #332 + * **@n3nash** - Introducing HoodieLogFormat V2 with versioning support #331 + * **@n3nash** - Re-factoring Compaction as first level API in WriteClient similar to upsert/insert #330 + + +Release 0.4.0 ------------------------------------ ### Highlights From 7b5cab0a187c4a4cdfa35953f323734de4117fb7 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Mon, 2 Apr 2018 10:08:06 -0700 Subject: [PATCH 043/374] Issue-329 : Refactoring TestHoodieClientOnCopyOnWriteStorage and adding test-cases --- .../java/com/uber/hoodie/TestCleaner.java | 700 +++++++++ .../com/uber/hoodie/TestClientRollback.java | 320 ++++ .../com/uber/hoodie/TestHoodieClientBase.java | 436 ++++++ .../TestHoodieClientOnCopyOnWriteStorage.java | 1378 +++-------------- .../com/uber/hoodie/TestHoodieReadClient.java | 195 +++ .../hoodie/common/HoodieClientTestUtils.java | 2 +- .../common/HoodieTestDataGenerator.java | 74 +- .../hoodie/common/model/HoodieTestUtils.java | 13 + 8 files changed, 1963 insertions(+), 1155 deletions(-) create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java new file mode 100644 index 0000000000000..086818274c938 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -0,0 +1,700 @@ +/* + * 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; + +import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; +import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; +import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.common.collect.Iterables; +import com.uber.hoodie.common.HoodieCleanStat; +import com.uber.hoodie.common.model.HoodieCleaningPolicy; +import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroup; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.model.HoodieWriteStat; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.scheduler.SparkListener; +import org.apache.spark.scheduler.SparkListenerTaskEnd; +import org.apache.spark.util.AccumulatorV2; +import org.junit.Test; +import scala.Option; +import scala.collection.Iterator; + +/** + * Test Cleaning related logic + */ +public class TestCleaner extends TestHoodieClientBase { + + private static final int BIG_BATCH_INSERT_SIZE = 500; + private static Logger logger = LogManager.getLogger(TestHoodieClientBase.class); + + /** + * Helper method to do first batch of insert for clean by versions/commits tests + * + * @param cfg Hoodie Write Config + * @param client Hoodie Client + * @param recordGenFunction Function to generate records for insertion + * @param insertFn Insertion API for testing + * @throws Exception in case of error + */ + private void insertFirstBigBatchForClientCleanerTest( + HoodieWriteConfig cfg, + HoodieWriteClient client, + Function2, String, Integer> recordGenFunction, + Function3, HoodieWriteClient, JavaRDD, String> insertFn) throws Exception { + + /** + * do a big insert + * (this is basically same as insert part of upsert, just adding it here so we can + * catch breakages in insert(), if the implementation diverges.) + */ + HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); + String newCommitTime = client.startCommit(); + + List records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE); + JavaRDD writeRecords = jsc.parallelize(records, 5); + + List statuses = insertFn.apply(client, writeRecords, newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + // verify that there is a commit + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + // Should have 100 records in table (check using Index), all in locations marked at commit + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + + assertFalse(table.getCompletedCommitTimeline().empty()); + String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); + assertFalse(table.getCompletedCleanTimeline().empty()); + assertEquals("The clean instant should be the same as the commit instant", commitTime, + table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); + + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); + checkTaggedRecords(taggedRecords, newCommitTime); + } + + /** + * Test Clean-By-Versions using insert/upsert API + */ + @Test + public void testInsertAndCleanByVersions() throws Exception { + testInsertAndCleanByVersions(HoodieWriteClient::insert, HoodieWriteClient::upsert, false); + } + + /** + * Test Clean-By-Versions using prepped versions of insert/upsert API + */ + @Test + public void testInsertPreppedAndCleanByVersions() throws Exception { + testInsertAndCleanByVersions(HoodieWriteClient::insertPreppedRecords, + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Test Clean-By-Versions using bulk-insert/upsert API + */ + @Test + public void testBulkInsertAndCleanByVersions() throws Exception { + testInsertAndCleanByVersions(HoodieWriteClient::bulkInsert, HoodieWriteClient::upsert, false); + } + + /** + * Test Clean-By-Versions using prepped versions of bulk-insert/upsert API + */ + @Test + public void testBulkInsertPreppedAndCleanByVersions() throws Exception { + testInsertAndCleanByVersions( + (client, recordRDD, commitTime) -> client.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()), + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective + * + * @param insertFn Insert API to be tested + * @param upsertFn Upsert API to be tested + * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * @throws Exception in case of errors + */ + private void testInsertAndCleanByVersions( + Function3, HoodieWriteClient, JavaRDD, String> insertFn, + Function3, HoodieWriteClient, JavaRDD, String> upsertFn, + boolean isPreppedAPI + ) throws Exception { + int maxVersions = 2; // keep upto 2 versions for each file + HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + .retainFileVersions(maxVersions).build()) + .withParallelism(1, 1).withBulkInsertParallelism(1) + .build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + + final Function2, String, Integer> recordInsertGenWrappedFunction = + generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); + + final Function2, String, Integer> recordUpsertGenWrappedFunction = + generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateUniqueUpdates); + + insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn); + + // Keep doing some writes and clean inline. Make sure we have expected number of files + // remaining. + HoodieTestUtils.monotonicIncreasingCommitTimestamps(8, 1).stream().forEach(newCommitTime -> { + try { + client.startCommitWithTime(newCommitTime); + List records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100); + + List statuses = + upsertFn.apply(client, jsc.parallelize(records, 1), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig()); + HoodieTimeline timeline = table.getCommitsTimeline(); + + TableFileSystemView fsView = table.getFileSystemView(); + // Need to ensure the following + for (String partitionPath : dataGen.getPartitionPaths()) { + // compute all the versions of all files, from time 0 + HashMap> fileIdToVersions = new HashMap<>(); + for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(entry).get()); + + for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) { + if (!fileIdToVersions.containsKey(wstat.getFileId())) { + fileIdToVersions.put(wstat.getFileId(), new TreeSet<>()); + } + fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName())); + } + } + + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); + + for (HoodieFileGroup fileGroup : fileGroups) { + // No file has no more than max versions + String fileId = fileGroup.getId(); + List dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList()); + + assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions", + dataFiles.size() <= maxVersions); + + // Each file, has the latest N versions (i.e cleaning gets rid of older versions) + List commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId)); + for (int i = 0; i < dataFiles.size(); i++) { + assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions, + Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i)); + } + } + } + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + }); + } + + /** + * Test Clean-By-Versions using insert/upsert API + */ + @Test + public void testInsertAndCleanByCommits() throws Exception { + testInsertAndCleanByCommits(HoodieWriteClient::insert, HoodieWriteClient::upsert, false); + } + + /** + * Test Clean-By-Versions using prepped version of insert/upsert API + */ + @Test + public void testInsertPreppedAndCleanByCommits() throws Exception { + testInsertAndCleanByCommits(HoodieWriteClient::insertPreppedRecords, + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Test Clean-By-Versions using prepped versions of bulk-insert/upsert API + */ + @Test + public void testBulkInsertPreppedAndCleanByCommits() throws Exception { + testInsertAndCleanByCommits( + (client, recordRDD, commitTime) -> client.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()), + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Test Clean-By-Versions using bulk-insert/upsert API + */ + @Test + public void testBulkInsertAndCleanByCommits() throws Exception { + testInsertAndCleanByCommits(HoodieWriteClient::bulkInsert, HoodieWriteClient::upsert, false); + } + + /** + * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective + * + * @param insertFn Insert API to be tested + * @param upsertFn Upsert API to be tested + * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * @throws Exception in case of errors + */ + private void testInsertAndCleanByCommits( + Function3, HoodieWriteClient, JavaRDD, String> insertFn, + Function3, HoodieWriteClient, JavaRDD, String> upsertFn, + boolean isPreppedAPI + ) throws Exception { + int maxCommits = 3; // keep upto 3 commits from the past + HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( + HoodieCompactionConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build()) + .withParallelism(1, 1).withBulkInsertParallelism(1).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + + final Function2, String, Integer> recordInsertGenWrappedFunction = + generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); + + final Function2, String, Integer> recordUpsertGenWrappedFunction = + generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateUniqueUpdates); + + insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn); + + // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. + HoodieTestUtils.monotonicIncreasingCommitTimestamps(8, 1).stream().forEach(newCommitTime -> { + try { + client.startCommitWithTime(newCommitTime); + List records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100); + + List statuses = + upsertFn.apply(client, jsc.parallelize(records, 1), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); + HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); + Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); + Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); + if (earliestRetainedCommit.isPresent()) { + acceptableCommits.removeAll( + activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()).getInstants() + .collect(Collectors.toSet())); + acceptableCommits.add(earliestRetainedCommit.get()); + } + + TableFileSystemView fsView = table1.getFileSystemView(); + // Need to ensure the following + for (String partitionPath : dataGen.getPartitionPaths()) { + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); + for (HoodieFileGroup fileGroup : fileGroups) { + Set commitTimes = new HashSet<>(); + fileGroup.getAllDataFiles().forEach(value -> { + logger.debug("Data File - " + value); + commitTimes.add(value.getCommitTime()); + }); + assertEquals("Only contain acceptable versions of file should be present", + acceptableCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), commitTimes); + } + } + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + }); + } + + /** + * Test HoodieTable.clean() Cleaning by versions logic + */ + @Test + public void testKeepLatestFileVersions() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); + + // make 1 commit, with 1 file per partition + HoodieTestUtils.createCommitFiles(basePath, "000"); + + String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); + String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + + List hoodieCleanStatsOne = table.clean(jsc); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsOne, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsOne, DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_SECOND_PARTITION_PATH, "000", file1P1C0)); + + // make next commit, with 1 insert & 1 update per partition + HoodieTestUtils.createCommitFiles(basePath, "001"); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config); + + String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert + String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0); // update + HoodieTestUtils.createDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001", file1P1C0); // update + + List hoodieCleanStatsTwo = table.clean(jsc); + assertEquals("Must clean 1 file", 1, + getCleanStat(hoodieCleanStatsTwo, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertEquals("Must clean 1 file", 1, + getCleanStat(hoodieCleanStatsTwo, DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file2P0C1)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_SECOND_PARTITION_PATH, "001", file2P1C1)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0C0)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_SECOND_PARTITION_PATH, "000", file1P1C0)); + + // make next commit, with 2 updates to existing files, and 1 insert + HoodieTestUtils.createCommitFiles(basePath, "002"); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update + String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002"); + + List hoodieCleanStatsThree = table.clean(jsc); + assertEquals("Must clean two files", 2, + getCleanStat(hoodieCleanStatsThree, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file2P0C1)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file3P0C2)); + + // No cleaning on partially written file, with no commit. + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file3P0C2); // update + List hoodieCleanStatsFour = table.clean(jsc); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsFour, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file3P0C2)); + } + + /** + * Test HoodieTable.clean() Cleaning by versions logic for MOR table with Log files + */ + @Test + public void testKeepLatestFileVersionsMOR() throws IOException { + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); + + HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, + HoodieTableType.MERGE_ON_READ); + + // Make 3 files, one base file and 2 log files associated with base file + String file1P0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); + String file2P0L0 = HoodieTestUtils + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Optional.empty()); + String file2P0L1 = HoodieTestUtils + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Optional.of(2)); + // make 1 compaction commit + HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000"); + + // Make 4 files, one base file and 3 log files associated with base file + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0); + file2P0L0 = HoodieTestUtils + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.empty()); + file2P0L0 = HoodieTestUtils + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.of(2)); + file2P0L0 = HoodieTestUtils + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.of(3)); + // make 1 compaction commit + HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); + + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + List hoodieCleanStats = table.clean(jsc); + assertEquals("Must clean three files, one parquet and 2 log files", 3, + getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0)); + assertFalse( + HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Optional.empty())); + assertFalse( + HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Optional.of(2))); + } + + /** + * Test HoodieTable.clean() Cleaning by commit logic for MOR table with Log files + */ + @Test + public void testKeepLatestCommits() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); + + // make 1 commit, with 1 file per partition + HoodieTestUtils.createCommitFiles(basePath, "000"); + + String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); + String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); + + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + + List hoodieCleanStatsOne = table.clean(jsc); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsOne, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsOne, DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_SECOND_PARTITION_PATH, "000", file1P1C0)); + + // make next commit, with 1 insert & 1 update per partition + HoodieTestUtils.createCommitFiles(basePath, "001"); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert + String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0); // update + HoodieTestUtils.createDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001", file1P1C0); // update + + List hoodieCleanStatsTwo = table.clean(jsc); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsTwo, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsTwo, DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file2P0C1)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_SECOND_PARTITION_PATH, "001", file2P1C1)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_SECOND_PARTITION_PATH, "000", file1P1C0)); + + // make next commit, with 2 updates to existing files, and 1 insert + HoodieTestUtils.createCommitFiles(basePath, "002"); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update + String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002"); + + List hoodieCleanStatsThree = table.clean(jsc); + assertEquals("Must not clean any file. We have to keep 1 version before the latest commit time to keep", 0, + getCleanStat(hoodieCleanStatsThree, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0C0)); + + // make next commit, with 2 updates to existing files, and 1 insert + HoodieTestUtils.createCommitFiles(basePath, "003"); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file1P0C0); // update + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update + String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003"); + + List hoodieCleanStatsFour = table.clean(jsc); + assertEquals("Must not clean one old file", 1, + getCleanStat(hoodieCleanStatsFour, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file2P0C1)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file3P0C2)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file4P0C3)); + + // No cleaning on partially written file, with no commit. + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "004", file3P0C2); // update + List hoodieCleanStatsFive = table.clean(jsc); + assertEquals("Must not clean any files", 0, + getCleanStat(hoodieCleanStatsFive, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file2P0C1)); + } + + /** + * Test Cleaning functionality of table.rollback() API. + */ + @Test + public void testCleanTemporaryDataFilesOnRollback() throws IOException { + HoodieTestUtils.createCommitFiles(basePath, "000"); + List tempFiles = createTempFiles("000", 10); + assertEquals("Some temp files are created.", 10, tempFiles.size()); + assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withUseTempFolderCopyOnWriteForCreate(false) + .withUseTempFolderCopyOnWriteForMerge(false).build(); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + table.rollback(jsc, Collections.emptyList()); + assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); + + config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(false).build(); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); + table.rollback(jsc, Collections.emptyList()); + assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); + } + + /** + * Test CLeaner Stat when there are no partition paths. + */ + @Test + public void testCleaningWithZeroPartitonPaths() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); + + // Make a commit, although there are no partitionPaths. + // Example use-case of this is when a client wants to create a table + // with just some commit metadata, but no data/partitionPaths. + HoodieTestUtils.createCommitFiles(basePath, "000"); + + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + + List hoodieCleanStatsOne = table.clean(jsc); + assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); + } + + /** + * Test Clean-by-commits behavior in the presence of skewed partitions + */ + @Test + public void testCleaningSkewedPartitons() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); + Map stageOneShuffleReadTaskRecordsCountMap = new HashMap<>(); + + // Since clean involves repartition in order to uniformly distribute data, + // we can inspect the number of records read by various tasks in stage 1. + // There should not be skew in the number of records read in the task. + + // SparkListener below listens to the stage end events and captures number of + // records read by various tasks in stage-1. + jsc.sc().addSparkListener(new SparkListener() { + + @Override + public void onTaskEnd(SparkListenerTaskEnd taskEnd) { + + Iterator> iterator = taskEnd.taskMetrics().accumulators().iterator(); + while (iterator.hasNext()) { + AccumulatorV2 accumulator = iterator.next(); + if (taskEnd.stageId() == 1 && accumulator.isRegistered() && accumulator.name().isDefined() + && accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) { + stageOneShuffleReadTaskRecordsCountMap.put(taskEnd.taskInfo().taskId(), (Long) accumulator.value()); + } + } + } + }); + + // make 1 commit, with 100 files in one partition and 10 in other two + HoodieTestUtils.createCommitFiles(basePath, "000"); + List filesP0C0 = createFilesInPartition(DEFAULT_FIRST_PARTITION_PATH, "000", 100); + List filesP1C0 = createFilesInPartition(DEFAULT_SECOND_PARTITION_PATH, "000", 10); + List filesP2C0 = createFilesInPartition(DEFAULT_THIRD_PARTITION_PATH, "000", 10); + + HoodieTestUtils.createCommitFiles(basePath, "001"); + updateAllFilesInPartition(filesP0C0, DEFAULT_FIRST_PARTITION_PATH, "001"); + updateAllFilesInPartition(filesP1C0, DEFAULT_SECOND_PARTITION_PATH, "001"); + updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "001"); + + HoodieTestUtils.createCommitFiles(basePath, "002"); + updateAllFilesInPartition(filesP0C0, DEFAULT_FIRST_PARTITION_PATH, "002"); + updateAllFilesInPartition(filesP1C0, DEFAULT_SECOND_PARTITION_PATH, "002"); + updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "002"); + + HoodieTestUtils.createCommitFiles(basePath, "003"); + updateAllFilesInPartition(filesP0C0, DEFAULT_FIRST_PARTITION_PATH, "003"); + updateAllFilesInPartition(filesP1C0, DEFAULT_SECOND_PARTITION_PATH, "003"); + updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "003"); + + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + List hoodieCleanStats = table.clean(jsc); + + assertEquals(100, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertEquals(10, getCleanStat(hoodieCleanStats, DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().size()); + assertEquals(10, getCleanStat(hoodieCleanStats, DEFAULT_THIRD_PARTITION_PATH).getSuccessDeleteFiles().size()); + + // 3 tasks are expected since the number of partitions is 3 + assertEquals(3, stageOneShuffleReadTaskRecordsCountMap.keySet().size()); + // Sum of all records processed = total number of files to clean + assertEquals(120, + stageOneShuffleReadTaskRecordsCountMap.values().stream().reduce((a, b) -> a + b).get().intValue()); + assertTrue("The skew in handling files to clean is not removed. " + + "Each task should handle more records than the partitionPath with least files " + + "and less records than the partitionPath with most files.", + stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3); + } + + /** + * Utility method to create temporary data files + * + * @param commitTime Commit Timestamp + * @param numFiles Number for files to be generated + * @return generated files + * @throws IOException in case of error + */ + private List createTempFiles(String commitTime, int numFiles) throws IOException { + List files = new ArrayList<>(); + for (int i = 0; i < numFiles; i++) { + files.add(HoodieTestUtils.createNewDataFile(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, commitTime)); + } + return files; + } + + /*** + * Helper method to return temporary files count + * @return Number of temporary files found + * @throws IOException in case of error + */ + private int getTotalTempFiles() throws IOException { + return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length; + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java new file mode 100644 index 0000000000000..6d9dea7b4803d --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java @@ -0,0 +1,320 @@ +/* + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieCleaningPolicy; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieRollbackException; +import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.table.HoodieTable; +import java.io.File; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Test; + +/** + * Test Cases for rollback of snapshots and commits + */ +public class TestClientRollback extends TestHoodieClientBase { + + /** + * Test case for rollback-savepoint interaction + */ + @Test + public void testSavepointAndRollback() throws Exception { + HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) + .build()).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + + /** + * Write 1 (only inserts) + */ + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + List statuses = client.upsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + + /** + * Write 2 (updates) + */ + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + + records = dataGen.generateUpdates(newCommitTime, records); + statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + client.savepoint("hoodie-unit-test", "test"); + + /** + * Write 3 (updates) + */ + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + + records = dataGen.generateUpdates(newCommitTime, records); + statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), + getConfig().shouldAssumeDatePartitioning()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); + + List dataFiles = partitionPaths.stream().flatMap(s -> { + return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003")); + }).collect(Collectors.toList()); + assertEquals("The data files for commit 003 should be present", 3, dataFiles.size()); + + dataFiles = partitionPaths.stream().flatMap(s -> { + return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); + }).collect(Collectors.toList()); + assertEquals("The data files for commit 002 should be present", 3, dataFiles.size()); + + /** + * Write 4 (updates) + */ + newCommitTime = "004"; + client.startCommitWithTime(newCommitTime); + + records = dataGen.generateUpdates(newCommitTime, records); + statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient, getConfig()); + final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); + + dataFiles = partitionPaths.stream().flatMap(s -> { + return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004")); + }).collect(Collectors.toList()); + assertEquals("The data files for commit 004 should be present", 3, dataFiles.size()); + + // rolling back to a non existent savepoint must not succeed + try { + client.rollbackToSavepoint("001"); + fail("Rolling back to non-existent savepoint should not be allowed"); + } catch (HoodieRollbackException e) { + // this is good + } + + // rollback to savepoint 002 + HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get(); + client.rollbackToSavepoint(savepoint.getTimestamp()); + + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient, getConfig()); + final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); + dataFiles = partitionPaths.stream().flatMap(s -> { + return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); + }).collect(Collectors.toList()); + assertEquals("The data files for commit 002 be available", 3, dataFiles.size()); + + dataFiles = partitionPaths.stream().flatMap(s -> { + return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003")); + }).collect(Collectors.toList()); + assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size()); + + dataFiles = partitionPaths.stream().flatMap(s -> { + return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004")); + }).collect(Collectors.toList()); + assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size()); + } + + /** + * Test Cases for effects of rollbacking completed/inflight commits + */ + @Test + public void testRollbackCommit() throws Exception { + // Let's create some commit files and parquet files + String commitTime1 = "20160501010101"; + String commitTime2 = "20160502020601"; + String commitTime3 = "20160506030611"; + new File(basePath + "/.hoodie").mkdirs(); + HoodieTestDataGenerator + .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); + + // Only first two have commit files + HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2); + // Third one has a .inflight intermediate commit file + HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3); + + // Make commit1 + String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11"); + String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12"); + String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13"); + + // Make commit2 + String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21"); + String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22"); + String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23"); + + // Make commit3 + String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31"); + String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32"); + String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + + HoodieWriteClient client = new HoodieWriteClient(jsc, config, false); + + // Rollback commit 1 (this should fail, since commit2 is still around) + try { + client.rollback(commitTime1); + assertTrue("Should have thrown an exception ", false); + } catch (HoodieRollbackException hrbe) { + // should get here + } + + // Rollback commit3 + client.rollback(commitTime3); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + + // simulate partial failure, where .inflight was not deleted, but data files were. + HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3); + client.rollback(commitTime3); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); + + // Rollback commit2 + client.rollback(commitTime2); + assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + + // simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a + // .inflight commit and a bunch of data files around. + HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2); + file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21"); + file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22"); + file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23"); + + client.rollback(commitTime2); + assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + + // Let's rollback commit1, Check results + client.rollback(commitTime1); + assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + } + + /** + * Test auto-rollback of commits which are in flight + */ + @Test + public void testAutoRollbackInflightCommit() throws Exception { + // Let's create some commit files and parquet files + String commitTime1 = "20160501010101"; + String commitTime2 = "20160502020601"; + String commitTime3 = "20160506030611"; + new File(basePath + "/.hoodie").mkdirs(); + HoodieTestDataGenerator + .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); + + // One good commit + HoodieTestUtils.createCommitFiles(basePath, commitTime1); + // Two inflight commits + HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2, commitTime3); + + // Make commit1 + String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11"); + String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12"); + String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13"); + + // Make commit2 + String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21"); + String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22"); + String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23"); + + // Make commit3 + String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31"); + String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32"); + String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); + + // Turn auto rollback off + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + + new HoodieWriteClient(jsc, config, false); + + // Check results, nothing changed + assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); + assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); + assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + + // Turn auto rollback on + new HoodieWriteClient(jsc, config, true); + assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); + assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java new file mode 100644 index 0000000000000..a9e374c403850 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -0,0 +1,436 @@ +/* + * 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; + +import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_PARTITION_DEPTH; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.uber.hoodie.common.HoodieCleanStat; +import com.uber.hoodie.common.HoodieClientTestUtils; +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodiePartitionMetadata; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieStorageConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.table.HoodieTable; +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; +import org.junit.After; +import org.junit.Before; +import org.junit.rules.TemporaryFolder; + +/** + * Base Class providing setup/cleanup and utility methods for testing Hoodie Client facing tests + */ +public class TestHoodieClientBase implements Serializable { + + protected transient JavaSparkContext jsc = null; + protected transient SQLContext sqlContext; + protected transient FileSystem fs; + protected String basePath = null; + protected transient HoodieTestDataGenerator dataGen = null; + + @Before + public void init() throws IOException { + // Initialize a local spark env + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient")); + + //SQLContext stuff + sqlContext = new SQLContext(jsc); + + // Create a temp folder as the base path + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + dataGen = new HoodieTestDataGenerator(); + } + + /** + * Get Default HoodieWriteConfig for tests + * + * @return Default Hoodie Write Config for tests + */ + protected HoodieWriteConfig getConfig() { + return getConfigBuilder().build(); + } + + /** + * Get Config builder with default configs set + * + * @return Config Builder + */ + HoodieWriteConfig.Builder getConfigBuilder() { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + } + + /** + * Assert no failures in writing hoodie files + * + * @param statuses List of Write Status + */ + void assertNoWriteErrors(List statuses) { + // Verify there are no errors + for (WriteStatus status : statuses) { + assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors()); + } + } + + /** + * Ensure presence of partition meta-data at known depth + * + * @param partitionPaths Partition paths to check + * @param fs File System + * @throws IOException in case of error + */ + void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException { + for (String partitionPath : partitionPaths) { + assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath))); + HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath)); + pmeta.readFromFS(); + assertEquals(DEFAULT_PARTITION_DEPTH, pmeta.getPartitionDepth()); + } + } + + /** + * Ensure records have location field set + * + * @param taggedRecords Tagged Records + * @param commitTime Commit Timestamp + */ + void checkTaggedRecords(List taggedRecords, String commitTime) { + for (HoodieRecord rec : taggedRecords) { + assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown()); + assertEquals("All records should have commit time " + commitTime + ", since updates were made", + rec.getCurrentLocation().getCommitTime(), commitTime); + } + } + + /** + * Assert that there is no duplicate key at the partition level + * + * @param records List of Hoodie records + */ + void assertNodupesWithinPartition(List records) { + Map> partitionToKeys = new HashMap<>(); + for (HoodieRecord r : records) { + String key = r.getRecordKey(); + String partitionPath = r.getPartitionPath(); + if (!partitionToKeys.containsKey(partitionPath)) { + partitionToKeys.put(partitionPath, new HashSet<>()); + } + assertTrue("key " + key + " is duplicate within partition " + partitionPath, + !partitionToKeys.get(partitionPath).contains(key)); + partitionToKeys.get(partitionPath).add(key); + } + } + + /** + * Helper to generate records generation function for testing Prepped version of API. Prepped APIs expect the records + * to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is + * guaranteed by record-generation function itself. + * + * @param writeConfig Hoodie Write Config + * @param recordGenFunction Records Generation function + * @return Wrapped function + */ + private Function2, String, Integer> wrapRecordsGenFunctionForPreppedCalls( + final HoodieWriteConfig writeConfig, + final Function2, String, Integer> recordGenFunction) { + return (commit, numRecords) -> { + final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc); + List records = recordGenFunction.apply(commit, numRecords); + final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); + HoodieTable.getHoodieTable(metaClient, writeConfig); + JavaRDD taggedRecords = + index.tagLocation(jsc.parallelize(records, 1), HoodieTable.getHoodieTable(metaClient, writeConfig)); + return taggedRecords.collect(); + }; + } + + /** + * Generate wrapper for record generation function for testing Prepped APIs + * + * @param isPreppedAPI Flag to indicate if this is for testing prepped-version of APIs + * @param writeConfig Hoodie Write Config + * @param wrapped Actual Records Generation function + * @return Wrapped Function + */ + Function2, String, Integer> generateWrapRecordsFn( + boolean isPreppedAPI, + HoodieWriteConfig writeConfig, + Function2, String, Integer> wrapped) { + if (isPreppedAPI) { + return wrapRecordsGenFunctionForPreppedCalls(writeConfig, wrapped); + } else { + return wrapped; + } + } + + /** + * Helper to insert first batch of records and do regular assertions on the state after successful completion + * + * @param writeConfig Hoodie Write Config + * @param client Hoodie Write Client + * @param newCommitTime New Commit Timestamp to be used + * @param initCommitTime Begin Timestamp (usually "000") + * @param numRecordsInThisCommit Number of records to be added in the new commit + * @param writeFn Write Function to be used for insertion + * @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records + * @param assertForCommit Enable Assertion of Writes + * @param expRecordsInThisCommit Expected number of records in this commit + * @return RDD of write-status + * @throws Exception in case of error + */ + JavaRDD insertFirstBatch( + HoodieWriteConfig writeConfig, + HoodieWriteClient client, + String newCommitTime, + String initCommitTime, + int numRecordsInThisCommit, + Function3, HoodieWriteClient, JavaRDD, String> writeFn, + boolean isPreppedAPI, + boolean assertForCommit, + int expRecordsInThisCommit) throws Exception { + final Function2, String, Integer> recordGenFunction = + generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); + + return writeBatch(client, newCommitTime, initCommitTime, Optional.empty(), initCommitTime, + numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, + expRecordsInThisCommit, expRecordsInThisCommit, 1); + } + + /** + * Helper to upsert batch of records and do regular assertions on the state after successful completion + * + * @param writeConfig Hoodie Write Config + * @param client Hoodie Write Client + * @param newCommitTime New Commit Timestamp to be used + * @param prevCommitTime Commit Timestamp used in previous commit + * @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime + * @param initCommitTime Begin Timestamp (usually "000") + * @param numRecordsInThisCommit Number of records to be added in the new commit + * @param writeFn Write Function to be used for upsert + * @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records + * @param assertForCommit Enable Assertion of Writes + * @param expRecordsInThisCommit Expected number of records in this commit + * @param expTotalRecords Expected number of records when scanned + * @param expTotalCommits Expected number of commits (including this commit) + * @return RDD of write-status + * @throws Exception in case of error + */ + JavaRDD updateBatch( + HoodieWriteConfig writeConfig, + HoodieWriteClient client, + String newCommitTime, + String prevCommitTime, + Optional> commitTimesBetweenPrevAndNew, + String initCommitTime, + int numRecordsInThisCommit, + Function3, HoodieWriteClient, JavaRDD, String> writeFn, + boolean isPreppedAPI, + boolean assertForCommit, + int expRecordsInThisCommit, + int expTotalRecords, + int expTotalCommits) + throws Exception { + final Function2, String, Integer> recordGenFunction = + generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); + + return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, + numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, + expRecordsInThisCommit, expTotalRecords, expTotalCommits); + } + + /** + * Helper to insert/upsert batch of records and do regular assertions on the state after successful completion + * + * @param client Hoodie Write Client + * @param newCommitTime New Commit Timestamp to be used + * @param prevCommitTime Commit Timestamp used in previous commit + * @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime + * @param initCommitTime Begin Timestamp (usually "000") + * @param numRecordsInThisCommit Number of records to be added in the new commit + * @param recordGenFunction Records Generation Function + * @param writeFn Write Function to be used for upsert + * @param assertForCommit Enable Assertion of Writes + * @param expRecordsInThisCommit Expected number of records in this commit + * @param expTotalRecords Expected number of records when scanned + * @param expTotalCommits Expected number of commits (including this commit) + * @throws Exception in case of error + */ + JavaRDD writeBatch( + HoodieWriteClient client, + String newCommitTime, + String prevCommitTime, + Optional> commitTimesBetweenPrevAndNew, + String initCommitTime, + int numRecordsInThisCommit, + Function2, String, Integer> recordGenFunction, + Function3, HoodieWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, + int expRecordsInThisCommit, + int expTotalRecords, + int expTotalCommits) + throws Exception { + + //Write 1 (only inserts) + client.startCommitWithTime(newCommitTime); + + List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + JavaRDD result = writeFn.apply(client, writeRecords, newCommitTime); + List statuses = result.collect(); + assertNoWriteErrors(statuses); + + // check the partition metadata is written out + assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); + + // verify that there is a commit + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + + if (assertForCommit) { + assertEquals("Expecting " + expTotalCommits + " commits.", expTotalCommits, + timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be " + newCommitTime, newCommitTime, + timeline.lastInstant().get().getTimestamp()); + assertEquals("Must contain " + expRecordsInThisCommit + " records", expRecordsInThisCommit, + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); + + // Check the entire dataset has all records still + String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; + for (int i = 0; i < fullPartitionPaths.length; i++) { + fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); + } + assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords, + HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); + + // Check that the incremental consumption from prevCommitTime + assertEquals("Incremental consumption from " + prevCommitTime + + " should give all records in latest commit", + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, prevCommitTime).count()); + if (commitTimesBetweenPrevAndNew.isPresent()) { + commitTimesBetweenPrevAndNew.get().forEach(ct -> { + assertEquals("Incremental consumption from " + ct + " should give all records in latest commit", + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, ct).count()); + }); + } + } + return result; + } + + @After + public void clean() { + if (basePath != null) { + new File(basePath).delete(); + } + if (jsc != null) { + jsc.stop(); + } + } + + /** + * Get Cleaner state corresponding to a partition path + * + * @param hoodieCleanStatsTwo List of Clean Stats + * @param partitionPath Partition path for filtering + * @return Cleaner state corresponding to partition path + */ + HoodieCleanStat getCleanStat(List hoodieCleanStatsTwo, String partitionPath) { + return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().get(); + } + + /** + * Utility to simulate commit touching files in a partition + * + * @param files List of file-Ids to be touched + * @param partitionPath Partition + * @param commitTime Commit Timestamp + * @throws IOException in case of error + */ + void updateAllFilesInPartition(List files, String partitionPath, String commitTime) + throws IOException { + for (String fileId : files) { + HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId); + } + } + + /** + * Helper methods to create new data files in a partition + * + * @param partitionPath Partition + * @param commitTime Commit Timestamp + * @param numFiles Number of files to be added + * @return Created files + * @throws IOException in case of error + */ + List createFilesInPartition(String partitionPath, String commitTime, int numFiles) + throws IOException { + List files = new ArrayList<>(); + for (int i = 0; i < numFiles; i++) { + files.add(HoodieTestUtils.createNewDataFile(basePath, partitionPath, commitTime)); + } + return files; + } + + // Functional Interfaces for passing lambda and Hoodie Write API contexts + + @FunctionalInterface + interface Function2 { + + R apply(T1 v1, T2 v2) throws IOException; + } + + @FunctionalInterface + interface Function3 { + + R apply(T1 v1, T2 v2, T3 v3) throws IOException; + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 881d0b7a8aa30..99790947e1adb 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -19,205 +19,154 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import com.google.common.collect.Iterables; -import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; -import com.uber.hoodie.common.model.HoodieCleaningPolicy; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieDataFile; -import com.uber.hoodie.common.model.HoodieFileGroup; import com.uber.hoodie.common.model.HoodieKey; -import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; -import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; -import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.config.HoodieCompactionConfig; -import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; -import java.io.File; import java.io.FileInputStream; -import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.TreeSet; import java.util.UUID; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.scheduler.SparkListener; -import org.apache.spark.scheduler.SparkListenerTaskEnd; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.util.AccumulatorV2; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import scala.Option; -import scala.collection.Iterator; - -public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { - - private transient JavaSparkContext jsc = null; - private transient SQLContext sqlContext; - private transient FileSystem fs; - private String basePath = null; - private transient HoodieTestDataGenerator dataGen = null; - private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"}; - - @Before - public void init() throws IOException { - // Initialize a local spark env - jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient")); - - //SQLContext stuff - sqlContext = new SQLContext(jsc); - - // Create a temp folder as the base path - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - basePath = folder.getRoot().getAbsolutePath(); - fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration()); - HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); - dataGen = new HoodieTestDataGenerator(); - } - - private HoodieWriteConfig getConfig() { - return getConfigBuilder().build(); - } +@SuppressWarnings("unchecked") +public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { - private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) - .forTable("test-trip-table") - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + /** + * Test Auto Commit behavior for HoodieWriteClient insert API + */ + @Test + public void testAutoCommitOnInsert() throws Exception { + testAutoCommit(HoodieWriteClient::insert, false); } - private void assertNoWriteErrors(List statuses) { - // Verify there are no errors - for (WriteStatus status : statuses) { - assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors()); - } + /** + * Test Auto Commit behavior for HoodieWriteClient insertPrepped API + */ + @Test + public void testAutoCommitOnInsertPrepped() throws Exception { + testAutoCommit(HoodieWriteClient::insertPreppedRecords, true); } - private void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException { - for (String partitionPath : partitionPaths) { - assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath))); - HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath)); - pmeta.readFromFS(); - assertEquals(3, pmeta.getPartitionDepth()); - } + /** + * Test Auto Commit behavior for HoodieWriteClient upsert API + */ + @Test + public void testAutoCommitOnUpsert() throws Exception { + testAutoCommit(HoodieWriteClient::upsert, false); } - private void checkTaggedRecords(List taggedRecords, String commitTime) { - for (HoodieRecord rec : taggedRecords) { - assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown()); - assertEquals("All records should have commit time " + commitTime + ", since updates were made", - rec.getCurrentLocation().getCommitTime(), commitTime); - } + /** + * Test Auto Commit behavior for HoodieWriteClient upsert Prepped API + */ + @Test + public void testAutoCommitOnUpsertPrepped() throws Exception { + testAutoCommit(HoodieWriteClient::upsertPreppedRecords, true); } - + /** + * Test Auto Commit behavior for HoodieWriteClient bulk-insert API + */ @Test - public void testFilterExist() throws Exception { - HoodieWriteConfig config = getConfig(); - HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); - String newCommitTime = writeClient.startCommit(); - List records = dataGen.generateInserts(newCommitTime, 100); - JavaRDD recordsRDD = jsc.parallelize(records, 1); - - HoodieReadClient readClient = new HoodieReadClient(jsc, config.getBasePath()); - JavaRDD filteredRDD = readClient.filterExists(recordsRDD); - - // Should not find any files - assertTrue(filteredRDD.collect().size() == 100); - - JavaRDD smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1); - // We create three parquet file, each having one record. (two different partitions) - List statuses = writeClient.bulkInsert(smallRecordsRDD, newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - readClient = new HoodieReadClient(jsc, config.getBasePath()); - filteredRDD = readClient.filterExists(recordsRDD); - List result = filteredRDD.collect(); - // Check results - assertTrue(result.size() == 25); + public void testAutoCommitOnBulkInsert() throws Exception { + testAutoCommit(HoodieWriteClient::bulkInsert, false); } + /** + * Test Auto Commit behavior for HoodieWriteClient bulk-insert prepped API + */ @Test - public void testAutoCommit() throws Exception { + public void testAutoCommitOnBulkInsertPrepped() throws Exception { + testAutoCommit((writeClient, recordRDD, commitTime) + -> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()), true); + } + + /** + * Test auto-commit by applying write function + * + * @param writeFn One of HoodieWriteClient Write API + * @throws Exception in case of failure + */ + private void testAutoCommit( + Function3, HoodieWriteClient, JavaRDD, String> writeFn, + boolean isPrepped) throws Exception { // Set autoCommit false HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + String prevCommitTime = "000"; String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - - List records = dataGen.generateInserts(newCommitTime, 200); - JavaRDD writeRecords = jsc.parallelize(records, 1); - - JavaRDD result = client.bulkInsertPreppedRecords(writeRecords, newCommitTime, Option.empty()); + int numRecords = 200; + JavaRDD result = + insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords); assertFalse("If Autocommit is false, then commit should not be made automatically", HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); assertTrue("Commit should succeed", client.commit(newCommitTime, result)); assertTrue("After explicit commit, commit file should be created", HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); + } - newCommitTime = "002"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, 100); - JavaRDD updateRecords = jsc.parallelize(records, 1); - result = client.upsert(updateRecords, newCommitTime); - assertFalse("If Autocommit is false, then commit should not be made automatically", - HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); - assertTrue("Commit should succeed", client.commit(newCommitTime, result)); - assertTrue("After explicit commit, commit file should be created", - HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); + /** + * Test De-duplication behavior for HoodieWriteClient insert API + */ + @Test + public void testDeduplicationOnInsert() throws Exception { + testDeduplication(HoodieWriteClient::insert); } + /** + * Test De-duplication behavior for HoodieWriteClient bulk-insert API + */ @Test - public void testUpserts() throws Exception { - testUpsertsInternal(getConfig()); + public void testDeduplicationOnBulkInsert() throws Exception { + testDeduplication(HoodieWriteClient::bulkInsert); } + /** + * Test De-duplication behavior for HoodieWriteClient upsert API + */ @Test - public void testDeduplication() throws Exception { + public void testDeduplicationOnUpsert() throws Exception { + testDeduplication(HoodieWriteClient::upsert); + } + + /** + * Test Deduplication Logic for write function + * + * @param writeFn One of HoddieWriteClient non-prepped write APIs + * @throws Exception in case of failure + */ + private void testDeduplication( + Function3, HoodieWriteClient, JavaRDD, String> writeFn) throws Exception { String newCommitTime = "001"; String recordKey = UUID.randomUUID().toString(); @@ -229,721 +178,183 @@ public void testDeduplication() throws Exception { HoodieRecord recordTwo = new HoodieRecord(keyTwo, HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime)); - JavaRDD records = jsc.parallelize(Arrays.asList(recordOne, recordTwo), 1); + // Same key and partition as keyTwo + HoodieRecord recordThree = new HoodieRecord(keyTwo, + HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime)); + + JavaRDD records = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); // dedup should be done based on recordKey only HoodieWriteClient clientWithDummyGlobalIndex = getWriteClientWithDummyIndex(true); - assertEquals(1, clientWithDummyGlobalIndex.deduplicateRecords(records, 1).collect().size()); + List dedupedRecs = clientWithDummyGlobalIndex.deduplicateRecords(records, 1).collect(); + assertEquals(1, dedupedRecs.size()); + assertNodupesWithinPartition(dedupedRecs); // dedup should be done based on both recordKey and partitionPath HoodieWriteClient clientWithDummyNonGlobalIndex = getWriteClientWithDummyIndex(false); - assertEquals(2, clientWithDummyNonGlobalIndex.deduplicateRecords(records, 1).collect().size()); - } - + dedupedRecs = + clientWithDummyNonGlobalIndex.deduplicateRecords(records, 1).collect(); + assertEquals(2, dedupedRecs.size()); + assertNodupesWithinPartition(dedupedRecs); + + // Perform write-action and check + HoodieWriteClient client = new HoodieWriteClient(jsc, + getConfigBuilder().combineInput(true, true).build()); + client.startCommitWithTime(newCommitTime); + List statuses = writeFn.apply(client, records, newCommitTime).collect(); + assertNoWriteErrors(statuses); + assertEquals(2, statuses.size()); + assertNodupesWithinPartition( + statuses.stream().map(WriteStatus::getWrittenRecords) + .flatMap(Collection::stream).collect(Collectors.toList())); + } + + /** + * Build a test Hoodie WriteClient with dummy index to configure isGlobal flag + * + * @param isGlobal Flag to control HoodieIndex.isGlobal + * @return Hoodie Write Client + * @throws Exception in case of error + */ private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception { HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(isGlobal); return new HoodieWriteClient(jsc, getConfigBuilder().build(), false, index); } - private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig) throws Exception { - HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig); - - /** - * Write 1 (only inserts) - */ - String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - - List records = dataGen.generateInserts(newCommitTime, 200); - JavaRDD writeRecords = jsc.parallelize(records, 1); - - List statuses = client.upsertPreppedRecords(writeRecords, newCommitTime).collect(); - assertNoWriteErrors(statuses); - - // check the partition metadata is written out - assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); - - // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - - assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); - assertEquals("Must contain 200 records", records.size(), - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); - // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); - List taggedRecords = readClient.tagLocation(jsc.parallelize(records, 1)).collect(); - checkTaggedRecords(taggedRecords, "001"); - - /** - * Write 2 (updates) - */ - newCommitTime = "004"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, 100); - LinkedHashMap recordsMap = new LinkedHashMap<>(); - for (HoodieRecord rec : records) { - if (!recordsMap.containsKey(rec.getKey())) { - recordsMap.put(rec.getKey(), rec); - } - } - List dedupedRecords = new ArrayList<>(recordsMap.values()); - - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - // verify there are now 2 commits - timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); - - // Index should be able to locate all updates in correct locations. - readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); - taggedRecords = readClient.tagLocation(jsc.parallelize(dedupedRecords, 1)).collect(); - checkTaggedRecords(taggedRecords, "004"); - - // Check the entire dataset has 100 records still - String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; - for (int i = 0; i < fullPartitionPaths.length; i++) { - fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); - } - assertEquals("Must contain 200 records", 200, - HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); - - // Check that the incremental consumption from time 000 - assertEquals("Incremental consumption from time 002, should give all records in commit 004", - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "002").count()); - assertEquals("Incremental consumption from time 001, should give all records in commit 004", - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); + /** + * Test Upsert API + */ + @Test + public void testUpserts() throws Exception { + testUpsertsInternal(getConfig(), + HoodieWriteClient::upsert, false); } + /** + * Test Upsert API using temporary folders. + */ @Test public void testUpsertsWithFinalizeWrite() throws Exception { - HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withUseTempFolderCopyOnWriteForCreate(true) - .withUseTempFolderCopyOnWriteForMerge(true).build(); - testUpsertsInternal(hoodieWriteConfig); + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder() + .withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(true) + .build(); + testUpsertsInternal(hoodieWriteConfig, + HoodieWriteClient::upsert, false); } + /** + * Test UpsertPrepped API + */ @Test - public void testDeletes() throws Exception { - - HoodieWriteConfig cfg = getConfig(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - - /** - * Write 1 (inserts and deletes) - * Write actual 200 insert records and ignore 100 delete records - */ - String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - - List fewRecordsForInsert = dataGen.generateInserts(newCommitTime, 200); - List fewRecordsForDelete = dataGen.generateDeletes(newCommitTime, 100); - - List records = new ArrayList(fewRecordsForInsert); - records.addAll(fewRecordsForDelete); - - JavaRDD writeRecords = jsc.parallelize(records, 1); - - List statuses = client.upsert(writeRecords, newCommitTime).collect(); - assertNoWriteErrors(statuses); - - // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); - assertEquals("Must contain 200 records", fewRecordsForInsert.size(), - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); - // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - - List taggedRecords = index.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); - checkTaggedRecords(taggedRecords, "001"); - - /** - * Write 2 (deletes+writes) - */ - newCommitTime = "004"; - client.startCommitWithTime(newCommitTime); - - fewRecordsForDelete = records.subList(0, 50); - List fewRecordsForUpdate = records.subList(50, 100); - records = dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete); - - records.addAll(fewRecordsForUpdate); - - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - // verify there are now 2 commits - timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); - - // Check the entire dataset has 150 records(200-50) still - String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; - for (int i = 0; i < fullPartitionPaths.length; i++) { - fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); - } - assertEquals("Must contain 150 records", 150, - HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); - - // Check that the incremental consumption from time 000 - assertEquals("Incremental consumption from latest commit, should give 50 updated records", 50, - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); - assertEquals("Incremental consumption from time 001, should give 50 updated records", 50, - HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); - assertEquals("Incremental consumption from time 000, should give 150", 150, - HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count()); + public void testUpsertsPrepped() throws Exception { + testUpsertsInternal(getConfig(), + HoodieWriteClient::upsertPreppedRecords, true); } - + /** + * Test UpsertPrepped API using temporary folders. + */ @Test - public void testCreateSavepoint() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) - .build()).build(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + public void testUpsertsPreppedWithFinalizeWrite() throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder() + .withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(true) + .build(); + testUpsertsInternal(hoodieWriteConfig, + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Test one of HoodieWriteClient upsert(Prepped) APIs + * + * @param hoodieWriteConfig Write Config + * @param writeFn One of Hoodie Write Function API + * @throws Exception in case of error + */ + private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig, + Function3, HoodieWriteClient, JavaRDD, String> writeFn, + boolean isPrepped) throws Exception { + HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig); - /** - * Write 1 (only inserts) - */ + //Write 1 (only inserts) String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - - List records = dataGen.generateInserts(newCommitTime, 200); - List statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - assertNoWriteErrors(statuses); + String initCommitTime = "000"; + int numRecords = 200; + insertFirstBatch(hoodieWriteConfig, + client, newCommitTime, initCommitTime, numRecords, HoodieWriteClient::insert, isPrepped, true, numRecords); - /** - * Write 2 (updates) - */ - newCommitTime = "002"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - client.savepoint("hoodie-unit-test", "test"); - try { - client.rollback(newCommitTime); - fail("Rollback of a savepoint was allowed " + newCommitTime); - } catch (HoodieRollbackException e) { - // this is good - } - - /** - * Write 3 (updates) - */ - newCommitTime = "003"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - /** - * Write 4 (updates) - */ + // Write 2 (updates) + String prevCommitTime = newCommitTime; newCommitTime = "004"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), - getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); - List dataFiles = partitionPaths.stream().flatMap(s -> { - return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); - }).collect(Collectors.toList()); - - assertEquals("The data files for commit 002 should not be cleaned", 3, dataFiles.size()); - - // Delete savepoint - assertFalse(table.getCompletedSavepointTimeline().empty()); - client.deleteSavepoint(table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp()); - // rollback and reupsert 004 - client.rollback(newCommitTime); - - client.startCommitWithTime(newCommitTime); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); - final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); - dataFiles = partitionPaths.stream().flatMap(s -> { - return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); - }).collect(Collectors.toList()); - - assertEquals("The data files for commit 002 should be cleaned now", 0, dataFiles.size()); + numRecords = 100; + String commitTimeBetweenPrevAndNew = "002"; + updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, + Optional.of(Arrays.asList(commitTimeBetweenPrevAndNew)), + initCommitTime, numRecords, writeFn, isPrepped, true, numRecords, 200, 2); } - + /** + * Tesst deletion of records + */ @Test - public void testRollbackToSavepoint() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) - .build()).build(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + public void testDeletes() throws Exception { + HoodieWriteClient client = new HoodieWriteClient(jsc, getConfig()); /** - * Write 1 (only inserts) + * Write 1 (inserts and deletes) + * Write actual 200 insert records and ignore 100 delete records */ + String initCommitTime = "000"; String newCommitTime = "001"; - client.startCommitWithTime(newCommitTime); - List records = dataGen.generateInserts(newCommitTime, 200); - JavaRDD writeRecords = jsc.parallelize(records, 1); - - List statuses = client.upsert(writeRecords, newCommitTime).collect(); - assertNoWriteErrors(statuses); - - /** - * Write 2 (updates) - */ - newCommitTime = "002"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - client.savepoint("hoodie-unit-test", "test"); - - /** - * Write 3 (updates) - */ - newCommitTime = "003"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), - getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); - - List dataFiles = partitionPaths.stream().flatMap(s -> { - return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003")); - }).collect(Collectors.toList()); - assertEquals("The data files for commit 003 should be present", 3, dataFiles.size()); + final List recordsInFirstBatch = new ArrayList<>(); + Function2, String, Integer> recordGenFunction = + (String commitTime, Integer numRecordsInThisCommit) -> { + List fewRecordsForInsert = dataGen.generateInserts(commitTime, 200); + List fewRecordsForDelete = dataGen.generateDeletes(commitTime, 100); + + recordsInFirstBatch.addAll(fewRecordsForInsert); + recordsInFirstBatch.addAll(fewRecordsForDelete); + return recordsInFirstBatch; + }; + writeBatch(client, newCommitTime, initCommitTime, Optional.empty(), initCommitTime, + //unused as genFn uses hard-coded number of inserts/updates/deletes + -1, + recordGenFunction, HoodieWriteClient::upsert, true, + 200, 200, 1); /** - * Write 4 (updates) + * Write 2 (deletes+writes) */ + String prevCommitTime = newCommitTime; newCommitTime = "004"; - client.startCommitWithTime(newCommitTime); - - records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); - final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); - - dataFiles = partitionPaths.stream().flatMap(s -> { - return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004")); - }).collect(Collectors.toList()); - assertEquals("The data files for commit 004 should be present", 3, dataFiles.size()); - - // rolling back to a non existent savepoint must not succeed - try { - client.rollbackToSavepoint("001"); - fail("Rolling back to non-existent savepoint should not be allowed"); - } catch (HoodieRollbackException e) { - // this is good - } - - // rollback to savepoint 002 - HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get(); - client.rollbackToSavepoint(savepoint.getTimestamp()); - - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); - final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); - dataFiles = partitionPaths.stream().flatMap(s -> { - return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); - }).collect(Collectors.toList()); - assertEquals("The data files for commit 002 be available", 3, dataFiles.size()); - - dataFiles = partitionPaths.stream().flatMap(s -> { - return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003")); - }).collect(Collectors.toList()); - assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size()); - - dataFiles = partitionPaths.stream().flatMap(s -> { - return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004")); - }).collect(Collectors.toList()); - assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size()); - } - - - @Test - public void testInsertAndCleanByVersions() throws Exception { - int maxVersions = 2; // keep upto 2 versions for each file - HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .retainFileVersions(maxVersions).build()).build(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - - /** - * do a big insert - * (this is basically same as insert part of upsert, just adding it here so we can - * catch breakages in insert(), if the implementation diverges.) - */ - String newCommitTime = client.startCommit(); - List records = dataGen.generateInserts(newCommitTime, 500); - JavaRDD writeRecords = jsc.parallelize(records, 5); - - List statuses = client.insert(writeRecords, newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); - assertFalse(table.getCompletedCleanTimeline().empty()); - assertEquals("The clean instant should be the same as the commit instant", commitTime, - table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); - - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); - checkTaggedRecords(taggedRecords, newCommitTime); - - // Keep doing some writes and clean inline. Make sure we have expected number of files - // remaining. - for (int writeCnt = 2; writeCnt < 10; writeCnt++) { - - Thread.sleep(1100); // make sure commits are unique - newCommitTime = client.startCommit(); - records = dataGen.generateUpdates(newCommitTime, 100); - - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metadata, getConfig()); - timeline = table.getCommitsTimeline(); - - TableFileSystemView fsView = table.getFileSystemView(); - // Need to ensure the following - for (String partitionPath : dataGen.getPartitionPaths()) { - // compute all the versions of all files, from time 0 - HashMap> fileIdToVersions = new HashMap<>(); - for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(entry).get()); - - for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) { - if (!fileIdToVersions.containsKey(wstat.getFileId())) { - fileIdToVersions.put(wstat.getFileId(), new TreeSet<>()); - } - fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName())); - } - } - - List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); - - for (HoodieFileGroup fileGroup : fileGroups) { - // No file has no more than max versions - String fileId = fileGroup.getId(); - List dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList()); - - assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions", - dataFiles.size() <= maxVersions); - - // Each file, has the latest N versions (i.e cleaning gets rid of older versions) - List commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId)); - for (int i = 0; i < dataFiles.size(); i++) { - assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions, - Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i)); - } - } - } - } - } - - @Test - public void testInsertAndCleanByCommits() throws Exception { - int maxCommits = 3; // keep upto 3 commits from the past - HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .retainCommits(maxCommits).build()).build(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); - - /** - * do a big insert - * (this is basically same as insert part of upsert, just adding it here so we can - * catch breakages in insert(), if the implementation diverges.) - */ - String newCommitTime = client.startCommit(); - List records = dataGen.generateInserts(newCommitTime, 500); - JavaRDD writeRecords = jsc.parallelize(records, 5); - - List statuses = client.insertPreppedRecords(writeRecords, newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); - - assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); - assertFalse(table.getCompletedCleanTimeline().empty()); - assertEquals("The clean instant should be the same as the commit instant", commitTime, - table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); - - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); - checkTaggedRecords(taggedRecords, newCommitTime); - - // Keep doing some writes and clean inline. Make sure we have expected number of files - // remaining. - for (int writeCnt = 2; writeCnt < 10; writeCnt++) { - Thread.sleep(1100); // make sure commits are unique - newCommitTime = client.startCommit(); - records = dataGen.generateUpdates(newCommitTime, 100); - - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); - // Verify there are no errors - assertNoWriteErrors(statuses); - - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); - HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); - Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); - Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); - if (earliestRetainedCommit.isPresent()) { - acceptableCommits.removeAll( - activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()).getInstants() - .collect(Collectors.toSet())); - acceptableCommits.add(earliestRetainedCommit.get()); - } - - TableFileSystemView fsView = table1.getFileSystemView(); - // Need to ensure the following - for (String partitionPath : dataGen.getPartitionPaths()) { - List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); - for (HoodieFileGroup fileGroup : fileGroups) { - Set commitTimes = new HashSet<>(); - fileGroup.getAllDataFiles().forEach(value -> { - System.out.println("Data File - " + value); - commitTimes.add(value.getCommitTime()); - }); - assertEquals("Only contain acceptable versions of file should be present", - acceptableCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), commitTimes); - } - } - } - } - - @Test - public void testRollbackCommit() throws Exception { - // Let's create some commit files and parquet files - String commitTime1 = "20160501010101"; - String commitTime2 = "20160502020601"; - String commitTime3 = "20160506030611"; - new File(basePath + "/.hoodie").mkdirs(); - HoodieTestDataGenerator - .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); - - // Only first two have commit files - HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2); - // Third one has a .inflight intermediate commit file - HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3); - - // Make commit1 - String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11"); - String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12"); - String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13"); - - // Make commit2 - String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21"); - String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22"); - String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23"); - - // Make commit3 - String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31"); - String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32"); - String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); - - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - - HoodieWriteClient client = new HoodieWriteClient(jsc, config, false); - - // Rollback commit 1 (this should fail, since commit2 is still around) - try { - client.rollback(commitTime1); - assertTrue("Should have thrown an exception ", false); - } catch (HoodieRollbackException hrbe) { - // should get here - } - - // Rollback commit3 - client.rollback(commitTime3); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - - // simulate partial failure, where .inflight was not deleted, but data files were. - HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3); - client.rollback(commitTime3); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - - // Rollback commit2 - client.rollback(commitTime2); - assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - - // simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a - // .inflight commit and a bunch of data files around. - HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2); - file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21"); - file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22"); - file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23"); - - client.rollback(commitTime2); - assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - - // Let's rollback commit1, Check results - client.rollback(commitTime1); - assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); - } - - @Test - public void testAutoRollbackCommit() throws Exception { - // Let's create some commit files and parquet files - String commitTime1 = "20160501010101"; - String commitTime2 = "20160502020601"; - String commitTime3 = "20160506030611"; - new File(basePath + "/.hoodie").mkdirs(); - HoodieTestDataGenerator - .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); - - // One good commit - HoodieTestUtils.createCommitFiles(basePath, commitTime1); - // Two inflight commits - HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2, commitTime3); - - // Make commit1 - String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11"); - String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12"); - String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13"); - - // Make commit2 - String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21"); - String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22"); - String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23"); - - // Make commit3 - String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31"); - String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32"); - String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); - - // Turn auto rollback off - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - - new HoodieWriteClient(jsc, config, false); - - // Check results, nothing changed - assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); - assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); - - // Turn auto rollback on - new HoodieWriteClient(jsc, config, true); - assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) - || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) - && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); - } - - - private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { - HoodieWriteConfig.Builder builder = getConfigBuilder(); - return builder.withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15) - .insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records - .withStorageConfig( - HoodieStorageConfig.newBuilder().limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20).build()) - .build(); - } - - + final List recordsInSecondBatch = new ArrayList<>(); + + recordGenFunction = + (String commitTime, Integer numRecordsInThisCommit) -> { + List fewRecordsForDelete = recordsInFirstBatch.subList(0, 50); + List fewRecordsForUpdate = recordsInFirstBatch.subList(50, 100); + recordsInSecondBatch.addAll(dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete)); + recordsInSecondBatch.addAll(fewRecordsForUpdate); + return recordsInSecondBatch; + }; + writeBatch(client, newCommitTime, prevCommitTime, Optional.empty(), initCommitTime, + 100, recordGenFunction, HoodieWriteClient::upsert, true, + 50, 150, 2); + } + + /** + * Test scenario of new file-group getting added during upsert() + */ @Test public void testSmallInsertHandlingForUpserts() throws Exception { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max - dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); + dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath}); HoodieWriteClient client = new HoodieWriteClient(jsc, config); @@ -1009,6 +420,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { List files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) .collect(Collectors.toList()); int numTotalInsertsInCommit3 = 0; + int numTotalUpdatesInCommit3 = 0; for (HoodieDataFile file : files) { if (file.getFileName().contains(file1)) { assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime()); @@ -1018,8 +430,8 @@ public void testSmallInsertHandlingForUpserts() throws Exception { String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); if (recordCommitTime.equals(commitTime3)) { if (keys2.contains(recordKey)) { - assertEquals("only expect commit3", commitTime3, recordCommitTime); keys2.remove(recordKey); + numTotalUpdatesInCommit3++; } else { numTotalInsertsInCommit3++; } @@ -1038,9 +450,13 @@ public void testSmallInsertHandlingForUpserts() throws Exception { numTotalInsertsInCommit3 += records.size(); } } + assertEquals("Total updates in commit3 must add up", inserts2.size(), numTotalUpdatesInCommit3); assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3); } + /** + * Test scenario of new file-group getting added during insert() + */ @Test public void testSmallInsertHandlingForInserts() throws Exception { @@ -1048,7 +464,7 @@ public void testSmallInsertHandlingForInserts() throws Exception { final int insertSplitLimit = 100; // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max - dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); + dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath}); HoodieWriteClient client = new HoodieWriteClient(jsc, config); // Inserts => will write file1 @@ -1060,7 +476,7 @@ public void testSmallInsertHandlingForInserts() throws Exception { List statuses = client.insert(insertRecordsRDD1, commitTime1).collect(); assertNoWriteErrors(statuses); - assertPartitionMetadata(new String[] {testPartitionPath}, fs); + assertPartitionMetadata(new String[]{testPartitionPath}, fs); assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); @@ -1119,307 +535,10 @@ public void testSmallInsertHandlingForInserts() throws Exception { inserts1.size() + inserts2.size() + insert3.size()); } + /** + * Test to ensure commit metadata points to valid files + */ @Test - public void testKeepLatestFileVersions() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( - HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) - .build(); - - // make 1 commit, with 1 file per partition - HoodieTestUtils.createCommitFiles(basePath, "000"); - - String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); - String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - - List hoodieCleanStatsOne = table.clean(jsc); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsOne, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsOne, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0)); - - // make next commit, with 1 insert & 1 update per partition - HoodieTestUtils.createCommitFiles(basePath, "001"); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config); - - String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert - String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update - HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update - - List hoodieCleanStatsTwo = table.clean(jsc); - assertEquals("Must clean 1 file", 1, - getCleanStat(hoodieCleanStatsTwo, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals("Must clean 1 file", 1, - getCleanStat(hoodieCleanStatsTwo, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "001", file2P1C1)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0)); - - // make next commit, with 2 updates to existing files, and 1 insert - HoodieTestUtils.createCommitFiles(basePath, "002"); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update - String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002"); - - List hoodieCleanStatsThree = table.clean(jsc); - assertEquals("Must clean two files", 2, - getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2)); - - // No cleaning on partially written file, with no commit. - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file3P0C2); // update - List hoodieCleanStatsFour = table.clean(jsc); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2)); - } - - @Test - public void testKeepLatestFileVersionsMOR() throws IOException { - - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( - HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) - .build(); - - HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, - HoodieTableType.MERGE_ON_READ); - - // Make 3 files, one base file and 2 log files associated with base file - String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); - String file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "000", file1P0, Optional.empty()); - String file2P0L1 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "000", file1P0, Optional.of(2)); - // make 1 compaction commit - HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000"); - - // Make 4 files, one base file and 3 log files associated with base file - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0); - file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); - file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); - file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); - // make 1 compaction commit - HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); - - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); - List hoodieCleanStats = table.clean(jsc); - assertEquals("Must clean three files, one parquet and 2 log files", 3, - getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0)); - assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.empty())); - assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.of(2))); - } - - @Test - public void testKeepLatestCommits() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( - HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); - - // make 1 commit, with 1 file per partition - HoodieTestUtils.createCommitFiles(basePath, "000"); - - String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); - String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); - - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - - List hoodieCleanStatsOne = table.clean(jsc); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsOne, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsOne, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0)); - - // make next commit, with 1 insert & 1 update per partition - HoodieTestUtils.createCommitFiles(basePath, "001"); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert - String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update - HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update - - List hoodieCleanStatsTwo = table.clean(jsc); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsTwo, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsTwo, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "001", file2P1C1)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0)); - - // make next commit, with 2 updates to existing files, and 1 insert - HoodieTestUtils.createCommitFiles(basePath, "002"); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update - String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002"); - - List hoodieCleanStatsThree = table.clean(jsc); - assertEquals("Must not clean any file. We have to keep 1 version before the latest commit time to keep", 0, - getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size()); - - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); - - // make next commit, with 2 updates to existing files, and 1 insert - HoodieTestUtils.createCommitFiles(basePath, "003"); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update - String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003"); - - List hoodieCleanStatsFour = table.clean(jsc); - assertEquals("Must not clean one old file", 1, - getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size()); - - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file2P0C1)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "003", file4P0C3)); - - // No cleaning on partially written file, with no commit. - HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "004", file3P0C2); // update - List hoodieCleanStatsFive = table.clean(jsc); - assertEquals("Must not clean any files", 0, - getCleanStat(hoodieCleanStatsFive, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1)); - } - - @Test - public void testCleaningWithZeroPartitonPaths() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( - HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); - - // Make a commit, although there are no partitionPaths. - // Example use-case of this is when a client wants to create a table - // with just some commit metadata, but no data/partitionPaths. - HoodieTestUtils.createCommitFiles(basePath, "000"); - - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - - List hoodieCleanStatsOne = table.clean(jsc); - assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); - } - - @Test - public void testCleaningSkewedPartitons() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( - HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); - Map stageOneShuffleReadTaskRecordsCountMap = new HashMap<>(); - - // Since clean involves repartition in order to uniformly distribute data, - // we can inspect the number of records read by various tasks in stage 1. - // There should not be skew in the number of records read in the task. - - // SparkListener below listens to the stage end events and captures number of - // records read by various tasks in stage-1. - jsc.sc().addSparkListener(new SparkListener() { - - @Override - public void onTaskEnd(SparkListenerTaskEnd taskEnd) { - - Iterator> iterator = taskEnd.taskMetrics().accumulators().iterator(); - while (iterator.hasNext()) { - AccumulatorV2 accumulator = iterator.next(); - if (taskEnd.stageId() == 1 && accumulator.isRegistered() && accumulator.name().isDefined() - && accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) { - stageOneShuffleReadTaskRecordsCountMap.put(taskEnd.taskInfo().taskId(), (Long) accumulator.value()); - } - } - } - }); - - // make 1 commit, with 100 files in one partition and 10 in other two - HoodieTestUtils.createCommitFiles(basePath, "000"); - List filesP0C0 = createFilesInPartition(partitionPaths[0], "000", 100); - List filesP1C0 = createFilesInPartition(partitionPaths[1], "000", 10); - List filesP2C0 = createFilesInPartition(partitionPaths[2], "000", 10); - - HoodieTestUtils.createCommitFiles(basePath, "001"); - updateAllFilesInPartition(filesP0C0, partitionPaths[0], "001"); - updateAllFilesInPartition(filesP1C0, partitionPaths[1], "001"); - updateAllFilesInPartition(filesP2C0, partitionPaths[2], "001"); - - HoodieTestUtils.createCommitFiles(basePath, "002"); - updateAllFilesInPartition(filesP0C0, partitionPaths[0], "002"); - updateAllFilesInPartition(filesP1C0, partitionPaths[1], "002"); - updateAllFilesInPartition(filesP2C0, partitionPaths[2], "002"); - - HoodieTestUtils.createCommitFiles(basePath, "003"); - updateAllFilesInPartition(filesP0C0, partitionPaths[0], "003"); - updateAllFilesInPartition(filesP1C0, partitionPaths[1], "003"); - updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003"); - - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - List hoodieCleanStats = table.clean(jsc); - - assertEquals(100, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[2]).getSuccessDeleteFiles().size()); - - // 3 tasks are expected since the number of partitions is 3 - assertEquals(3, stageOneShuffleReadTaskRecordsCountMap.keySet().size()); - // Sum of all records processed = total number of files to clean - assertEquals(120, - stageOneShuffleReadTaskRecordsCountMap.values().stream().reduce((a, b) -> a + b).get().intValue()); - assertTrue("The skew in handling files to clean is not removed. " - + "Each task should handle more records than the partitionPath with least files " - + "and less records than the partitionPath with most files.", - stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3); - } - - @Test - public void testCleanTemporaryDataFiles() throws IOException { - HoodieTestUtils.createCommitFiles(basePath, "000"); - List tempFiles = createTempFiles("000", 10); - assertEquals("Some temp files are created.", 10, tempFiles.size()); - assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); - - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withUseTempFolderCopyOnWriteForCreate(false) - .withUseTempFolderCopyOnWriteForMerge(false).build(); - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - table.rollback(jsc, Collections.emptyList()); - assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); - - config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) - .withUseTempFolderCopyOnWriteForMerge(false).build(); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); - table.rollback(jsc, Collections.emptyList()); - assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); - } - public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); @@ -1462,45 +581,16 @@ public void testCommitWritesRelativePaths() throws Exception { } } - private HoodieCleanStat getCleanStat(List hoodieCleanStatsTwo, String partitionPath) { - return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().get(); - } - - private void updateAllFilesInPartition(List files, String partitionPath, String commitTime) - throws IOException { - for (String fileId : files) { - HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId); - } - } - - private List createFilesInPartition(String partitionPath, String commitTime, int numFiles) - throws IOException { - List files = new ArrayList<>(); - for (int i = 0; i < numFiles; i++) { - files.add(HoodieTestUtils.createNewDataFile(basePath, partitionPath, commitTime)); - } - return files; - } - - private List createTempFiles(String commitTime, int numFiles) throws IOException { - List files = new ArrayList<>(); - for (int i = 0; i < numFiles; i++) { - files.add(HoodieTestUtils.createNewDataFile(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, commitTime)); - } - return files; - } - - private int getTotalTempFiles() throws IOException { - return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length; - } - - @After - public void clean() { - if (basePath != null) { - new File(basePath).delete(); - } - if (jsc != null) { - jsc.stop(); - } + /** + * Build Hoodie Write Config for small data file sizes + */ + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { + HoodieWriteConfig.Builder builder = getConfigBuilder(); + return builder.withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15) + .insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records + .withStorageConfig( + HoodieStorageConfig.newBuilder().limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20).build()) + .build(); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java new file mode 100644 index 0000000000000..45cc4d381edc3 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java @@ -0,0 +1,195 @@ +/* + * 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; + +import static org.junit.Assert.assertTrue; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.config.HoodieWriteConfig; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Test; +import scala.Option; + +@SuppressWarnings("unchecked") +/** + * Test-cases for covering HoodieReadClient APIs + */ +public class TestHoodieReadClient extends TestHoodieClientBase { + + /** + * Test ReadFilter API after writing new records using HoodieWriteClient.insert + */ + @Test + public void testReadFilterExistAfterInsert() throws Exception { + testReadFilterExist(getConfig(), HoodieWriteClient::insert); + } + + /** + * Test ReadFilter API after writing new records using HoodieWriteClient.insertPrepped + */ + @Test + public void testReadFilterExistAfterInsertPrepped() throws Exception { + testReadFilterExist(getConfig(), HoodieWriteClient::insertPreppedRecords); + } + + /** + * Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsert + */ + @Test + public void testReadFilterExistAfterBulkInsert() throws Exception { + testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert); + } + + /** + * Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsertPrepped + */ + @Test + public void testReadFilterExistAfterBulkInsertPrepped() throws Exception { + testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), + (writeClient, recordRDD, commitTime) -> { + return writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()); + }); + } + + /** + * Helper to write new records using one of HoodieWriteClient's write API and use ReadClient to test filterExists() + * API works correctly + * + * @param config Hoodie Write Config + * @param writeFn Write Function for writing records + * @throws Exception in case of error + */ + private void testReadFilterExist(HoodieWriteConfig config, + Function3, HoodieWriteClient, JavaRDD, String> writeFn) throws Exception { + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + String newCommitTime = writeClient.startCommit(); + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + + HoodieReadClient readClient = new HoodieReadClient(jsc, config.getBasePath()); + JavaRDD filteredRDD = readClient.filterExists(recordsRDD); + + // Should not find any files + assertTrue(filteredRDD.collect().size() == 100); + + JavaRDD smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1); + // We create three parquet file, each having one record. (3 different partitions) + List statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + readClient = new HoodieReadClient(jsc, config.getBasePath()); + filteredRDD = readClient.filterExists(recordsRDD); + List result = filteredRDD.collect(); + // Check results + assertTrue(result.size() == 25); + } + + /** + * Test tagLocation API after insert() + */ + @Test + public void testTagLocationAfterInsert() throws Exception { + testTagLocation(getConfig(), HoodieWriteClient::insert, + HoodieWriteClient::upsert, false); + } + + /** + * Test tagLocation API after insertPrepped() + */ + @Test + public void testTagLocationAfterInsertPrepped() throws Exception { + testTagLocation(getConfig(), HoodieWriteClient::insertPreppedRecords, + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Test tagLocation API after bulk-insert() + */ + @Test + public void testTagLocationAfterBulkInsert() throws Exception { + testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert, + HoodieWriteClient::upsert, false); + } + + /** + * Test tagLocation API after bulkInsertPrepped() + */ + @Test + public void testTagLocationAfterBulkInsertPrepped() throws Exception { + testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), + (writeClient, recordRDD, commitTime) + -> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()), + HoodieWriteClient::upsertPreppedRecords, true); + } + + /** + * Helper method to test tagLocation after using different HoodieWriteClient write APIS + * + * @param hoodieWriteConfig Write Config + * @param insertFn Hoodie Write Client first Insert API + * @param updateFn Hoodie Write Client upsert API + * @param isPrepped isPrepped flag. + * @throws Exception in case of error + */ + private void testTagLocation( + HoodieWriteConfig hoodieWriteConfig, + Function3, HoodieWriteClient, JavaRDD, String> insertFn, + Function3, HoodieWriteClient, JavaRDD, String> updateFn, + boolean isPrepped) + throws Exception { + HoodieWriteClient client = new HoodieWriteClient(jsc, hoodieWriteConfig); + //Write 1 (only inserts) + String newCommitTime = "001"; + String initCommitTime = "000"; + int numRecords = 200; + JavaRDD result = + insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, insertFn, isPrepped, + true, numRecords); + JavaRDD recordRDD = + jsc.parallelize( + result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) + .collect(Collectors.toList())); + // Should have 100 records in table (check using Index), all in locations marked at commit + HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); + List taggedRecords = readClient.tagLocation(recordRDD).collect(); + checkTaggedRecords(taggedRecords, newCommitTime); + + // Write 2 (updates) + String prevCommitTime = newCommitTime; + newCommitTime = "004"; + numRecords = 100; + String commitTimeBetweenPrevAndNew = "002"; + result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, + Optional.of(Arrays.asList(commitTimeBetweenPrevAndNew)), + initCommitTime, numRecords, updateFn, isPrepped, + true, numRecords, 200, 2); + recordRDD = + jsc.parallelize( + result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) + .collect(Collectors.toList())); + // Index should be able to locate all updates in correct locations. + readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); + taggedRecords = readClient.tagLocation(recordRDD).collect(); + checkTaggedRecords(taggedRecords, newCommitTime); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index b87e1768c95c7..d092c8304e040 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -99,7 +99,6 @@ public static void fakeDataFile(String basePath, String partitionPath, String co } public static SparkConf getSparkConfForTest(String appName) { - System.out.println("HIII" + "HII2"); SparkConf sparkConf = new SparkConf().setAppName(appName) .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .setMaster("local[1]"); @@ -125,6 +124,7 @@ public static Dataset readCommit(String basePath, SQLContext sqlContext, Ho try { HashMap paths = getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant)); + System.out.println("Path :" + paths.values()); return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()])) .filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime)); } catch (Exception e) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index f623fc7cbe235..b6dc2871d0067 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -28,9 +28,12 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Random; +import java.util.Set; import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -48,7 +51,17 @@ public class HoodieTestDataGenerator { // based on examination of sample file, the schema produces the following per record size public static final int SIZE_PER_RECORD = 50 * 1024; - public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; + public static final String DEFAULT_FIRST_PARTITION_PATH = "2016/03/15"; + public static final String DEFAULT_SECOND_PARTITION_PATH = "2015/03/16"; + public static final String DEFAULT_THIRD_PARTITION_PATH = "2015/03/17"; + + public static final String[] DEFAULT_PARTITION_PATHS = { + DEFAULT_FIRST_PARTITION_PATH, + DEFAULT_SECOND_PARTITION_PATH, + DEFAULT_THIRD_PARTITION_PATH + }; + public static final int DEFAULT_PARTITION_DEPTH = 3; + public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"double\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"}," @@ -62,14 +75,14 @@ public class HoodieTestDataGenerator { public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); private static Random rand = new Random(46474747); private List existingKeysList = new ArrayList<>(); - private String[] partitionPaths = DEFAULT_PARTITION_PATHS; + private String[] partitionPaths; public HoodieTestDataGenerator(String[] partitionPaths) { - this.partitionPaths = partitionPaths; + this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length); } public HoodieTestDataGenerator() { - this(new String[] {"2016/03/15", "2015/03/16", "2015/03/17"}); + this(DEFAULT_PARTITION_PATHS); } public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) { @@ -133,7 +146,7 @@ public static void createSavepointFile(String basePath, String commitTime) throw /** * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ - public List generateInserts(String commitTime, int n) throws IOException { + public List generateInserts(String commitTime, Integer n) throws IOException { List inserts = new ArrayList<>(); for (int i = 0; i < n; i++) { String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)]; @@ -149,7 +162,7 @@ public List generateInserts(String commitTime, int n) throws IOExc return inserts; } - public List generateDeletes(String commitTime, int n) throws IOException { + public List generateDeletes(String commitTime, Integer n) throws IOException { List inserts = generateInserts(commitTime, n); return generateDeletesFromExistingRecords(inserts); } @@ -159,36 +172,77 @@ public List generateDeletesFromExistingRecords(List for (HoodieRecord existingRecord : existingRecords) { HoodieRecord record = generateDeleteRecord(existingRecord); deletes.add(record); - } return deletes; } public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException { HoodieKey key = existingRecord.getKey(); + return generateDeleteRecord(key); + } + + public HoodieRecord generateDeleteRecord(HoodieKey key) throws IOException { TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), null, true); return new HoodieRecord(key, payload); } + public HoodieRecord generateUpdateRecord(HoodieKey key, String commitTime) throws IOException { + return new HoodieRecord(key, generateRandomValue(key, commitTime)); + } + public List generateUpdates(String commitTime, List baseRecords) throws IOException { List updates = new ArrayList<>(); for (HoodieRecord baseRecord : baseRecords) { - HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime)); + HoodieRecord record = generateUpdateRecord(baseRecord.getKey(), commitTime); updates.add(record); } return updates; } /** - * Generates new updates, randomly distributed across the keys above. + * Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list + * @param commitTime Commit Timestamp + * @param n Number of updates (including dups) + * @return list of hoodie record updates + * @throws IOException */ - public List generateUpdates(String commitTime, int n) throws IOException { + public List generateUpdates(String commitTime, Integer n) throws IOException { List updates = new ArrayList<>(); for (int i = 0; i < n; i++) { KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1)); + HoodieRecord record = generateUpdateRecord(kp.key, commitTime); + updates.add(record); + } + return updates; + } + + /** + * Generates deduped updates of keys previously inserted, randomly distributed across the keys above. + * @param commitTime Commit Timestamp + * @param n Number of unique records + * @return list of hoodie record updates + * @throws IOException + */ + public List generateUniqueUpdates(String commitTime, Integer n) throws IOException { + List updates = new ArrayList<>(); + Set used = new HashSet<>(); + + if (n > existingKeysList.size()) { + throw new IllegalArgumentException("Requested unique updates is greater than number of available keys"); + } + + for (int i = 0; i < n; i++) { + int index = rand.nextInt(existingKeysList.size() - 1); + KeyPartition kp = existingKeysList.get(index); + // Find the available keyPartition starting from randomly chosen one. + while (used.contains(kp)) { + index = (index + 1) % existingKeysList.size(); + kp = existingKeysList.get(index); + } HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime)); updates.add(record); + used.add(kp); } return updates; } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 361bd81e7ec54..742b05b3f70fb 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -34,6 +34,7 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; @@ -45,6 +46,7 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Calendar; import java.util.Date; import java.util.Iterator; import java.util.List; @@ -310,4 +312,15 @@ public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath } return returns.toArray(new FileStatus[returns.size()]); } + + public static List monotonicIncreasingCommitTimestamps(int numTimestamps, int startSecsDelta) { + Calendar cal = Calendar.getInstance(); + cal.add(Calendar.SECOND, startSecsDelta); + List commits = new ArrayList<>(); + for (int i = 0; i < numTimestamps; i++) { + commits.add(HoodieActiveTimeline.COMMIT_FORMATTER.format(cal.getTime())); + cal.add(Calendar.SECOND, 1); + } + return commits; + } } From 2a28ee09964310ebd205c016c8a1237bd8e1cbc6 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 1 Apr 2018 21:43:05 -0700 Subject: [PATCH 044/374] Parallelized read-write operations in Hoodie Merge phase --- .../uber/hoodie/func/BufferedIterator.java | 66 ++++---------- .../hoodie/func/BufferedIteratorExecutor.java | 89 ++++++++++++++++++ .../uber/hoodie/func/LazyInsertIterable.java | 91 +++++++++---------- .../hoodie/func/ParquetReaderIterator.java | 72 +++++++++++++++ .../AbstractBufferedIteratorPayload.java | 42 +++++++++ .../GenericRecordBufferedIteratorPayload.java | 31 +++++++ .../HoodieRecordBufferedIteratorPayload.java | 47 ++++++++++ .../com/uber/hoodie/io/HoodieMergeHandle.java | 10 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 62 ++++++++++--- .../hoodie/func/TestBufferedIterator.java | 24 +++-- .../func/TestBufferedIteratorExecutor.java | 79 ++++++++++++++++ .../func/TestParquetReaderIterator.java | 61 +++++++++++++ .../common/util/collection/DiskBasedMap.java | 5 +- 13 files changed, 553 insertions(+), 126 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/func/TestParquetReaderIterator.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java index e69c09b1ac784..58616408491da 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java @@ -18,8 +18,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.exception.HoodieException; import java.util.Iterator; import java.util.Optional; @@ -29,8 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; +import java.util.function.Function; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.util.SizeEstimator; @@ -39,9 +36,11 @@ * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It * internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in * buffer accordingly. This is done to ensure that we don't OOM. + * + * @param input payload data type + * @param output payload data type */ -public class BufferedIterator> implements - Iterator> { +public class BufferedIterator implements Iterator { // interval used for polling records in the queue. public static final int RECORD_POLL_INTERVAL_SEC = 5; @@ -58,20 +57,18 @@ public class BufferedIterator>> buffer = new + private final LinkedBlockingQueue> buffer = new LinkedBlockingQueue<>(); // maximum amount of memory to be used for buffering records. private final long bufferMemoryLimit; // original iterator from where records are read for buffering. - private final Iterator inputIterator; + private final Iterator inputIterator; // it holds the root cause of the exception in case either buffering records (reading from // inputIterator) fails or // thread reading records from buffer fails. private final AtomicReference hasFailed = new AtomicReference(null); // used for indicating that all the records from buffer are read successfully. private final AtomicBoolean isDone = new AtomicBoolean(false); - // schema used for fetching insertValue from HoodieRecord. - private final Schema schema; // indicates rate limit (number of records to cache). it is updated whenever there is a change // in avg record size. @VisibleForTesting @@ -82,13 +79,15 @@ public class BufferedIterator nextRecord; + private O nextRecord; + // Function to transform the input payload to the expected output payload + private Function bufferedIteratorTransform; - public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, - final Schema schema) { + public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, + final Function bufferedIteratorTransform) { this.inputIterator = iterator; this.bufferMemoryLimit = bufferMemoryLimit; - this.schema = schema; + this.bufferedIteratorTransform = bufferedIteratorTransform; } @VisibleForTesting @@ -101,7 +100,7 @@ public int size() { // for determining how many maximum records to buffer. Based on change in avg size it may // increase or decrease // available permits. - private void adjustBufferSizeIfNeeded(final T record) throws InterruptedException { + private void adjustBufferSizeIfNeeded(final I record) throws InterruptedException { if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { return; } @@ -110,14 +109,8 @@ private void adjustBufferSizeIfNeeded(final T record) throws InterruptedExceptio .max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); final int newRateLimit = (int) Math .min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes)); - // System.out.println("recordSizeInBytes:" + recordSizeInBytes + ":newAvgRecordSizeInBytes:" + - // newAvgRecordSizeInBytes - // + ":newRateLimit:" + newRateLimit + ":currentRateLimit:" + currentRateLimit + - // ":numSamples:" + numSamples - // + ":avgRecordSizeInBytes:" + avgRecordSizeInBytes); - // If there is any change in number of records to cache then we will either release (if it - // increased) or acquire + // If there is any change in number of records to cache then we will either release (if it increased) or acquire // (if it decreased) to adjust rate limiting to newly computed value. if (newRateLimit > currentRateLimit) { rateLimiter.release(newRateLimit - currentRateLimit); @@ -132,19 +125,19 @@ private void adjustBufferSizeIfNeeded(final T record) throws InterruptedExceptio // inserts record into internal buffer. It also fetches insert value from the record to offload // computation work on to // buffering thread. - private void insertRecord(T t) throws Exception { + private void insertRecord(I t) throws Exception { rateLimiter.acquire(); adjustBufferSizeIfNeeded(t); // We are retrieving insert value in the record buffering thread to offload computation // around schema validation // and record creation to it. - final BufferedIteratorPayload payload = new BufferedIteratorPayload<>(t, this.schema); + final O payload = bufferedIteratorTransform.apply(t); buffer.put(Optional.of(payload)); } private void readNextRecord() { rateLimiter.release(); - Optional> newRecord; + Optional newRecord; while (true) { try { throwExceptionIfFailed(); @@ -194,9 +187,9 @@ public boolean hasNext() { } @Override - public BufferedIteratorPayload next() { + public O next() { Preconditions.checkState(hasNext() && this.nextRecord != null); - final BufferedIteratorPayload ret = this.nextRecord; + final O ret = this.nextRecord; this.nextRecord = null; return ret; } @@ -213,23 +206,4 @@ public void markAsFailed(Exception e) { // get it. this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); } - - // Used for caching HoodieRecord along with insertValue. We need this to offload computation - // work to buffering thread. - static class BufferedIteratorPayload { - - public T record; - public Optional insertValue; - // It caches the exception seen while fetching insert value. - public Optional exception = Optional.empty(); - - public BufferedIteratorPayload(T record, Schema schema) { - this.record = record; - try { - this.insertValue = record.getData().getInsertValue(schema); - } catch (Exception e) { - this.exception = Optional.of(e); - } - } - } } \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java new file mode 100644 index 0000000000000..fea0f28bbcce2 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java @@ -0,0 +1,89 @@ +/* + * 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.func; + +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieException; +import java.util.Iterator; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.function.Function; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; + +/** + * Executor for a BufferedIterator operation. This class takes as input the input iterator which + * needs to be buffered, the runnable function that needs to be executed in the reader thread and + * return the transformed output based on the writer function + */ +public class BufferedIteratorExecutor { + + private static Logger logger = LogManager.getLogger(BufferedIteratorExecutor.class); + + // Executor service used for launching writer thread. + final ExecutorService writerService; + // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. + final BufferedIterator bufferedIterator; + // Need to set current spark thread's TaskContext into newly launched thread so that new + // thread can access + // TaskContext properties. + final TaskContext sparkThreadTaskContext; + + public BufferedIteratorExecutor(final HoodieWriteConfig hoodieConfig, final Iterator inputItr, + final Function bufferedIteratorTransform, + final ExecutorService writerService) { + this.sparkThreadTaskContext = TaskContext.get(); + this.writerService = writerService; + this.bufferedIterator = new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(), + bufferedIteratorTransform); + } + + /** + * Starts buffering and executing the writer function + */ + public Future start(Function writerFunction) { + try { + Future future = writerService.submit( + () -> { + logger.info("starting hoodie writer thread"); + // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext + // properties. + TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + try { + E result = writerFunction.apply(bufferedIterator); + logger.info("hoodie write is done; notifying reader thread"); + return result; + } catch (Exception e) { + logger.error("error writing hoodie records", e); + bufferedIterator.markAsFailed(e); + throw e; + } + }); + bufferedIterator.startBuffering(); + return future; + } catch (Exception e) { + throw new HoodieException(e); + } + } + + public boolean isRemaining() { + return bufferedIterator.hasNext(); + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index 6e1ae02c4370e..36aba53054e6f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -21,6 +21,8 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; +import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload; import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; @@ -29,14 +31,15 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import java.util.function.Function; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -45,7 +48,6 @@ public class LazyInsertIterable extends LazyIterableIterator, List> { - private static Logger logger = LogManager.getLogger(LazyInsertIterable.class); private final HoodieWriteConfig hoodieConfig; private final String commitTime; private final HoodieTable hoodieTable; @@ -65,44 +67,35 @@ public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWrite protected void start() { } + /** + * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some + * expensive operations of transformation to the reader thread. + * @param schema + * @param + * @return + */ + public static Function, AbstractBufferedIteratorPayload> + bufferedItrPayloadTransform(Schema schema) { + return (hoodieRecord) -> new HoodieRecordBufferedIteratorPayload(hoodieRecord, schema); + } + @Override protected List computeNext() { - // Need to set current spark thread's TaskContext into newly launched thread so that new - // thread can access - // TaskContext properties. - final TaskContext sparkThreadTaskContext = TaskContext.get(); // Executor service used for launching writer thread. final ExecutorService writerService = Executors.newFixedThreadPool(1); try { - // Used for buffering records which is controlled by - // HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - final BufferedIterator> bufferedIterator = new BufferedIterator<>(inputItr, - hoodieConfig.getWriteBufferLimitBytes(), - HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)); - Future> writerResult = writerService.submit(() -> { - logger.info("starting hoodie writer thread"); - // Passing parent thread's TaskContext to newly launched thread for it to access original - // TaskContext - // properties. - TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + Function> function = (bufferedIterator) -> { List statuses = new LinkedList<>(); - try { - statuses.addAll(handleWrite(bufferedIterator)); - logger.info("hoodie write is done; notifying reader thread"); - return statuses; - } catch (Exception e) { - logger.error("error writing hoodie records", e); - bufferedIterator.markAsFailed(e); - throw e; - } - }); - // Buffering records into internal buffer. This can throw exception either if reading - // records from spark fails or - // if writing buffered records into parquet file fails. - bufferedIterator.startBuffering(); - logger.info("waiting for hoodie write to finish"); + statuses.addAll(handleWrite(bufferedIterator)); + return statuses; + }; + BufferedIteratorExecutor, AbstractBufferedIteratorPayload, List> + bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieConfig, inputItr, + bufferedItrPayloadTransform(HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)), + writerService); + Future> writerResult = bufferedIteratorExecutor.start(function); final List result = writerResult.get(); - assert result != null && !result.isEmpty() && !bufferedIterator.hasNext(); + assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); return result; } catch (Exception e) { throw new HoodieException(e); @@ -112,38 +105,38 @@ protected List computeNext() { } private List handleWrite( - final BufferedIterator> bufferedIterator) { + final BufferedIterator, AbstractBufferedIteratorPayload> bufferedIterator) { List statuses = new ArrayList<>(); while (bufferedIterator.hasNext()) { - final BufferedIterator.BufferedIteratorPayload> payload = bufferedIterator + final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator .next(); - + final HoodieRecord insertPayload = (HoodieRecord) payload.getInputPayload(); // clean up any partial failures - if (!partitionsCleaned.contains(payload.record.getPartitionPath())) { + if (!partitionsCleaned + .contains(insertPayload.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with // the same data again. Thus, before we open any files under a given partition, we // first delete any files in the same partitionPath written by same Spark partition - HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, - payload.record.getPartitionPath(), TaskContext.getPartitionId(), hoodieTable); - partitionsCleaned.add(payload.record.getPartitionPath()); + HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, insertPayload.getPartitionPath(), + TaskContext.getPartitionId(), hoodieTable); + partitionsCleaned.add(insertPayload.getPartitionPath()); } // lazily initialize the handle, for the first time if (handle == null) { - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, - payload.record.getPartitionPath()); + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); } - if (handle.canWrite(payload.record)) { + if (handle.canWrite(((HoodieRecord) payload.getInputPayload()))) { // write the payload, if the handle has capacity - handle.write(payload.record, payload.insertValue, payload.exception); + handle.write(insertPayload, (Optional) payload.getOutputPayload(), payload.exception); } else { // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, - payload.record.getPartitionPath()); - handle.write(payload.record, payload.insertValue, + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); + handle.write(insertPayload, + (Optional) payload.getOutputPayload(), payload.exception); // we should be able to write 1 payload. } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java new file mode 100644 index 0000000000000..ed81da0359c32 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java @@ -0,0 +1,72 @@ +/* + * 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.func; + +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Iterator; +import org.apache.parquet.hadoop.ParquetReader; + +/** + * This class wraps a parquet reader and provides an iterator based api to + * read from a parquet file. This is used in {@link BufferedIterator} + */ +public class ParquetReaderIterator implements Iterator { + + // Parquet reader for an existing parquet file + private final ParquetReader parquetReader; + // Holds the next entry returned by the parquet reader + private T next; + + public ParquetReaderIterator(ParquetReader parquetReader) { + this.parquetReader = parquetReader; + } + + @Override + public boolean hasNext() { + try { + // To handle when hasNext() is called multiple times for idempotency and/or the first time + if (this.next == null) { + this.next = parquetReader.read(); + } + return this.next != null; + } catch (IOException io) { + throw new HoodieIOException("unable to read next record from parquet file ", io); + } + } + + @Override + public T next() { + try { + // To handle case when next() is called before hasNext() + if (this.next == null) { + if (!hasNext()) { + throw new HoodieIOException("No more records left to read from parquet file"); + } + } + T retVal = this.next; + this.next = parquetReader.read(); + return retVal; + } catch (IOException io) { + throw new HoodieIOException("unable to read next record from parquet file ", io); + } + } + + public void close() throws IOException { + parquetReader.close(); + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java new file mode 100644 index 0000000000000..4a7e322243535 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java @@ -0,0 +1,42 @@ +/* + * 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.func.payload; + +/** + * @param Input data type for BufferedIterator + * @param Output data type for BufferedIterator + */ +public abstract class AbstractBufferedIteratorPayload { + + // input payload for iterator + protected I inputPayload; + // output payload for iterator, this is used in cases where the output payload is computed + // from the input payload and most of this computation is off-loaded to the reader + protected O outputPayload; + + public AbstractBufferedIteratorPayload(I record) { + this.inputPayload = record; + } + + public I getInputPayload() { + return inputPayload; + } + + public O getOutputPayload() { + return outputPayload; + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java new file mode 100644 index 0000000000000..9d934d9ea7a8f --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java @@ -0,0 +1,31 @@ +/* + * 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.func.payload; + +import org.apache.avro.generic.GenericRecord; + +/** + * BufferedIteratorPayload that takes GenericRecord as input and GenericRecord as output + */ +public class GenericRecordBufferedIteratorPayload + extends AbstractBufferedIteratorPayload { + + public GenericRecordBufferedIteratorPayload(GenericRecord record) { + super(record); + this.outputPayload = record; + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java new file mode 100644 index 0000000000000..a79f0e01ab499 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java @@ -0,0 +1,47 @@ +/* + * 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.func.payload; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import java.util.Optional; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; + +/** + * BufferedIteratorPayload that takes HoodieRecord as input and transforms to output Optional + * @param + */ +public class HoodieRecordBufferedIteratorPayload + extends AbstractBufferedIteratorPayload, Optional> { + + // It caches the exception seen while fetching insert value. + public Optional exception = Optional.empty(); + + public HoodieRecordBufferedIteratorPayload(HoodieRecord record, Schema schema) { + super(record); + try { + this.outputPayload = record.getData().getInsertValue(schema); + } catch (Exception e) { + this.exception = Optional.of(e); + } + } + + public Optional getException() { + return exception; + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index dc2930aae477e..f5592c3811ad6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -135,8 +135,8 @@ private String init(String fileId, Iterator> newRecordsItr) { try { // Load the new records in a map logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); - this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), - Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass())); + this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), Optional.empty(), + new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass())); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } @@ -148,7 +148,7 @@ private String init(String fileId, Iterator> newRecordsItr) { // update the new location of the record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); } - logger.debug("Number of entries in MemoryBasedMap => " + logger.info("Number of entries in MemoryBasedMap => " + ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() + "Total size in bytes of MemoryBasedMap => " + ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() @@ -156,7 +156,6 @@ private String init(String fileId, Iterator> newRecordsItr) { + ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + "Size of file spilled to disk => " + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); - return partitionPath; } @@ -186,7 +185,8 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, } /** - * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file. + * Go through an old record. Here if we detect a newer version shows up, we write the new one to + * the file. */ public void write(GenericRecord oldRecord) { String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index db32a908d4586..bdf40f79ab086 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -34,10 +34,16 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieUpsertException; +import com.uber.hoodie.func.BufferedIterator; +import com.uber.hoodie.func.BufferedIteratorExecutor; import com.uber.hoodie.func.LazyInsertIterable; +import com.uber.hoodie.func.ParquetReaderIterator; +import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; +import com.uber.hoodie.func.payload.GenericRecordBufferedIteratorPayload; import com.uber.hoodie.io.HoodieCleanHelper; import com.uber.hoodie.io.HoodieMergeHandle; import java.io.IOException; @@ -52,6 +58,9 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -173,8 +182,19 @@ public Iterator> handleUpdate(String commitTime, String fileLo return handleUpdateInternal(upsertHandle, commitTime, fileLoc); } + /** + * Transformer function to help transform a GenericRecord. This transformer is used by BufferedIterator to offload + * some expensive operations of transformation to the reader thread. + * + */ + public static java.util.function.Function + bufferedItrPayloadTransform() { + return (genericRecord) -> new GenericRecordBufferedIteratorPayload(genericRecord); + } + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, - String commitTime, String fileLoc) throws IOException { + String commitTime, String fileLoc) + throws IOException { if (upsertHandle.getOldFilePath() == null) { throw new HoodieUpsertException( "Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc); @@ -182,32 +202,44 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) .withConf(getHadoopConf()).build(); + final ExecutorService writerService = Executors.newFixedThreadPool(1); try { - IndexedRecord record; - while ((record = reader.read()) != null) { - // Two types of writes here (new record, and old record). - // We have already catch the exception during writing new records. - // But for old records, we should fail if any exception happens. - upsertHandle.write((GenericRecord) record); - } - } catch (IOException e) { - throw new HoodieUpsertException( - "Failed to read record from " + upsertHandle.getOldFilePath() + " with new Schema " - + upsertHandle.getSchema(), e); + java.util.function.Function runnableFunction = (bufferedIterator) -> { + handleWrite(bufferedIterator, upsertHandle); + return null; + }; + BufferedIteratorExecutor wrapper = + new BufferedIteratorExecutor(config, new ParquetReaderIterator(reader), bufferedItrPayloadTransform(), + writerService); + Future writerResult = wrapper.start(runnableFunction); + writerResult.get(); + } catch (Exception e) { + throw new HoodieException(e); } finally { reader.close(); upsertHandle.close(); + writerService.shutdownNow(); } } + //TODO(vc): This needs to be revisited if (upsertHandle.getWriteStatus().getPartitionPath() == null) { - logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.getWriteStatus()); + logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + + ", " + upsertHandle.getWriteStatus()); } return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())) .iterator(); } + private void handleWrite(final BufferedIterator bufferedIterator, + final HoodieMergeHandle upsertHandle) { + while (bufferedIterator.hasNext()) { + final GenericRecordBufferedIteratorPayload payload = (GenericRecordBufferedIteratorPayload) bufferedIterator + .next(); + upsertHandle.write(payload.getOutputPayload()); + } + } + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, Iterator> recordItr) { return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); @@ -792,4 +824,4 @@ public int getPartition(Object key) { } } } -} +} \ No newline at end of file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java index 6c47f2d4c3e4d..e55db1b05f856 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java @@ -23,6 +23,8 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; +import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload; import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -66,7 +68,7 @@ public void testRecordReading() throws IOException, ExecutionException, Interrup final int numRecords = 128; final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, - HoodieTestDataGenerator.avroSchema); + LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); Future result = recordReader.submit(() -> { bufferedIterator.startBuffering(); return true; @@ -77,11 +79,12 @@ public void testRecordReading() throws IOException, ExecutionException, Interrup final HoodieRecord originalRecord = originalRecordIterator.next(); final Optional originalInsertValue = originalRecord.getData() .getInsertValue(HoodieTestDataGenerator.avroSchema); - final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next(); + final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator.next(); // Ensure that record ordering is guaranteed. - Assert.assertEquals(originalRecord, payload.record); + Assert.assertEquals(originalRecord, payload.getInputPayload()); // cached insert value matches the expected insert value. - Assert.assertEquals(originalInsertValue, payload.insertValue); + Assert.assertEquals(originalInsertValue, + ((HoodieRecord) payload.getInputPayload()).getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); recordsRead++; } Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); @@ -99,8 +102,9 @@ public void testMemoryLimitForBuffering() throws IOException, InterruptedExcepti // maximum number of records to keep in memory. final int recordLimit = 5; final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); - final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, - HoodieTestDataGenerator.avroSchema); + final BufferedIterator bufferedIterator = + new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, + LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); Future result = recordReader.submit(() -> { bufferedIterator.startBuffering(); return true; @@ -115,8 +119,8 @@ public void testMemoryLimitForBuffering() throws IOException, InterruptedExcepti Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); // try to read 2 records. - Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record); - Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record); + Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().getInputPayload()); + Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().getInputPayload()); // waiting for permits to expire. while (!isQueueFull(bufferedIterator.rateLimiter)) { @@ -145,7 +149,7 @@ public void testException() throws IOException, InterruptedException { // stops and throws // correct exception back. BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, - HoodieTestDataGenerator.avroSchema); + LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); Future result = recordReader.submit(() -> { bufferedIterator1.startBuffering(); return true; @@ -173,7 +177,7 @@ public void testException() throws IOException, InterruptedException { when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, - HoodieTestDataGenerator.avroSchema); + LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); Future result2 = recordReader.submit(() -> { bufferedIterator2.startBuffering(); return true; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java new file mode 100644 index 0000000000000..5c3695b8b1831 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java @@ -0,0 +1,79 @@ +/* + * 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.func; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.config.HoodieWriteConfig; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.Function; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestBufferedIteratorExecutor { + + private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); + private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); + private ExecutorService executorService = null; + + @Before + public void beforeTest() { + this.executorService = Executors.newFixedThreadPool(1); + } + + @After + public void afterTest() { + if (this.executorService != null) { + this.executorService.shutdownNow(); + this.executorService = null; + } + } + + @Test + public void testExecutor() throws Exception { + + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, 100); + + HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); + when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + BufferedIteratorExecutor bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieWriteConfig, + hoodieRecords.iterator(), LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema), + executorService); + Function function = (bufferedIterator) -> { + Integer count = 0; + while (bufferedIterator.hasNext()) { + count++; + bufferedIterator.next(); + } + return count; + }; + Future future = bufferedIteratorExecutor.start(function); + // It should buffer and write 100 records + Assert.assertEquals((int) future.get(), 100); + // There should be no remaining records in the buffer + Assert.assertFalse(bufferedIteratorExecutor.isRemaining()); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestParquetReaderIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestParquetReaderIterator.java new file mode 100644 index 0000000000000..a082894fe0557 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestParquetReaderIterator.java @@ -0,0 +1,61 @@ +/* + * 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.func; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import org.apache.parquet.hadoop.ParquetReader; +import org.junit.Assert; +import org.junit.Test; + +public class TestParquetReaderIterator { + + @Test + public void testParquetIteratorIdempotency() throws IOException { + ParquetReader reader = mock(ParquetReader.class); + // only 1 record in reader + when(reader.read()).thenReturn(1).thenReturn(null); + ParquetReaderIterator iterator = new ParquetReaderIterator<>(reader); + int idempotencyCheckCounter = 0; + // call hasNext() 3 times + while (idempotencyCheckCounter < 3) { + Assert.assertTrue(iterator.hasNext()); + idempotencyCheckCounter++; + } + } + + @Test + public void testParquetIterator() throws IOException { + + ParquetReader reader = mock(ParquetReader.class); + // only one record to read + when(reader.read()).thenReturn(1).thenReturn(null); + ParquetReaderIterator iterator = new ParquetReaderIterator<>(reader); + // should return value even though hasNext() hasn't been called + Assert.assertTrue(iterator.next() == 1); + // no more entries to iterate on + Assert.assertFalse(iterator.hasNext()); + try { + iterator.next(); + } catch (HoodieIOException e) { + // should throw an exception since there is only 1 record + } + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java index 0c3ab8626a409..7ae620ef65c44 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -26,6 +26,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.RandomAccessFile; +import java.net.InetAddress; import java.util.AbstractMap; import java.util.Collection; import java.util.Date; @@ -94,7 +95,9 @@ private void initFile(File writeOnlyFileHandle) throws IOException { } writeOnlyFileHandle.createNewFile(); - log.info("Spilling to file location " + writeOnlyFileHandle.getAbsolutePath()); + log.info( + "Spilling to file location " + writeOnlyFileHandle.getAbsolutePath() + " in host (" + InetAddress.getLocalHost() + .getHostAddress() + ") with hostname (" + InetAddress.getLocalHost().getHostName() + ")"); // Open file in readFromDisk-only mode readOnlyFileHandle = new RandomAccessFile(filePath, "r"); readOnlyFileHandle.seek(0); From 14210e96e2f2fa26928d7072e04bf23759faca18 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Mon, 2 Apr 2018 22:53:28 -0700 Subject: [PATCH 045/374] Using BufferedFsInputStream to wrap FSInputStream for FSDataInputStream --- .../cli/commands/HoodieLogFileCommand.java | 6 +- .../hoodie/config/HoodieMemoryConfig.java | 12 ++ .../uber/hoodie/config/HoodieWriteConfig.java | 12 ++ .../compact/HoodieRealtimeTableCompactor.java | 2 +- .../common/HoodieTestDataGenerator.java | 15 ++- .../hoodie/io/TestHoodieCommitArchiveLog.java | 120 +++++++++++------- .../uber/hoodie/io/TestHoodieCompactor.java | 2 + .../log/HoodieCompactedLogRecordScanner.java | 4 +- .../common/table/log/HoodieLogFileReader.java | 75 ++++++----- .../common/table/log/HoodieLogFormat.java | 2 +- .../table/log/HoodieLogFormatReader.java | 15 +-- .../hoodie/common/model/HoodieTestUtils.java | 22 +++- .../common/table/log/HoodieLogFormatTest.java | 42 +++--- .../realtime/HoodieRealtimeRecordReader.java | 9 +- .../HoodieRealtimeRecordReaderTest.java | 1 + 15 files changed, 222 insertions(+), 117 deletions(-) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 9d9790bba8e74..78bd834dbee2f 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -92,6 +92,9 @@ public String showLogFileCommits( if (n instanceof HoodieCorruptBlock) { try { instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME); + if (instantTime == null) { + throw new Exception("Invalid instant time " + instantTime); + } } catch (Exception e) { numCorruptBlocks++; instantTime = "corrupt_block_" + numCorruptBlocks; @@ -172,7 +175,8 @@ public String showLogFileRecords(@CliOption(key = { .getTimestamp(), Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES), Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), - Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED)); + Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED), + Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); for (HoodieRecord hoodieRecord : scanner) { Optional record = hoodieRecord.getData().getInsertValue(readerSchema); if (allRecords.size() >= limit) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java index aa5f28cc74229..3fd6d6c4e6305 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java @@ -45,6 +45,9 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig { public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size"; // Property to set the max memory for compaction public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size"; + // Property to set the max memory for dfs inputstream buffer size + public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size"; + public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 16 * 1024 * 1024; // 16MB private HoodieMemoryConfig(Properties props) { @@ -86,6 +89,12 @@ public Builder withMaxMemoryFractionPerCompaction(long maxMemoryFractionPerCompa return this; } + public Builder withMaxDFSStreamBufferSize(int maxStreamBufferSize) { + props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP, + String.valueOf(maxStreamBufferSize)); + return this; + } + /** * Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory * * (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime @@ -143,6 +152,9 @@ public HoodieMemoryConfig build() { !props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP), MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf( getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP)))); + setDefaultOnCondition(props, + !props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP), + MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 683ec952d7f3d..633e5695b82c4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -369,6 +369,12 @@ public Long getMaxMemoryPerCompaction() { props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP)); } + public int getMaxDFSStreamBufferSize() { + return Integer + .valueOf( + props.getProperty(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP)); + } + public static class Builder { private final Properties props = new Properties(); @@ -469,6 +475,12 @@ public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) { return this; } + public Builder withMemoryConfig(HoodieMemoryConfig memoryConfig) { + props.putAll(memoryConfig.getProps()); + isMemoryConfigSet = true; + return this; + } + public Builder withAutoCommit(boolean autoCommit) { props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit)); return this; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 8d4f9b7a319d3..9d68fe1baf4ce 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -105,7 +105,7 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(), - config.getCompactionReverseLogReadEnabled()); + config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize()); if (!scanner.iterator().hasNext()) { return Lists.newArrayList(); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index b6dc2871d0067..99af0e2e31643 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -38,6 +38,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -116,9 +117,14 @@ public static GenericRecord generateGenericRecord(String rowKey, String riderNam } public static void createCommitFile(String basePath, String commitTime) throws IOException { + createCommitFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); + } + + public static void createCommitFile(String basePath, String commitTime, Configuration configuration) + throws IOException { Path commitFile = new Path( basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FileSystem fs = FSUtils.getFs(basePath, configuration); FSDataOutputStream os = fs.create(commitFile, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); try { @@ -130,9 +136,14 @@ public static void createCommitFile(String basePath, String commitTime) throws I } public static void createSavepointFile(String basePath, String commitTime) throws IOException { + createSavepointFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); + } + + public static void createSavepointFile(String basePath, String commitTime, Configuration configuration) + throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeSavePointFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FileSystem fs = FSUtils.getFs(basePath, configuration); FSDataOutputStream os = fs.create(commitFile, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); try { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index b099d88736081..0b30faf74f70c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -22,6 +22,7 @@ import com.google.common.collect.Sets; import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry; import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.minicluster.HdfsTestService; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -29,7 +30,6 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import java.io.IOException; @@ -42,23 +42,57 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.AfterClass; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.rules.TemporaryFolder; public class TestHoodieCommitArchiveLog { private String basePath; - private FileSystem fs; private Configuration hadoopConf; + //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) + //The implementation and gurantees of many API's differ, for example check rename(src,dst) + // We need to use DFS here instead of LocalFs since the FsDataInputStream.getWrappedStream() returns a + // FsDataInputStream instead of a InputStream and thus throws java.lang.ClassCastException: + // org.apache.hadoop.fs.FSDataInputStream cannot be cast to org.apache.hadoop.fs.FSInputStream + private static MiniDFSCluster dfsCluster; + private static DistributedFileSystem dfs; + private static HdfsTestService hdfsTestService; + + @AfterClass + public static void cleanUp() throws Exception { + if (hdfsTestService != null) { + hdfsTestService.stop(); + dfsCluster.shutdown(); + } + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM + FileSystem.closeAll(); + } + + @BeforeClass + public static void setUpDFS() throws IOException { + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM + FileSystem.closeAll(); + if (hdfsTestService == null) { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + // Create a temp folder as the base path + dfs = dfsCluster.getFileSystem(); + } + } @Before public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); - fs = FSUtils.getFs(basePath, hadoopConf); + hadoopConf = dfs.getConf(); HoodieTestUtils.init(hadoopConf, basePath); } @@ -68,7 +102,7 @@ public void testArchiveEmptyDataset() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").build(); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, - new HoodieTableMetaClient(fs.getConf(), cfg.getBasePath(), true)); + new HoodieTableMetaClient(dfs.getConf(), cfg.getBasePath(), true)); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); } @@ -81,26 +115,26 @@ public void testArchiveDatasetWithArchival() throws IOException { HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build()) .forTable("test-trip-table").build(); HoodieTestUtils.init(hadoopConf, basePath); - HoodieTestDataGenerator.createCommitFile(basePath, "100"); - HoodieTestDataGenerator.createCommitFile(basePath, "101"); - HoodieTestDataGenerator.createCommitFile(basePath, "102"); - HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTestDataGenerator.createCommitFile(basePath, "104"); - HoodieTestDataGenerator.createCommitFile(basePath, "105"); - - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); - HoodieTestUtils.createCleanFiles(basePath, "100"); - HoodieTestUtils.createInflightCleanFiles(basePath, "101"); - HoodieTestUtils.createCleanFiles(basePath, "101"); - HoodieTestUtils.createCleanFiles(basePath, "102"); - HoodieTestUtils.createCleanFiles(basePath, "103"); - HoodieTestUtils.createCleanFiles(basePath, "104"); - HoodieTestUtils.createCleanFiles(basePath, "105"); - HoodieTestUtils.createInflightCleanFiles(basePath, "106", "107"); + HoodieTestUtils.createCleanFiles(basePath, "100", dfs.getConf()); + HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "101"); + HoodieTestUtils.createCleanFiles(basePath, "101", dfs.getConf()); + HoodieTestUtils.createCleanFiles(basePath, "102", dfs.getConf()); + HoodieTestUtils.createCleanFiles(basePath, "103", dfs.getConf()); + HoodieTestUtils.createCleanFiles(basePath, "104", dfs.getConf()); + HoodieTestUtils.createCleanFiles(basePath, "105", dfs.getConf()); + HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "106", "107"); //reload the timeline and get all the commmits before archive timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); @@ -112,7 +146,7 @@ public void testArchiveDatasetWithArchival() throws IOException { verifyInflightInstants(metaClient, 3); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, - new HoodieTableMetaClient(fs.getConf(), basePath, true)); + new HoodieTableMetaClient(dfs.getConf(), basePath, true)); assertTrue(archiveLog.archiveIfRequired()); @@ -121,7 +155,7 @@ public void testArchiveDatasetWithArchival() throws IOException { originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); //read the file - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(dfs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema()); @@ -156,12 +190,12 @@ public void testArchiveDatasetWithNoArchival() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); - HoodieTestDataGenerator.createCommitFile(basePath, "100"); - HoodieTestDataGenerator.createCommitFile(basePath, "101"); - HoodieTestDataGenerator.createCommitFile(basePath, "102"); - HoodieTestDataGenerator.createCommitFile(basePath, "103"); + HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); @@ -177,14 +211,14 @@ public void testArchiveCommitSafety() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); - HoodieTestDataGenerator.createCommitFile(basePath, "100"); - HoodieTestDataGenerator.createCommitFile(basePath, "101"); - HoodieTestDataGenerator.createCommitFile(basePath, "102"); - HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTestDataGenerator.createCommitFile(basePath, "104"); - HoodieTestDataGenerator.createCommitFile(basePath, "105"); + HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); @@ -203,15 +237,15 @@ public void testArchiveCommitSavepointNoHole() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); - HoodieTestDataGenerator.createCommitFile(basePath, "100"); - HoodieTestDataGenerator.createCommitFile(basePath, "101"); - HoodieTestDataGenerator.createSavepointFile(basePath, "101"); - HoodieTestDataGenerator.createCommitFile(basePath, "102"); - HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTestDataGenerator.createCommitFile(basePath, "104"); - HoodieTestDataGenerator.createCommitFile(basePath, "105"); + HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createSavepointFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index 2c93f3d181dd5..83c005a9c0cec 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -32,6 +32,7 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieMemoryConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; @@ -97,6 +98,7 @@ private HoodieWriteConfig.Builder getConfigBuilder() { .withParallelism(2, 2).withCompactionConfig( HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false) .build()).withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .withMemoryConfig(HoodieMemoryConfig.newBuilder().withMaxDFSStreamBufferSize(1 * 1024 * 1024).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 7cd4ee30772ab..d23792fc7fae8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -87,7 +87,7 @@ public class HoodieCompactedLogRecordScanner implements public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, - boolean readBlocksLazily, boolean reverseReader) { + boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); @@ -102,7 +102,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List new HoodieLogFile(new Path(logFile))) - .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader); + .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize); while (logFormatReaderWrapper.hasNext()) { HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); log.info("Scanning log file " + logFile); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java index 1ccf43cfab041..dd0348b909679 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -35,7 +35,9 @@ import java.util.Map; import java.util.Optional; import org.apache.avro.Schema; +import org.apache.hadoop.fs.BufferedFSInputStream; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -47,7 +49,7 @@ */ class HoodieLogFileReader implements HoodieLogFormat.Reader { - private static final int DEFAULT_BUFFER_SIZE = 4096; + public static final int DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024; // 16 MB private static final Logger log = LogManager.getLogger(HoodieLogFileReader.class); private final FSDataInputStream inputStream; @@ -63,7 +65,9 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, boolean readBlockLazily, boolean reverseReader) throws IOException { - this.inputStream = fs.open(logFile.getPath(), bufferSize); + this.inputStream = new FSDataInputStream( + new BufferedFSInputStream((FSInputStream) fs.open(logFile.getPath(), bufferSize).getWrappedStream(), + bufferSize)); this.logFile = logFile; this.readerSchema = readerSchema; this.readBlockLazily = readBlockLazily; @@ -116,7 +120,6 @@ private HoodieLogBlock readBlock() throws IOException { Map header = null; try { - if (isOldMagic()) { // 1 Read the block type for a log block type = inputStream.readInt(); @@ -131,8 +134,7 @@ private HoodieLogBlock readBlock() throws IOException { // 1 Read the total size of the block blocksize = (int) inputStream.readLong(); } - - } catch (Exception e) { + } catch (EOFException | CorruptedLogFileException e) { // An exception reading any of the above indicates a corrupt block // Create a corrupt block by finding the next OLD_MAGIC marker or EOF return createCorruptBlock(); @@ -237,6 +239,10 @@ private boolean isBlockCorrupt(int blocksize) throws IOException { inputStream.seek(currentPos + blocksize); } catch (EOFException e) { // this is corrupt + // This seek is required because contract of seek() is different for naked DFSInputStream vs BufferedFSInputStream + // release-3.1.0-RC1/DFSInputStream.java#L1455 + // release-3.1.0-RC1/BufferedFSInputStream.java#L73 + inputStream.seek(currentPos); return true; } @@ -256,11 +262,15 @@ private long scanForNextAvailableBlockOffset() throws IOException { while (true) { long currentPos = inputStream.getPos(); try { - boolean isEOF = readMagic(); - return isEOF ? inputStream.getPos() : currentPos; - } catch (CorruptedLogFileException e) { - // No luck - advance and try again - inputStream.seek(currentPos + 1); + boolean hasNextMagic = hasNextMagic(); + if (hasNextMagic) { + return currentPos; + } else { + // No luck - advance and try again + inputStream.seek(currentPos + 1); + } + } catch (EOFException e) { + return inputStream.getPos(); } } } @@ -276,12 +286,7 @@ public void close() throws IOException { */ public boolean hasNext() { try { - boolean isEOF = readMagic(); - if (isEOF) { - return false; - } - // If not hasNext(), we either we reach EOF or throw an exception on invalid magic header - return true; + return readMagic(); } catch (IOException e) { throw new HoodieIOException("IOException when reading logfile " + logFile, e); } @@ -307,27 +312,35 @@ private boolean isOldMagic() { private boolean readMagic() throws IOException { try { - long pos = inputStream.getPos(); - // 1. Read magic header from the start of the block - inputStream.readFully(magicBuffer, 0, 6); - if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) { - inputStream.seek(pos); - // 1. Read old magic header from the start of the block - // (for backwards compatibility of older log files written without log version) - inputStream.readFully(oldMagicBuffer, 0, 4); - if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { - throw new CorruptedLogFileException( - logFile - + "could not be read. Did not find the magic bytes at the start of the block"); - } + boolean hasMagic = hasNextMagic(); + if (!hasMagic) { + throw new CorruptedLogFileException( + logFile + + "could not be read. Did not find the magic bytes at the start of the block"); } - return false; + return hasMagic; } catch (EOFException e) { // We have reached the EOF - return true; + return false; } } + private boolean hasNextMagic() throws IOException { + long pos = inputStream.getPos(); + // 1. Read magic header from the start of the block + inputStream.readFully(magicBuffer, 0, 6); + if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) { + inputStream.seek(pos); + // 1. Read old magic header from the start of the block + // (for backwards compatibility of older log files written without log version) + inputStream.readFully(oldMagicBuffer, 0, 4); + if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { + return false; + } + } + return true; + } + @Override public HoodieLogBlock next() { try { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java index cb12f9ef8aff1..30d0093d80bee 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java @@ -212,7 +212,7 @@ static WriterBuilder newWriterBuilder() { static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { - return new HoodieLogFileReader(fs, logFile, readerSchema, false, false); + return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index 00a3a74985e76..8e8033b5b8b2a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -34,28 +34,24 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final Schema readerSchema; private final boolean readBlocksLazily; private final boolean reverseLogReader; + private int bufferSize; private static final Logger log = LogManager.getLogger(HoodieLogFormatReader.class); HoodieLogFormatReader(FileSystem fs, List logFiles, - Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { + Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader, int bufferSize) throws IOException { this.logFiles = logFiles; this.fs = fs; this.readerSchema = readerSchema; this.readBlocksLazily = readBlocksLazily; this.reverseLogReader = reverseLogReader; + this.bufferSize = bufferSize; if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); - this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, - false); + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false); } } - HoodieLogFormatReader(FileSystem fs, List logFiles, - Schema readerSchema) throws IOException { - this(fs, logFiles, readerSchema, false, false); - } - @Override public void close() throws IOException { if (currentReader != null) { @@ -73,8 +69,7 @@ public boolean hasNext() { } else if (logFiles.size() > 0) { try { HoodieLogFile nextLogFile = logFiles.remove(0); - this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, - readBlocksLazily, + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false); } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 742b05b3f70fb..6aab1e1cd29b2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -122,14 +122,21 @@ public static final void createInflightCommitFiles(String basePath, String... co } } - public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException { + public static final void createInflightCleanFiles(String basePath, Configuration configuration, String... commitTimes) + throws IOException { for (String commitTime : commitTimes) { - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline + Path commitFile = new Path((basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline .makeInflightCleanerFileName( - commitTime)).createNewFile(); + commitTime))); + FileSystem fs = FSUtils.getFs(basePath, configuration); + FSDataOutputStream os = fs.create(commitFile, true); } } + public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException { + createInflightCleanFiles(basePath, HoodieTestUtils.getDefaultHadoopConf(), commitTimes); + } + public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) throws IOException { String fileID = UUID.randomUUID().toString(); @@ -214,10 +221,11 @@ public static String makeInflightTestFileName(String instant) { return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION; } - public static void createCleanFiles(String basePath, String commitTime) throws IOException { + public static void createCleanFiles(String basePath, String commitTime, Configuration configuration) + throws IOException { Path commitFile = new Path( basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FileSystem fs = FSUtils.getFs(basePath, configuration); FSDataOutputStream os = fs.create(commitFile, true); try { HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, @@ -233,6 +241,10 @@ public static void createCleanFiles(String basePath, String commitTime) throws I } } + public static void createCleanFiles(String basePath, String commitTime) throws IOException { + createCleanFiles(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); + } + public static String makeTestFileName(String instant) { return instant + TEST_EXTENSION; } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 3d37e6816c25d..93eed42798956 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; @@ -75,6 +76,7 @@ public class HoodieLogFormatTest { private FileSystem fs; private Path partitionPath; private static String basePath; + private int bufferSize = 4096; private Boolean readBlocksLazily = true; @@ -399,7 +401,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect // scan all log blocks (across multiple log files) HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", - 10240L, readBlocksLazily, false); + 10240L, readBlocksLazily, false, bufferSize); List scannedRecords = new ArrayList<>(); for (HoodieRecord record : scanner) { @@ -525,7 +527,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false); + "100", 10240L, readBlocksLazily, false, bufferSize); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -585,7 +587,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "102", 10240L, readBlocksLazily, false); + "102", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -663,7 +665,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "103", 10240L, true, false); + "103", 10240L, true, false, bufferSize); assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -717,7 +719,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "102", 10240L, readBlocksLazily, false); + "102", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -737,7 +739,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect readKeys.clear(); scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, - false); + false, bufferSize); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); } @@ -798,7 +800,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect // all data must be rolled back before merge HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false); + "100", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(); @@ -847,7 +849,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false); + "100", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -879,7 +881,7 @@ public void testAvroLogRecordReaderWithInvalidRollback() "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false); + "100", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -929,7 +931,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "101", 10240L, readBlocksLazily, false); + "101", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -948,7 +950,6 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); @@ -956,7 +957,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() writer = writer.appendBlock(dataBlock); writer.close(); - // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + // Append some arbit byte[] to the end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with @@ -969,7 +970,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() outputStream.flush(); outputStream.close(); - // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + // Append some arbit byte[] to the end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); outputStream = fs.append(writer.getLogFile().getPath()); // create a block with @@ -989,7 +990,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() writer = writer.appendBlock(dataBlock); writer.close(); - // Append some arbit byte[] to thee end of the log (mimics a partially written commit) + // Append some arbit byte[] to the end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); outputStream = fs.append(writer.getLogFile().getPath()); // create a block with @@ -1005,18 +1006,20 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); - // Write 1 rollback block for a failed write + // Write 1 rollback block for the last commit instant header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); + header.put(HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); + writer.close(); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "101", 10240L, readBlocksLazily, false); + "101", 10240L, readBlocksLazily, false, bufferSize); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -1136,7 +1139,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect writer.close(); HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), - readBlocksLazily, true); + bufferSize, readBlocksLazily, true); assertTrue("Last block should be available", reader.hasPrev()); HoodieLogBlock prevBlock = reader.prev(); @@ -1207,7 +1210,8 @@ public void testAppendAndReadOnCorruptedLogInReverse() throws IOException, URISy writer.close(); // First round of reads - we should be able to read the first block and then EOF - HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), schema, bufferSize, + readBlocksLazily, true); assertTrue("Last block should be available", reader.hasPrev()); HoodieLogBlock block = reader.prev(); @@ -1261,7 +1265,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect writer.close(); HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), - readBlocksLazily, true); + bufferSize, readBlocksLazily, true); assertTrue("Third block should be available", reader.hasPrev()); reader.moveToPrev(); diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 357023b4bded0..f96060b214554 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -76,9 +76,14 @@ public class HoodieRealtimeRecordReader implements RecordReader deltaRecordMap; @@ -136,7 +141,7 @@ private void readAndCompactLog(JobConf jobConf) throws IOException { .valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) * jobConf.getMemoryForMapTask()), Boolean.valueOf(jobConf .get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), - false); + false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit // but can return records for completed commits > the commit we are trying to read (if using // readCommit() API) diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 8e43aa25e056a..ea9016bb965f9 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -71,6 +71,7 @@ public class HoodieRealtimeRecordReaderTest { @Before public void setUp() { jobConf = new JobConf(); + jobConf.set(HoodieRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024)); hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf); } From fb093d0a739ee722faec7fa71524daf07cdc6076 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Thu, 19 Apr 2018 14:20:50 -0700 Subject: [PATCH 046/374] Update Gemfile.lock --- docs/Gemfile.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index e49bc0e79a9f5..7a62573949228 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -115,7 +115,7 @@ GEM minitest (5.11.1) multipart-post (2.0.0) net-dns (0.8.0) - nokogiri (1.8.1) + nokogiri (1.8.2) mini_portile2 (~> 2.3.0) octokit (4.8.0) sawyer (~> 0.8.0, >= 0.5.3) From c7ce582da7cf31a89abb9b412f8840b65424215f Mon Sep 17 00:00:00 2001 From: Sunil Ramaiah Date: Mon, 23 Apr 2018 15:23:42 -0700 Subject: [PATCH 047/374] Fix for updating duplicate records in same/different files in same parition --- .../hoodie/index/bloom/HoodieBloomIndex.java | 7 + .../com/uber/hoodie/io/HoodieMergeHandle.java | 14 +- ...TestHoodieMergeHandleDuplicateRecords.java | 276 ++++++++++++++++++ 3 files changed, 294 insertions(+), 3 deletions(-) create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 2b756c591840a..b85f0fed979f9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -394,6 +394,13 @@ private JavaRDD> tagLocationBacktoRecords( if (v1._2().isPresent()) { String filename = v1._2().get(); if (filename != null && !filename.isEmpty()) { + // When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD will have 2 + // entries with the same exact in memory copy of the HoodieRecord and the 2 separate filenames that the + // record is found in. This will result in setting currentLocation 2 times and it will fail the second time. + // This check will create a new in memory copy of the hoodie record. + if (record.getCurrentLocation() != null) { + record = new HoodieRecord(record.getKey(), record.getData()); + } record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename))); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index f5592c3811ad6..1d2133d518fb1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -35,9 +35,11 @@ import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.util.HashSet; import java.util.Iterator; import java.util.Map; import java.util.Optional; +import java.util.Set; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -52,6 +54,7 @@ public class HoodieMergeHandle extends HoodieIOHa private WriteStatus writeStatus; private Map> keyToNewRecords; + private Set writtenRecordKeys; private HoodieStorageWriter storageWriter; private TableFileSystemView.ReadOptimizedView fileSystemView; private Path newFilePath; @@ -81,6 +84,8 @@ public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTabl * Extract old file path, initialize StorageWriter and WriteStatus */ private void init(String fileId, String partitionPath) { + this.writtenRecordKeys = new HashSet<>(); + WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieWriteStat()); this.writeStatus = writeStatus; @@ -205,7 +210,7 @@ public void write(GenericRecord oldRecord) { */ copyOldRecord = false; } - keyToNewRecords.remove(key); + writtenRecordKeys.add(key); } catch (Exception e) { throw new HoodieUpsertException( "Failed to combine/merge new record with old value in storage, for new record {" @@ -239,10 +244,13 @@ public void close() { Iterator pendingRecordsItr = keyToNewRecords.keySet().iterator(); while (pendingRecordsItr.hasNext()) { String key = pendingRecordsItr.next(); - HoodieRecord hoodieRecord = keyToNewRecords.get(key); - writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema)); + if (!writtenRecordKeys.contains(key)) { + HoodieRecord hoodieRecord = keyToNewRecords.get(key); + writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema)); + } } keyToNewRecords.clear(); + writtenRecordKeys.clear(); if (storageWriter != null) { storageWriter.close(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java new file mode 100644 index 0000000000000..d199be72e2c9b --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java @@ -0,0 +1,276 @@ +/* + * 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.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.HoodieClientTestUtils; +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieStorageConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.HoodieIndex; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +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.Before; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +@SuppressWarnings("unchecked") +public class TestHoodieMergeHandleDuplicateRecords { + + protected transient JavaSparkContext jsc = null; + protected transient SQLContext sqlContext; + protected transient FileSystem fs; + protected String basePath = null; + protected transient HoodieTestDataGenerator dataGen = null; + + @Before + public void init() throws IOException { + // Initialize a local spark env + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandleDuplicateRecords")); + + //SQLContext stuff + sqlContext = new SQLContext(jsc); + + // Create a temp folder as the base path + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + dataGen = new HoodieTestDataGenerator(); + } + + @After + public void clean() { + if (basePath != null) { + new File(basePath).delete(); + } + if (jsc != null) { + jsc.stop(); + } + } + + @Test + public void testUpsertsForMultipleRecordsInSameFile() throws Exception { + + // Create records in a single partition + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; + dataGen = new HoodieTestDataGenerator(new String[]{partitionPath}); + + // Build a write config with bulkinsertparallelism set + HoodieWriteConfig cfg = getConfigBuilder().build(); + + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + + /** + * Write 1 (only inserts) + */ + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 4); + + HoodieRecord record1 = records.get(0); + HoodieRecord record2 = records.get(1); + + for (int i = 0; i < 20; i++) { + HoodieRecord dup = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime); + records.add(dup); + } + + for (int i = 0; i < 20; i++) { + HoodieRecord dup = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime); + records.add(dup); + } + + JavaRDD writeRecords = jsc.parallelize(records, 1); + + List statuses = client.bulkInsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + + // verify that there is a commit + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals("Must contain 44 records", + records.size(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); + + /** + * Write 2 (insert) + */ + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + + // Do 1 more bulk insert with the same dup record1 + List newRecords = new ArrayList<>(); + HoodieRecord sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime); + newRecords.add(sameAsRecord1); + + writeRecords = jsc.parallelize(newRecords, 1); + + statuses = client.bulkInsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + + // verify that there are 2 commits + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + + assertEquals("Expecting two commits.", 2, timeline.findInstantsAfter("000", Integer.MAX_VALUE) + .countInstants()); + assertEquals("Latest commit should be 002", newCommitTime, timeline.lastInstant().get().getTimestamp()); + Dataset dataSet = getRecords(); + assertEquals("Must contain 45 records", 45, dataSet.count()); + + /** + * Write 3 (insert) + */ + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + + newRecords = dataGen.generateInserts(newCommitTime, 2); + + writeRecords = jsc.parallelize(newRecords, 1); + + statuses = client.bulkInsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + + // verify that there are not 3 commits + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + + assertEquals("Expecting three commits.", 3, timeline.findInstantsAfter("000", Integer.MAX_VALUE) + .countInstants()); + assertEquals("Latest commit should be 003", newCommitTime, timeline.lastInstant().get().getTimestamp()); + dataSet = getRecords(); + assertEquals("Must contain 47 records", 47, dataSet.count()); + + /** + * Write 4 (updates) + */ + newCommitTime = "004"; + client.startCommitWithTime(newCommitTime); + + List updateRecords = new ArrayList<>(); + + // This exists in 001 and 002 and should be updated in both + sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime); + updateRecords.add(sameAsRecord1); + + // This exists in 001 and should be updated + HoodieRecord sameAsRecord2 = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime); + updateRecords.add(sameAsRecord2); + + JavaRDD updateRecordsRDD = jsc.parallelize(updateRecords, 1); + + statuses = client.upsert(updateRecordsRDD, newCommitTime).collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + // verify there are now 4 commits + timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + assertEquals("Expecting four commits.", 4, timeline.findInstantsAfter("000", Integer.MAX_VALUE) + .countInstants()); + assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); + + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + + // Check the entire dataset has 47 records still + dataSet = getRecords(); + assertEquals("Must contain 47 records", 47, dataSet.count()); + + Row[] rows = (Row[]) dataSet.collect(); + int record1Count = 0; + int record2Count = 0; + for (Row row : rows) { + if (row.getAs("_hoodie_record_key").equals(record1.getKey().getRecordKey())) { + record1Count++; + + // assert each duplicate record is updated + assertEquals(row.getAs("rider"), "rider-004"); + assertEquals(row.getAs("driver"), "driver-004"); + } else if (row.getAs("_hoodie_record_key").equals(record2.getKey().getRecordKey())) { + record2Count++; + + // assert each duplicate record is updated + assertEquals(row.getAs("rider"), "rider-004"); + assertEquals(row.getAs("driver"), "driver-004"); + } + } + + assertEquals(22, record1Count); + assertEquals(21, record2Count); + } + + private Dataset getRecords() { + // Check the entire dataset has 8 records still + String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; + for (int i = 0; i < fullPartitionPaths.length; i++) { + fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); + } + + Dataset dataSet = HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths); + + return dataSet; + } + + /** + * Assert no failures in writing hoodie files + * + * @param statuses List of Write Status + */ + void assertNoWriteErrors(List statuses) { + // Verify there are no errors + for (WriteStatus status : statuses) { + assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors()); + } + } + + HoodieWriteConfig.Builder getConfigBuilder() { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withBulkInsertParallelism(2); + } +} From 3bc76b4e5f08ad6a469faa7376908367e26a9ec2 Mon Sep 17 00:00:00 2001 From: Sunil Ramaiah Date: Wed, 25 Apr 2018 12:40:24 -0700 Subject: [PATCH 048/374] Added more comments and removed the extra new lines --- ...TestHoodieMergeHandleDuplicateRecords.java | 55 +++++++++---------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java index d199be72e2c9b..bbb826c101304 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; @@ -87,47 +88,40 @@ public void clean() { @Test public void testUpsertsForMultipleRecordsInSameFile() throws Exception { - // Create records in a single partition String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; dataGen = new HoodieTestDataGenerator(new String[]{partitionPath}); // Build a write config with bulkinsertparallelism set HoodieWriteConfig cfg = getConfigBuilder().build(); - HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); /** * Write 1 (only inserts) + * This will do a bulk insert of 44 records of which there are 2 records repeated 21 times each. + * id1 (21 records), id2 (21 records), id3, id4 */ String newCommitTime = "001"; client.startCommitWithTime(newCommitTime); - List records = dataGen.generateInserts(newCommitTime, 4); - HoodieRecord record1 = records.get(0); HoodieRecord record2 = records.get(1); - for (int i = 0; i < 20; i++) { HoodieRecord dup = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime); records.add(dup); } - for (int i = 0; i < 20; i++) { HoodieRecord dup = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime); records.add(dup); } - JavaRDD writeRecords = jsc.parallelize(records, 1); - List statuses = client.bulkInsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); // verify that there is a commit HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); assertEquals("Must contain 44 records", @@ -136,6 +130,10 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { /** * Write 2 (insert) + * This will do a bulk insert of 1 record with the same row_key as record1 in the previous insert - id1. + * At this point, we will have 2 files with the row_keys as shown here - + * File 1 - id1 (21 records), id2 (21 records), id3, id4 + * File 2 - id1 */ newCommitTime = "002"; client.startCommitWithTime(newCommitTime); @@ -144,16 +142,13 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { List newRecords = new ArrayList<>(); HoodieRecord sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime); newRecords.add(sameAsRecord1); - writeRecords = jsc.parallelize(newRecords, 1); - statuses = client.bulkInsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); // verify that there are 2 commits metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", 2, timeline.findInstantsAfter("000", Integer.MAX_VALUE) .countInstants()); assertEquals("Latest commit should be 002", newCommitTime, timeline.lastInstant().get().getTimestamp()); @@ -162,21 +157,22 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { /** * Write 3 (insert) + * This will bulk insert 2 new completely new records. + * At this point, we will have 2 files with the row_keys as shown here - + * File 1 - id1 (21 records), id2 (21 records), id3, id4 + * File 2 - id1 + * File 3 - id5, id6 */ newCommitTime = "003"; client.startCommitWithTime(newCommitTime); - newRecords = dataGen.generateInserts(newCommitTime, 2); - writeRecords = jsc.parallelize(newRecords, 1); - statuses = client.bulkInsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - // verify that there are not 3 commits + // verify that there are now 3 commits metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting three commits.", 3, timeline.findInstantsAfter("000", Integer.MAX_VALUE) .countInstants()); assertEquals("Latest commit should be 003", newCommitTime, timeline.lastInstant().get().getTimestamp()); @@ -185,10 +181,14 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { /** * Write 4 (updates) + * This will generate 2 upsert records with id1 and id2. The rider and driver names in the update records + * will be rider-004 and driver-004. + * After the upsert is complete, all the records with id1 in File 1 and File 2 must be updated, all the records + * with id2 in File 2 must also be updated. + * Also, none of the other records in File 1, File 2 and File 3 must be updated. */ newCommitTime = "004"; client.startCommitWithTime(newCommitTime); - List updateRecords = new ArrayList<>(); // This exists in 001 and 002 and should be updated in both @@ -198,10 +198,9 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { // This exists in 001 and should be updated HoodieRecord sameAsRecord2 = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime); updateRecords.add(sameAsRecord2); - JavaRDD updateRecordsRDD = jsc.parallelize(updateRecords, 1); - statuses = client.upsert(updateRecordsRDD, newCommitTime).collect(); + // Verify there are no errors assertNoWriteErrors(statuses); @@ -211,32 +210,34 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { .countInstants()); assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - // Check the entire dataset has 47 records still dataSet = getRecords(); assertEquals("Must contain 47 records", 47, dataSet.count()); - Row[] rows = (Row[]) dataSet.collect(); int record1Count = 0; int record2Count = 0; for (Row row : rows) { if (row.getAs("_hoodie_record_key").equals(record1.getKey().getRecordKey())) { record1Count++; - // assert each duplicate record is updated assertEquals(row.getAs("rider"), "rider-004"); assertEquals(row.getAs("driver"), "driver-004"); } else if (row.getAs("_hoodie_record_key").equals(record2.getKey().getRecordKey())) { record2Count++; - // assert each duplicate record is updated assertEquals(row.getAs("rider"), "rider-004"); assertEquals(row.getAs("driver"), "driver-004"); + } else { + assertNotEquals(row.getAs("rider"), "rider-004"); + assertNotEquals(row.getAs("driver"), "rider-004"); } } - + // Assert that id1 record count which has been updated to rider-004 and driver-004 is 22, which is the total + // number of records with row_key id1 assertEquals(22, record1Count); + + // Assert that id2 record count which has been updated to rider-004 and driver-004 is 21, which is the total + // number of records with row_key id2 assertEquals(21, record2Count); } @@ -246,9 +247,7 @@ private Dataset getRecords() { for (int i = 0; i < fullPartitionPaths.length; i++) { fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } - Dataset dataSet = HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths); - return dataSet; } From ff2e7ae1e518e17c821bac1332f5969b61fb8089 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 24 Apr 2018 10:56:05 -0700 Subject: [PATCH 049/374] Add Support for ordering and limiting results in CLI show commands --- docs/admin_guide.md | 10 +- .../uber/hoodie/cli/HoodiePrintHelper.java | 78 ++++++++ .../main/java/com/uber/hoodie/cli/Table.java | 175 ++++++++++++++++++ .../java/com/uber/hoodie/cli/TableHeader.java | 69 +++++++ .../cli/commands/ArchivedCommitsCommand.java | 48 ++--- .../hoodie/cli/commands/CleansCommand.java | 56 ++++-- .../hoodie/cli/commands/CommitsCommand.java | 115 +++++++++--- .../cli/commands/HoodieLogFileCommand.java | 30 ++- .../hoodie/cli/commands/StatsCommand.java | 90 ++++++--- 9 files changed, 563 insertions(+), 108 deletions(-) create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/TableHeader.java diff --git a/docs/admin_guide.md b/docs/admin_guide.md index 950a0b7f5aa54..42b2185d63ca2 100644 --- a/docs/admin_guide.md +++ b/docs/admin_guide.md @@ -79,9 +79,9 @@ To view some basic information about the last 10 commits, ``` -hoodie:trips->commits show +hoodie:trips->commits show --sortBy "Total Bytes Written" --desc true --limit 10 ________________________________________________________________________________________________________________________________________________________________________ - | CommitTime | Total Written (B)| Total Files Added| Total Files Updated| Total Partitions Written| Total Records Written| Total Update Records Written| Total Errors| + | CommitTime | Total Bytes Written| Total Files Added| Total Files Updated| Total Partitions Written| Total Records Written| Total Update Records Written| Total Errors| |=======================================================================================================================================================================| .... .... @@ -105,7 +105,7 @@ To understand how the writes spread across specific partiions, ``` -hoodie:trips->commit showpartitions --commit 20161005165855 +hoodie:trips->commit showpartitions --commit 20161005165855 --sortBy "Total Bytes Written" --desc true --limit 10 __________________________________________________________________________________________________________________________________________ | Partition Path| Total Files Added| Total Files Updated| Total Records Inserted| Total Records Updated| Total Bytes Written| Total Errors| |=========================================================================================================================================| @@ -117,7 +117,7 @@ If you need file level granularity , we can do the following ``` -hoodie:trips->commit showfiles --commit 20161005165855 +hoodie:trips->commit showfiles --commit 20161005165855 --sortBy "Partition Path" ________________________________________________________________________________________________________________________________________________________ | Partition Path| File ID | Previous Commit| Total Records Updated| Total Records Written| Total Bytes Written| Total Errors| |=======================================================================================================================================================| @@ -131,7 +131,7 @@ Since Hoodie directly manages file sizes for HDFS dataset, it might be good to g ``` -hoodie:trips->stats filesizes --partitionPath 2016/09/01 +hoodie:trips->stats filesizes --partitionPath 2016/09/01 --sortBy "95th" --desc true --limit 10 ________________________________________________________________________________________________ | CommitTime | Min | 10th | 50th | avg | 95th | Max | NumFiles| StdDev | |===============================================================================================| diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java index b6625718b8c2b..e7b7c9c655fad 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java @@ -20,11 +20,89 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.nio.charset.Charset; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +/** + * Helper class to render table for hoodie-cli + */ public class HoodiePrintHelper { + /** + * Print header and raw rows + * @param header Header + * @param rows Raw Rows + * @return output + */ public static String print(String[] header, String[][] rows) { TextTable textTable = new TextTable(header, rows); + return printTextTable(textTable); + } + + /** + * Serialize Table to printable string + * @param rowHeader Row Header + * @param fieldNameToConverterMap Field Specific Converters + * @param sortByField Sorting field + * @param isDescending Order + * @param limit Limit + * @param headerOnly Headers only + * @param rows List of rows + * @return Serialized form for printing + */ + public static String print(TableHeader rowHeader, + Map> fieldNameToConverterMap, + String sortByField, boolean isDescending, Integer limit, boolean headerOnly, + List rows) { + + if (headerOnly) { + return HoodiePrintHelper.print(rowHeader); + } + + Table table = new Table(rowHeader, fieldNameToConverterMap, + Optional.ofNullable(sortByField.isEmpty() ? null : sortByField), + Optional.ofNullable(isDescending), + Optional.ofNullable(limit <= 0 ? null : limit)).addAllRows(rows).flip(); + + return HoodiePrintHelper.print(table); + } + + /** + * Render rows in Table + * @param buffer Table + * @return output + */ + private static String print(Table buffer) { + String[] header = new String[buffer.getFieldNames().size()]; + buffer.getFieldNames().toArray(header); + + String[][] rows = buffer.getRenderRows().stream() + .map(l -> l.stream().toArray(String[]::new)) + .toArray(String[][]::new); + TextTable textTable = new TextTable(header, rows); + return printTextTable(textTable); + } + + /** + * Render only header of the table + * @param header Table Header + * @return output + */ + private static String print(TableHeader header) { + String[] head = new String[header.getFieldNames().size()]; + header.getFieldNames().toArray(head); + TextTable textTable = new TextTable(head, new String[][]{}); + return printTextTable(textTable); + } + + /** + * Print Text table + * @param textTable Text table to be printed + * @return + */ + private static String printTextTable(TextTable textTable) { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintStream ps = new PrintStream(baos); textTable.printTable(ps, 4); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java new file mode 100644 index 0000000000000..d02b7810a4c01 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java @@ -0,0 +1,175 @@ +/* + * 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.cli; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Table to be rendered. This class takes care of ordering + * rows and limiting before renderer renders it. + */ +public class Table implements Iterable> { + + // Header for this table + private final TableHeader rowHeader; + // User-specified conversions before rendering + private final Map> fieldNameToConverterMap; + // Optional attribute to track sorting field + private final Optional orderingFieldNameOptional; + // Whether sorting has to be in descending order (by default : optional) + private final Optional isDescendingOptional; + // Limit the number of entries rendered + private final Optional limitOptional; + // Raw list of rows + private final List> rawRows; + // Flag to determine if all the rows have been added + private boolean finishedAdding = false; + // Rows ready for Rendering + private List> renderRows; + + public Table(TableHeader rowHeader, + Map> fieldNameToConverterMap, + Optional orderingFieldNameOptional, + Optional isDescendingOptional, + Optional limitOptional) { + this.rowHeader = rowHeader; + this.fieldNameToConverterMap = fieldNameToConverterMap; + this.orderingFieldNameOptional = orderingFieldNameOptional; + this.isDescendingOptional = isDescendingOptional; + this.limitOptional = limitOptional; + this.rawRows = new ArrayList<>(); + } + + /** + * Main API to add row to the table + * @param row Row + */ + public Table add(List row) { + if (finishedAdding) { + throw new IllegalStateException("Container already marked done for adding. No more entries can be added."); + } + + if (rowHeader.getFieldNames().size() != row.size()) { + throw new IllegalArgumentException("Incorrect number of fields in row. Expected: " + + rowHeader.getFieldNames().size() + ", Got: " + row.size() + ", Row: " + row); + } + + this.rawRows.add(new ArrayList<>(row)); + return this; + } + + /** + * Add all rows + * @param rows Rows to be aded + * @return + */ + public Table addAll(List> rows) { + rows.forEach(r -> add(r)); + return this; + } + + /** + * Add all rows + * @param rows Rows to be added + * @return + */ + public Table addAllRows(List rows) { + rows.forEach(r -> add(Arrays.asList(r))); + return this; + } + + /** + * API to let the table know writing is over and reading is going to start + */ + public Table flip() { + this.finishedAdding = true; + sortAndLimit(); + return this; + } + + /** + * Sorting of rows by a specified field + * @return + */ + private List> orderRows() { + return orderingFieldNameOptional.map(orderingColumnName -> { + return rawRows.stream().sorted(new Comparator>() { + @Override + public int compare(List row1, List row2) { + Comparable fieldForRow1 = row1.get(rowHeader.indexOf(orderingColumnName)); + Comparable fieldForRow2 = row2.get(rowHeader.indexOf(orderingColumnName)); + int cmpRawResult = fieldForRow1.compareTo(fieldForRow2); + return isDescendingOptional.map(isDescending -> { + return isDescending ? -1 * cmpRawResult : cmpRawResult; + }).orElse(cmpRawResult); + } + }).collect(Collectors.toList()); + }).orElse(rawRows); + } + + /** + * Prepares for rendering. Rows are sorted and limited + */ + private void sortAndLimit() { + this.renderRows = new ArrayList<>(); + final int limit = this.limitOptional.orElse(rawRows.size()); + final List> orderedRows = orderRows(); + renderRows = orderedRows.stream().limit(limit).map(row -> { + return IntStream.range(0, rowHeader.getNumFields()).mapToObj(idx -> { + String fieldName = rowHeader.get(idx); + if (fieldNameToConverterMap.containsKey(fieldName)) { + return fieldNameToConverterMap.get(fieldName).apply(row.get(idx)); + } + return row.get(idx).toString(); + }).collect(Collectors.toList()); + }).collect(Collectors.toList()); + } + + @Override + public Iterator> iterator() { + if (!finishedAdding) { + throw new IllegalStateException("Container must be flipped before reading the data"); + } + return renderRows.iterator(); + } + + @Override + public void forEach(Consumer> action) { + if (!finishedAdding) { + throw new IllegalStateException("Container must be flipped before reading the data"); + } + renderRows.forEach(action); + } + + public List getFieldNames() { + return rowHeader.getFieldNames(); + } + + public List> getRenderRows() { + return renderRows; + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/TableHeader.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/TableHeader.java new file mode 100644 index 0000000000000..1f02e6f003d8b --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/TableHeader.java @@ -0,0 +1,69 @@ +/* + * 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.cli; + +import java.util.ArrayList; +import java.util.List; + +/** + * Header for the table to be rendered + */ +public class TableHeader { + + // List of fields (columns) + private final List fieldNames = new ArrayList<>(); + + /** + * Add a field (column) to table + * + * @param fieldName field Name + */ + public TableHeader addTableHeaderField(String fieldName) { + fieldNames.add(fieldName); + return this; + } + + /** + * Get all field names + */ + public List getFieldNames() { + return fieldNames; + } + + /** + * Index of the field in the table + * + * @param fieldName Field Name + */ + public int indexOf(String fieldName) { + return fieldNames.indexOf(fieldName); + } + + /** + * Lookup field by offset + */ + public String get(int index) { + return fieldNames.get(index); + } + + /** + * Get number of fields in the table + */ + public int getNumFields() { + return fieldNames.size(); + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 99c79863b8f84..893e03fcf76d8 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -19,6 +19,7 @@ import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry; import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; @@ -26,6 +27,7 @@ import com.uber.hoodie.common.util.FSUtils; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; @@ -47,16 +49,19 @@ public boolean isShowArchivedCommitAvailable() { } @CliCommand(value = "show archived commits", help = "Read commits from archived files and show details") - public String showCommits(@CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) + public String showCommits( + @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "10") 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(); FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf) .globStatus(new Path(basePath + "/.hoodie/.commits_.archive*")); - List allCommits = new ArrayList<>(); - int commits = 0; + List allCommits = new ArrayList<>(); for (FileStatus fs : fsStatuses) { //read the archived file HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf), @@ -68,62 +73,59 @@ public String showCommits(@CliOption(key = { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); List records = blk.getRecords(); readRecords.addAll(records); - if (commits == limit) { - break; - } - commits++; } - List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r)) + List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r)) .collect(Collectors.toList()); allCommits.addAll(readCommits); - if (commits == limit) { - break; - } } - return HoodiePrintHelper.print(new String[] {"CommitTime", "CommitType", "CommitDetails"}, - allCommits.toArray(new String[allCommits.size()][])); + + TableHeader header = new TableHeader().addTableHeaderField("CommitTime") + .addTableHeaderField("CommitType") + .addTableHeaderField("CommitDetails"); + + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allCommits); } - private String[] readCommit(GenericRecord record) { - List commitDetails = new ArrayList<>(); + private Comparable[] readCommit(GenericRecord record) { + List commitDetails = new ArrayList<>(); try { switch (record.get("actionType").toString()) { case HoodieTimeline.CLEAN_ACTION: { - commitDetails.add(record.get("commitTime").toString()); + commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); commitDetails.add(record.get("hoodieCleanMetadata").toString()); break; } case HoodieTimeline.COMMIT_ACTION: { - commitDetails.add(record.get("commitTime").toString()); + commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); commitDetails.add(record.get("hoodieCommitMetadata").toString()); break; } case HoodieTimeline.DELTA_COMMIT_ACTION: { - commitDetails.add(record.get("commitTime").toString()); + commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); commitDetails.add(record.get("hoodieCommitMetadata").toString()); break; } case HoodieTimeline.ROLLBACK_ACTION: { - commitDetails.add(record.get("commitTime").toString()); + commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); commitDetails.add(record.get("hoodieRollbackMetadata").toString()); break; } case HoodieTimeline.SAVEPOINT_ACTION: { - commitDetails.add(record.get("commitTime").toString()); + commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); commitDetails.add(record.get("hoodieSavePointMetadata").toString()); break; } default: - return commitDetails.toArray(new String[commitDetails.size()]); + return commitDetails.toArray(new Comparable[commitDetails.size()]); } } catch (Exception e) { e.printStackTrace(); } - return commitDetails.toArray(new String[commitDetails.size()]); + return commitDetails.toArray(new Comparable[commitDetails.size()]); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java index aa1792ecb36d6..bfc08658fbcea 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java @@ -20,6 +20,7 @@ import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata; import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; @@ -28,6 +29,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -56,22 +58,33 @@ public boolean isCommitShowAvailable() { } @CliCommand(value = "cleans show", help = "Show the cleans") - public String showCleans() throws IOException { + public String showCleans( + @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 { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants(); List cleans = timeline.getInstants().collect(Collectors.toList()); - String[][] rows = new String[cleans.size()][]; + List rows = new ArrayList<>(); Collections.reverse(cleans); for (int i = 0; i < cleans.size(); i++) { HoodieInstant clean = cleans.get(i); HoodieCleanMetadata cleanMetadata = AvroUtils .deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); - rows[i] = new String[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), - String.valueOf(cleanMetadata.getTotalFilesDeleted()), String.valueOf(cleanMetadata.getTimeTakenInMillis())}; + rows.add(new Comparable[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), + cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()}); } - return HoodiePrintHelper - .print(new String[] {"CleanTime", "EarliestCommandRetained", "Total Files Deleted", "Total Time Taken"}, - rows); + + TableHeader header = new TableHeader() + .addTableHeaderField("CleanTime") + .addTableHeaderField("EarliestCommandRetained") + .addTableHeaderField("Total Files Deleted") + .addTableHeaderField("Total Time Taken"); + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); } @CliCommand(value = "cleans refresh", help = "Refresh the commits") @@ -82,8 +95,15 @@ public String refreshCleans() throws IOException { } @CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean") - public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to show") final String commitTime) + public String showCleanPartitions( + @CliOption(key = {"clean"}, help = "clean to show") final String commitTime, + @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 Exception { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants(); HoodieInstant cleanInstant = new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime); @@ -91,19 +111,25 @@ public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to s if (!timeline.containsInstant(cleanInstant)) { return "Clean " + commitTime + " not found in metadata " + timeline; } + HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata( timeline.getInstantDetails(cleanInstant).get()); - List rows = new ArrayList<>(); + List rows = new ArrayList<>(); for (Map.Entry entry : cleanMetadata.getPartitionMetadata().entrySet()) { String path = entry.getKey(); HoodieCleanPartitionMetadata stats = entry.getValue(); String policy = stats.getPolicy(); - String totalSuccessDeletedFiles = String.valueOf(stats.getSuccessDeleteFiles().size()); - String totalFailedDeletedFiles = String.valueOf(stats.getFailedDeleteFiles().size()); - rows.add(new String[] {path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles}); + Integer totalSuccessDeletedFiles = stats.getSuccessDeleteFiles().size(); + Integer totalFailedDeletedFiles = stats.getFailedDeleteFiles().size(); + rows.add(new Comparable[]{path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles}); } - return HoodiePrintHelper.print( - new String[] {"Partition Path", "Cleaning policy", "Total Files Successfully Deleted", - "Total Failed Deletions"}, rows.toArray(new String[rows.size()][])); + + TableHeader header = new TableHeader() + .addTableHeaderField("Partition Path") + .addTableHeaderField("Cleaning policy") + .addTableHeaderField("Total Files Successfully Deleted") + .addTableHeaderField("Total Failed Deletions"); + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); + } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index 0f5f87b0966fb..fe6a7736947aa 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -18,6 +18,7 @@ import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.cli.utils.InputStreamConsumer; import com.uber.hoodie.cli.utils.SparkUtil; import com.uber.hoodie.common.model.HoodieCommitMetadata; @@ -30,8 +31,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.spark.launcher.SparkLauncher; import org.springframework.shell.core.CommandMarker; @@ -65,28 +68,46 @@ public boolean isCommitShowAvailable() { @CliCommand(value = "commits show", help = "Show the commits") public String showCommits(@CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) + "limit"}, mandatory = false, 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 { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); List commits = timeline.getInstants().collect(Collectors.toList()); - String[][] rows = new String[commits.size()][]; + List rows = new ArrayList<>(); Collections.reverse(commits); for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get()); - rows[i] = new String[] {commit.getTimestamp(), - NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()), - String.valueOf(commitMetadata.fetchTotalFilesInsert()), - String.valueOf(commitMetadata.fetchTotalFilesUpdated()), - String.valueOf(commitMetadata.fetchTotalPartitionsWritten()), - String.valueOf(commitMetadata.fetchTotalRecordsWritten()), - String.valueOf(commitMetadata.fetchTotalUpdateRecordsWritten()), - String.valueOf(commitMetadata.fetchTotalWriteErrors())}; + rows.add(new Comparable[]{commit.getTimestamp(), + commitMetadata.fetchTotalBytesWritten(), + commitMetadata.fetchTotalFilesInsert(), + commitMetadata.fetchTotalFilesUpdated(), + commitMetadata.fetchTotalPartitionsWritten(), + commitMetadata.fetchTotalRecordsWritten(), + commitMetadata.fetchTotalUpdateRecordsWritten(), + commitMetadata.fetchTotalWriteErrors()}); } - return HoodiePrintHelper.print( - new String[] {"CommitTime", "Total Written (B)", "Total Files Added", "Total Files Updated", - "Total Partitions Written", "Total Records Written", "Total Update Records Written", "Total Errors"}, rows); + + Map> fieldNameToConverterMap = new HashMap<>(); + fieldNameToConverterMap.put("Total Bytes Written", entry -> { + return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString()))); + }); + + TableHeader header = new TableHeader() + .addTableHeaderField("CommitTime") + .addTableHeaderField("Total Bytes Written") + .addTableHeaderField("Total Files Added") + .addTableHeaderField("Total Files Updated") + .addTableHeaderField("Total Partitions Written") + .addTableHeaderField("Total Records Written") + .addTableHeaderField("Total Update Records Written") + .addTableHeaderField("Total Errors"); + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); } @CliCommand(value = "commits refresh", help = "Refresh the commits") @@ -123,8 +144,15 @@ public String rollbackCommit(@CliOption(key = {"commit"}, help = "Commit to roll } @CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit") - public String showCommitPartitions(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) + public String showCommitPartitions( + @CliOption(key = {"commit"}, help = "Commit to show") final String commitTime, + @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 Exception { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); @@ -133,7 +161,7 @@ public String showCommitPartitions(@CliOption(key = {"commit"}, help = "Commit t return "Commit " + commitTime + " not found in Commits " + timeline; } HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); - List rows = new ArrayList(); + List rows = new ArrayList<>(); for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); List stats = entry.getValue(); @@ -154,19 +182,38 @@ public String showCommitPartitions(@CliOption(key = {"commit"}, help = "Commit t totalBytesWritten += stat.getTotalWriteBytes(); totalWriteErrors += stat.getTotalWriteErrors(); } - rows.add(new String[] {path, String.valueOf(totalFilesAdded), String.valueOf(totalFilesUpdated), - String.valueOf(totalRecordsInserted), String.valueOf(totalRecordsUpdated), - NumericUtils.humanReadableByteCount(totalBytesWritten), String.valueOf(totalWriteErrors)}); - + rows.add(new Comparable[]{path, totalFilesAdded, totalFilesUpdated, + totalRecordsInserted, totalRecordsUpdated, + totalBytesWritten, totalWriteErrors}); } - return HoodiePrintHelper.print( - new String[] {"Partition Path", "Total Files Added", "Total Files Updated", "Total Records Inserted", - "Total Records Updated", "Total Bytes Written", "Total Errors"}, rows.toArray(new String[rows.size()][])); + + Map> fieldNameToConverterMap = new HashMap<>(); + fieldNameToConverterMap.put("Total Bytes Written", entry -> { + return NumericUtils.humanReadableByteCount((Long.valueOf(entry.toString()))); + }); + + TableHeader header = new TableHeader() + .addTableHeaderField("Partition Path") + .addTableHeaderField("Total Files Added") + .addTableHeaderField("Total Files Updated") + .addTableHeaderField("Total Records Inserted") + .addTableHeaderField("Total Records Updated") + .addTableHeaderField("Total Bytes Written") + .addTableHeaderField("Total Errors"); + + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); } @CliCommand(value = "commit showfiles", help = "Show file level details of a commit") - public String showCommitFiles(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) + public String showCommitFiles( + @CliOption(key = {"commit"}, help = "Commit to show") final String commitTime, + @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 Exception { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); @@ -175,19 +222,27 @@ public String showCommitFiles(@CliOption(key = {"commit"}, help = "Commit to sho return "Commit " + commitTime + " not found in Commits " + timeline; } HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); - List rows = new ArrayList(); + List rows = new ArrayList<>(); for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); List stats = entry.getValue(); for (HoodieWriteStat stat : stats) { - rows.add(new String[] {path, stat.getFileId(), stat.getPrevCommit(), String.valueOf(stat.getNumUpdateWrites()), - String.valueOf(stat.getNumWrites()), String.valueOf(stat.getTotalWriteBytes()), - String.valueOf(stat.getTotalWriteErrors())}); + rows.add(new Comparable[]{path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(), + stat.getNumWrites(), stat.getTotalWriteBytes(), + stat.getTotalWriteErrors()}); } } - return HoodiePrintHelper.print( - new String[] {"Partition Path", "File ID", "Previous Commit", "Total Records Updated", "Total Records Written", - "Total Bytes Written", "Total Errors"}, rows.toArray(new String[rows.size()][])); + + TableHeader header = new TableHeader() + .addTableHeaderField("Partition Path") + .addTableHeaderField("File ID") + .addTableHeaderField("Previous Commit") + .addTableHeaderField("Total Records Updated") + .addTableHeaderField("Total Records Written") + .addTableHeaderField("Total Bytes Written") + .addTableHeaderField("Total Errors"); + + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); } @CliAvailabilityIndicator({"commits compare"}) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 78bd834dbee2f..b22d45d72f026 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -36,6 +37,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -65,8 +67,12 @@ public boolean isShowArchivedLogFileAvailable() { @CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files") public String showLogFileCommits( @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") final - String logFilePathPattern) - throws IOException { + String logFilePathPattern, + @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 { FileSystem fs = HoodieCLI.tableMetadata.getFs(); List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) @@ -120,7 +126,7 @@ public String showLogFileCommits( } } } - String[][] rows = new String[totalEntries + 1][]; + List rows = new ArrayList<>(); int i = 0; ObjectMapper objectMapper = new ObjectMapper(); for (Map.Entry, Map>, Integer> tuple3 : entry.getValue()) { - String[] output = new String[5]; + Comparable[] output = new Comparable[5]; output[0] = instantTime; - output[1] = String.valueOf(tuple3._3()); + output[1] = tuple3._3(); output[2] = tuple3._1().toString(); output[3] = objectMapper.writeValueAsString(tuple3._2()._1()); output[4] = objectMapper.writeValueAsString(tuple3._2()._2()); - rows[i] = output; + rows.add(output); i++; } } - return HoodiePrintHelper - .print(new String[] {"InstantTime", "RecordCount", "BlockType", "HeaderMetadata", "FooterMetadata"}, - rows); + + TableHeader header = new TableHeader() + .addTableHeaderField("InstantTime") + .addTableHeaderField("RecordCount") + .addTableHeaderField("BlockType") + .addTableHeaderField("HeaderMetadata") + .addTableHeaderField("FooterMetadata"); + + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); } @CliCommand(value = "show logfile records", help = "Read records from log files") diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java index 2cdf826e8f408..eeb6bff5c21ad 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java @@ -21,6 +21,7 @@ import com.codahale.metrics.UniformReservoir; import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; @@ -29,7 +30,11 @@ import com.uber.hoodie.common.util.NumericUtils; import java.io.IOException; import java.text.DecimalFormat; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -52,14 +57,20 @@ public boolean isWriteAmpAvailable() { @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many " + "records were actually written") - public String writeAmplificationStats() throws IOException { + public String writeAmplificationStats( + @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 { + long totalRecordsUpserted = 0; long totalRecordsWritten = 0; HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants(); - String[][] rows = new String[new Long(timeline.countInstants()).intValue() + 1][]; + List rows = new ArrayList<>(); int i = 0; DecimalFormat df = new DecimalFormat("#.00"); for (HoodieInstant commitTime : timeline.getInstants().collect(Collectors.toList())) { @@ -68,8 +79,8 @@ public String writeAmplificationStats() throws IOException { if (commit.fetchTotalUpdateRecordsWritten() > 0) { waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten()); } - rows[i++] = new String[] {commitTime.getTimestamp(), String.valueOf(commit.fetchTotalUpdateRecordsWritten()), - String.valueOf(commit.fetchTotalRecordsWritten()), waf}; + rows.add(new Comparable[]{commitTime.getTimestamp(), commit.fetchTotalUpdateRecordsWritten(), + commit.fetchTotalRecordsWritten(), waf}); totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten(); totalRecordsWritten += commit.fetchTotalRecordsWritten(); } @@ -77,26 +88,33 @@ public String writeAmplificationStats() throws IOException { if (totalRecordsUpserted > 0) { waf = df.format((float) totalRecordsWritten / totalRecordsUpserted); } - rows[i] = new String[] {"Total", String.valueOf(totalRecordsUpserted), String.valueOf(totalRecordsWritten), waf}; - return HoodiePrintHelper - .print(new String[] {"CommitTime", "Total Upserted", "Total Written", "Write Amplifiation Factor"}, - rows); - + rows.add(new Comparable[]{"Total", totalRecordsUpserted, totalRecordsWritten, waf}); + + TableHeader header = new TableHeader() + .addTableHeaderField("CommitTime") + .addTableHeaderField("Total Upserted") + .addTableHeaderField("Total Written") + .addTableHeaderField("Write Amplifiation Factor"); + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); } - - private String[] printFileSizeHistogram(String commitTime, Snapshot s) { - return new String[] {commitTime, NumericUtils.humanReadableByteCount(s.getMin()), - NumericUtils.humanReadableByteCount(s.getValue(0.1)), NumericUtils.humanReadableByteCount(s.getMedian()), - NumericUtils.humanReadableByteCount(s.getMean()), NumericUtils.humanReadableByteCount(s.get95thPercentile()), - NumericUtils.humanReadableByteCount(s.getMax()), String.valueOf(s.size()), - NumericUtils.humanReadableByteCount(s.getStdDev())}; + private Comparable[] printFileSizeHistogram(String commitTime, Snapshot s) { + return new Comparable[]{commitTime, s.getMin(), + s.getValue(0.1), s.getMedian(), + s.getMean(), s.get95thPercentile(), + s.getMax(), s.size(), + s.getStdDev()}; } @CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files") - public String fileSizeStats(@CliOption(key = { - "partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") final - String globRegex) throws IOException { + public String fileSizeStats( + @CliOption(key = {"partitionPath"}, + help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") final String globRegex, + @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 { FileSystem fs = HoodieCLI.fs; String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex); @@ -115,17 +133,37 @@ public String fileSizeStats(@CliOption(key = { globalHistogram.update(sz); } - String[][] rows = new String[commitHistoMap.size() + 1][]; + List rows = new ArrayList<>(); int ind = 0; for (String commitTime : commitHistoMap.keySet()) { Snapshot s = commitHistoMap.get(commitTime).getSnapshot(); - rows[ind++] = printFileSizeHistogram(commitTime, s); + rows.add(printFileSizeHistogram(commitTime, s)); } Snapshot s = globalHistogram.getSnapshot(); - rows[ind++] = printFileSizeHistogram("ALL", s); - - return HoodiePrintHelper - .print(new String[] {"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", "StdDev"}, - rows); + rows.add(printFileSizeHistogram("ALL", s)); + + Function converterFunction = entry -> { + return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString()))); + }; + Map> fieldNameToConverterMap = new HashMap<>(); + fieldNameToConverterMap.put("Min", converterFunction); + fieldNameToConverterMap.put("10th", converterFunction); + fieldNameToConverterMap.put("50th", converterFunction); + fieldNameToConverterMap.put("avg", converterFunction); + fieldNameToConverterMap.put("95th", converterFunction); + fieldNameToConverterMap.put("Max", converterFunction); + fieldNameToConverterMap.put("StdDev", converterFunction); + + TableHeader header = new TableHeader() + .addTableHeaderField("CommitTime") + .addTableHeaderField("Min") + .addTableHeaderField("10th") + .addTableHeaderField("50th") + .addTableHeaderField("avg") + .addTableHeaderField("95th") + .addTableHeaderField("Max") + .addTableHeaderField("NumFiles") + .addTableHeaderField("StdDev"); + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); } } From 7d2a01846cc8670ed9db4902dfaa7cffa379628b Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 25 Mar 2018 11:12:41 -0700 Subject: [PATCH 050/374] Adding metrics for MOR and COW --- .../com/uber/hoodie/HoodieWriteClient.java | 31 +++- .../uber/hoodie/io/HoodieAppendHandle.java | 8 + .../uber/hoodie/io/HoodieCreateHandle.java | 4 + .../com/uber/hoodie/io/HoodieIOHandle.java | 3 + .../com/uber/hoodie/io/HoodieMergeHandle.java | 4 + .../compact/HoodieRealtimeTableCompactor.java | 48 ++++- .../strategy/BoundedIOCompactionStrategy.java | 34 +--- .../compact/strategy/CompactionStrategy.java | 53 ++++-- .../strategy/UnBoundedCompactionStrategy.java | 12 +- .../uber/hoodie/metrics/HoodieMetrics.java | 51 +++-- .../hoodie/table/TestMergeOnReadTable.java | 43 +++++ .../src/main/avro/HoodieCommitMetadata.avsc | 2 +- .../main/avro/HoodieCompactionMetadata.avsc | 2 +- .../common/model/HoodieCommitMetadata.java | 88 ++++++++- .../hoodie/common/model/HoodieWriteStat.java | 174 +++++++++++++++--- .../log/HoodieCompactedLogRecordScanner.java | 46 ++++- .../uber/hoodie/common/util/HoodieTimer.java | 70 +++++++ .../hoodie/common/model/HoodieTestUtils.java | 22 +++ .../model/TestHoodieCommitMetadata.java | 42 +++++ 19 files changed, 616 insertions(+), 121 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieTimer.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 93dd4c2924cf4..16aba54513ab9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -543,7 +543,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, long durationInMs = metrics.getDurationInMs(writeContext.stop()); metrics .updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), - durationInMs, metadata); + durationInMs, metadata, actionType); writeContext = null; } logger.info("Committed " + commitTime); @@ -926,6 +926,8 @@ public JavaRDD compact(String commitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction + writeContext = metrics.getCommitCtx(); JavaRDD statuses = table.compact(jsc, commitTime); // Trigger the insert and collect statuses statuses = statuses.persist(config.getWriteStatusStorageLevel()); @@ -960,9 +962,22 @@ private void forceCompact(String compactionCommitTime) throws IOException { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and + // then use getTableAndInitCtx + Timer.Context writeContext = metrics.getCommitCtx(); JavaRDD compactedStatuses = table.compact(jsc, compactionCommitTime); if (!compactedStatuses.isEmpty()) { - commitForceCompaction(compactedStatuses, metaClient, compactionCommitTime); + HoodieCommitMetadata metadata = commitForceCompaction(compactedStatuses, metaClient, compactionCommitTime); + long durationInMs = metrics.getDurationInMs(writeContext.stop()); + try { + metrics + .updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.COMMIT_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException( + "Commit time is not of valid format.Failed to commit " + config.getBasePath() + + " at time " + compactionCommitTime, e); + } logger.info("Compacted successfully on commit " + compactionCommitTime); } else { logger.info("Compaction did not run for commit " + compactionCommitTime); @@ -979,7 +994,7 @@ private String forceCompact() throws IOException { return compactionCommitTime; } - private void commitForceCompaction(JavaRDD writeStatuses, + private HoodieCommitMetadata commitForceCompaction(JavaRDD writeStatuses, HoodieTableMetaClient metaClient, String compactionCommitTime) { List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) .collect(); @@ -1002,6 +1017,7 @@ private void commitForceCompaction(JavaRDD writeStatuses, throw new HoodieCompactionException( "Failed to commit " + metaClient.getBasePath() + " at time " + compactionCommitTime, e); } + return metadata; } /** @@ -1043,9 +1059,14 @@ private void rollbackInflightCommits() { } private HoodieTable getTableAndInitCtx() { - writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - return HoodieTable.getHoodieTable( + HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + if (table.getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { + writeContext = metrics.getCommitCtx(); + } else { + writeContext = metrics.getDeltaCommitCtx(); + } + return table; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 82d5a9d0b74f5..7ead846deacf9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -24,6 +24,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; @@ -73,6 +74,7 @@ public class HoodieAppendHandle extends HoodieIOH private HoodieLogFile currentLogFile; private Writer writer; private boolean doInit = true; + private long estimatedNumberOfBytesWritten; public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String fileId, Iterator> recordItr) { @@ -180,6 +182,7 @@ public void doAppend() { logger.info("AvgRecordSize => " + averageRecordSize); averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2; doAppend(header); + estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; numberOfRecords = 0; } Optional indexedRecord = getIndexedRecord(record); @@ -191,6 +194,7 @@ public void doAppend() { numberOfRecords++; } doAppend(header); + estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; } private void doAppend(Map header) { @@ -217,7 +221,11 @@ public void close() { } writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); + writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten); writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size()); + RuntimeStats runtimeStats = new RuntimeStats(); + runtimeStats.setTotalUpsertTime(timer.endTimer()); + writeStatus.getStat().setRuntimeStats(runtimeStats); } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index f16b2d21169b7..c2ee5f8c0ed70 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -22,6 +22,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; +import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -136,6 +137,9 @@ public WriteStatus close() { stat.setPaths(new Path(config.getBasePath()), path, tempPath); stat.setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath())); stat.setTotalWriteErrors(status.getFailedRecords().size()); + RuntimeStats runtimeStats = new RuntimeStats(); + runtimeStats.setTotalCreateTime(timer.endTimer()); + stat.setRuntimeStats(runtimeStats); status.setStat(stat); return status; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index a0032de2b8a62..e4a43af1998b0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -21,6 +21,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.HoodieTimer; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; @@ -41,6 +42,7 @@ public abstract class HoodieIOHandle { protected final HoodieTable hoodieTable; protected final Schema schema; protected HoodieTimeline hoodieTimeline; + protected HoodieTimer timer; public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { this.commitTime = commitTime; @@ -49,6 +51,7 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable totalLogFiles; + // Accumulator to keep track of total log file slices for a dataset + private AccumulatorV2 totalFileSlices; @Override public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable hoodieTable, String compactionCommitTime) throws IOException { + totalLogFiles = new LongAccumulator(); + totalFileSlices = new LongAccumulator(); + jsc.sc().register(totalLogFiles); + jsc.sc().register(totalFileSlices); + List operations = getCompactionWorkload(jsc, hoodieTable, config, compactionCommitTime); if (operations == null) { @@ -117,10 +130,18 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { - s.getStat().setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate()); - s.getStat().setTotalLogFiles(scanner.getTotalLogFiles()); + s.getStat().setTotalUpdatedRecordsCompacted(scanner.getTotalRecordsToUpdate()); + s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); s.getStat().setPartitionPath(operation.getPartitionPath()); + s.getStat().setTotalLogSizeCompacted((long) operation.getMetrics().get( + CompactionStrategy.TOTAL_LOG_FILE_SIZE)); + s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks()); + s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks()); + s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks()); + RuntimeStats runtimeStats = new RuntimeStats(); + runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks()); + s.getStat().setRuntimeStats(runtimeStats); return s; }).collect(toList()); } @@ -145,14 +166,23 @@ private List getCompactionWorkload(JavaSparkContext jsc, TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) - .flatMap((FlatMapFunction) partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath).map( - s -> new CompactionOperation(s.getDataFile().get(), partitionPath, - s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed()) - .collect(Collectors.toList()), config)) - .filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator()).collect(); + List operations = + jsc.parallelize(partitionPaths, partitionPaths.size()) + .flatMap((FlatMapFunction) partitionPath -> fileSystemView + .getLatestFileSlices(partitionPath).map( + s -> { + List logFiles = s.getLogFiles().sorted(HoodieLogFile + .getLogVersionComparator().reversed()).collect(Collectors.toList()); + totalLogFiles.add((long) logFiles.size()); + totalFileSlices.add(1L); + return new CompactionOperation(s.getDataFile().get(), partitionPath, logFiles, config); + }) + .filter(c -> !c.getDeltaFilePaths().isEmpty()) + .collect(toList()).iterator()).collect(); log.info("Total of " + operations.size() + " compactions are retrieved"); + log.info("Total number of latest files slices " + totalFileSlices.value()); + log.info("Total number of log files " + totalLogFiles.value()); + log.info("Total number of file slices " + totalFileSlices.value()); // Filter the compactions with the passed in filter. This lets us choose most effective // compactions only diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java index 9393a04790808..4f4cdf12801c8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java @@ -17,15 +17,9 @@ package com.uber.hoodie.io.compact.strategy; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.uber.hoodie.common.model.HoodieDataFile; -import com.uber.hoodie.common.model.HoodieLogFile; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.CompactionOperation; import java.util.List; -import java.util.Map; -import java.util.Optional; /** * CompactionStrategy which looks at total IO to be done for the compaction (read + write) and @@ -33,33 +27,7 @@ * * @see CompactionStrategy */ -public class BoundedIOCompactionStrategy implements CompactionStrategy { - - public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB"; - public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB"; - public static final String TOTAL_IO_MB = "TOTAL_IO_MB"; - - @Override - public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, - List logFiles) { - Map metrics = Maps.newHashMap(); - // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) - .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) - .orElse(0L); - // Total read will be the base file + all the log files - Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize); - // Total write will be similar to the size of the base file - Long totalIOWrite = FSUtils.getSizeInMB(dataFile.getFileSize()); - // Total IO will the the IO for read + write - Long totalIO = totalIORead + totalIOWrite; - // Save these metrics and we will use during the filter - metrics.put(TOTAL_IO_READ_MB, totalIORead); - metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite); - metrics.put(TOTAL_IO_MB, totalIO); - return metrics; - - } +public class BoundedIOCompactionStrategy extends CompactionStrategy { @Override public List orderAndFilter(HoodieWriteConfig writeConfig, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index c5a666cbd8f52..75d62e803368e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -16,47 +16,76 @@ package com.uber.hoodie.io.compact.strategy; +import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.CompactionOperation; import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Optional; /** - * Strategy for compaction. Pluggable implementation of define how compaction should be done. The - * implementations of this interface can capture the relevant metrics to order and filter the final - * list of compaction operation to run in a single compaction. - *

+ * Strategy for compaction. Pluggable implementation to define how compaction should be done. The + * over-ridden implementations of this abstract class can capture the relevant metrics to order + * and filter the final list of compaction operation to run in a single compaction. * Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be * passed in every time * * @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor * @see CompactionOperation */ -public interface CompactionStrategy extends Serializable { +public abstract class CompactionStrategy implements Serializable { + + public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB"; + public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB"; + public static final String TOTAL_IO_MB = "TOTAL_IO_MB"; + public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE"; + public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES"; /** * Callback hook when a CompactionOperation is created. Individual strategies can capture the * metrics they need to decide on the priority. * - * @param dataFile - Base file to compact + * @param dataFile - Base file to compact * @param partitionPath - Partition path - * @param logFiles - List of log files to compact with the base file + * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ - Map captureMetrics(HoodieDataFile dataFile, String partitionPath, - List logFiles); + public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, + List logFiles) { + Map metrics = Maps.newHashMap(); + // Total size of all the log files + Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent) + .map(Optional::get).reduce((size1, size2) -> size1 + size2).orElse(0L); + // Total read will be the base file + all the log files + Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize); + // Total write will be similar to the size of the base file + Long totalIOWrite = FSUtils.getSizeInMB(dataFile.getFileSize()); + // Total IO will the the IO for read + write + Long totalIO = totalIORead + totalIOWrite; + // Save these metrics and we will use during the filter + metrics.put(TOTAL_IO_READ_MB, totalIORead); + metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite); + metrics.put(TOTAL_IO_MB, totalIO); + metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize); + metrics.put(TOTAL_LOG_FILES, logFiles.size()); + return metrics; + + } /** * Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to * order and filter out compactions * * @param writeConfig - HoodieWriteConfig - config for this compaction is passed in - * @param operations - list of compactions collected + * @param operations - list of compactions collected * @return list of compactions to perform in this run */ - List orderAndFilter(HoodieWriteConfig writeConfig, - List operations); + public List orderAndFilter(HoodieWriteConfig writeConfig, + List operations) { + return operations; + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java index 08f46019fdd7a..3f8297f280aa2 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java @@ -16,13 +16,9 @@ package com.uber.hoodie.io.compact.strategy; -import com.google.common.collect.Maps; -import com.uber.hoodie.common.model.HoodieDataFile; -import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.CompactionOperation; import java.util.List; -import java.util.Map; /** * UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a @@ -31,13 +27,7 @@ * * @see CompactionStrategy */ -public class UnBoundedCompactionStrategy implements CompactionStrategy { - - @Override - public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, - List logFiles) { - return Maps.newHashMap(); - } +public class UnBoundedCompactionStrategy extends CompactionStrategy { @Override public List orderAndFilter(HoodieWriteConfig config, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index 78ef4960c025f..901bd236e8bde 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -21,6 +21,7 @@ import com.codahale.metrics.Timer; import com.google.common.annotations.VisibleForTesting; import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.config.HoodieWriteConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -35,12 +36,14 @@ public class HoodieMetrics { public String rollbackTimerName = null; public String cleanTimerName = null; public String commitTimerName = null; + public String deltaCommitTimerName = null; public String finalizeTimerName = null; private HoodieWriteConfig config = null; private String tableName = null; private Timer rollbackTimer = null; private Timer cleanTimer = null; private Timer commitTimer = null; + private Timer deltaCommitTimer = null; private Timer finalizeTimer = null; public HoodieMetrics(HoodieWriteConfig config, String tableName) { @@ -48,9 +51,10 @@ public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.tableName = tableName; if (config.isMetricsOn()) { Metrics.init(config); - this.rollbackTimerName = getMetricsName("timer", "rollback"); - this.cleanTimerName = getMetricsName("timer", "clean"); - this.commitTimerName = getMetricsName("timer", "commit"); + this.rollbackTimerName = getMetricsName("timer", HoodieTimeline.ROLLBACK_ACTION); + this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION); + this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION); + this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION); this.finalizeTimerName = getMetricsName("timer", "finalize"); } } @@ -87,8 +91,15 @@ public Timer.Context getFinalizeCtx() { return finalizeTimer == null ? null : finalizeTimer.time(); } + public Timer.Context getDeltaCommitCtx() { + if (config.isMetricsOn() && deltaCommitTimer == null) { + deltaCommitTimer = createTimer(deltaCommitTimerName); + } + return deltaCommitTimer == null ? null : deltaCommitTimer.time(); + } + public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, - HoodieCommitMetadata metadata) { + HoodieCommitMetadata metadata, String actionType) { if (config.isMetricsOn()) { long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten(); long totalFilesInsert = metadata.fetchTotalFilesInsert(); @@ -97,17 +108,27 @@ public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten(); long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten(); long totalBytesWritten = metadata.fetchTotalBytesWritten(); - registerGauge(getMetricsName("commit", "duration"), durationInMs); - registerGauge(getMetricsName("commit", "totalPartitionsWritten"), totalPartitionsWritten); - registerGauge(getMetricsName("commit", "totalFilesInsert"), totalFilesInsert); - registerGauge(getMetricsName("commit", "totalFilesUpdate"), totalFilesUpdate); - registerGauge(getMetricsName("commit", "totalRecordsWritten"), totalRecordsWritten); - registerGauge(getMetricsName("commit", "totalUpdateRecordsWritten"), - totalUpdateRecordsWritten); - registerGauge(getMetricsName("commit", "totalInsertRecordsWritten"), - totalInsertRecordsWritten); - registerGauge(getMetricsName("commit", "totalBytesWritten"), totalBytesWritten); - registerGauge(getMetricsName("commit", "commitTime"), commitEpochTimeInMs); + long totalTimeTakenByScanner = metadata.getTotalScanTime(); + long totalTimeTakenForInsert = metadata.getTotalCreateTime(); + long totalTimeTakenForUpsert = metadata.getTotalUpsertTime(); + long totalCompactedRecordsUpdated = metadata.getTotalCompactedRecordsUpdated(); + long totalLogFilesCompacted = metadata.getTotalLogFilesCompacted(); + long totalLogFilesSize = metadata.getTotalLogFilesSize(); + registerGauge(getMetricsName(actionType, "duration"), durationInMs); + registerGauge(getMetricsName(actionType, "totalPartitionsWritten"), totalPartitionsWritten); + registerGauge(getMetricsName(actionType, "totalFilesInsert"), totalFilesInsert); + registerGauge(getMetricsName(actionType, "totalFilesUpdate"), totalFilesUpdate); + registerGauge(getMetricsName(actionType, "totalRecordsWritten"), totalRecordsWritten); + registerGauge(getMetricsName(actionType, "totalUpdateRecordsWritten"), totalUpdateRecordsWritten); + registerGauge(getMetricsName(actionType, "totalInsertRecordsWritten"), totalInsertRecordsWritten); + registerGauge(getMetricsName(actionType, "totalBytesWritten"), totalBytesWritten); + registerGauge(getMetricsName(actionType, "commitTime"), commitEpochTimeInMs); + registerGauge(getMetricsName(actionType, "totalScanTime"), totalTimeTakenByScanner); + registerGauge(getMetricsName(actionType, "totalCreateTime"), totalTimeTakenForInsert); + registerGauge(getMetricsName(actionType, "totalUpsertTime"), totalTimeTakenForUpsert); + registerGauge(getMetricsName(actionType, "totalCompactedRecordsUpdated"), totalCompactedRecordsUpdated); + registerGauge(getMetricsName(actionType, "totalLogFilesCompacted"), totalLogFilesCompacted); + registerGauge(getMetricsName(actionType, "totalLogFilesSize"), totalLogFilesSize); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 67f8d6d6e5c0c..a57069d1dc8cb 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -67,6 +67,7 @@ import org.apache.spark.sql.SQLContext; import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; @@ -660,6 +661,48 @@ public void testLogFileCountsAfterCompaction() throws Exception { } } + @Test + public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception { + // insert 100 records + HoodieWriteConfig config = getConfig(false); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD statuses = writeClient.insert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + + // total time taken for creating files should be greater than 0 + long totalCreateTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalCreateTime()) + .reduce((a,b) -> a + b).intValue(); + Assert.assertTrue(totalCreateTime > 0); + + // Update all the 100 records + newCommitTime = "101"; + writeClient.startCommitWithTime(newCommitTime); + + List updatedRecords = dataGen.generateUpdates(newCommitTime, records); + JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); + statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + // total time taken for upsert all records should be greater than 0 + long totalUpsertTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalUpsertTime()) + .reduce((a,b) -> a + b).intValue(); + Assert.assertTrue(totalUpsertTime > 0); + + // Do a compaction + String commitTime = writeClient.startCompaction(); + statuses = writeClient.compact(commitTime); + writeClient.commitCompaction(commitTime, statuses); + // total time taken for scanning log files should be greater than 0 + long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime()) + .reduce((a,b) -> a + b).longValue(); + Assert.assertTrue(timeTakenForScanner > 0); + } + private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit).build(); } diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index ae3df4ffc524d..832b5fc9ccf7a 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -58,7 +58,7 @@ "type":["null","long"] }, { - "name":"totalRecordsToBeUpdate", + "name":"totalUpdatedRecordsCompacted", "type":["null","long"] } ] diff --git a/hoodie-common/src/main/avro/HoodieCompactionMetadata.avsc b/hoodie-common/src/main/avro/HoodieCompactionMetadata.avsc index 92e774d78fcf1..871b1b5131a41 100644 --- a/hoodie-common/src/main/avro/HoodieCompactionMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCompactionMetadata.avsc @@ -26,7 +26,7 @@ "type":["null","long"] }, { - "name":"totalRecordsToBeUpdate", + "name":"totalUpdatedRecordsCompacted", "type":["null","long"] }, { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index 3f1b69080f786..fe97a4e3427a5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -208,6 +208,92 @@ public long fetchTotalWriteErrors() { return totalWriteErrors; } + public long getTotalRecordsDeleted() { + long totalDeletes = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + totalDeletes += stat.getNumDeletes(); + } + } + return totalDeletes; + } + + public Long getTotalLogRecordsCompacted() { + Long totalLogRecords = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + totalLogRecords += writeStat.getTotalLogRecords(); + } + } + return totalLogRecords; + } + + public Long getTotalLogFilesCompacted() { + Long totalLogFiles = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + totalLogFiles += writeStat.getTotalLogFilesCompacted(); + } + } + return totalLogFiles; + } + + public Long getTotalCompactedRecordsUpdated() { + Long totalUpdateRecords = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + totalUpdateRecords += writeStat.getTotalUpdatedRecordsCompacted(); + } + } + return totalUpdateRecords; + } + + public Long getTotalLogFilesSize() { + Long totalLogFilesSize = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + totalLogFilesSize += writeStat.getTotalLogSizeCompacted(); + } + } + return totalLogFilesSize; + } + + public Long getTotalScanTime() { + Long totalScanTime = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + if (writeStat.getRuntimeStats() != null) { + totalScanTime += writeStat.getRuntimeStats().getTotalScanTime(); + } + } + } + return totalScanTime; + } + + public Long getTotalCreateTime() { + Long totalCreateTime = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + if (writeStat.getRuntimeStats() != null) { + totalCreateTime += writeStat.getRuntimeStats().getTotalCreateTime(); + } + } + } + return totalCreateTime; + } + + public Long getTotalUpsertTime() { + Long totalUpsertTime = 0L; + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat writeStat : entry.getValue()) { + if (writeStat.getRuntimeStats() != null) { + totalUpsertTime += writeStat.getRuntimeStats().getTotalUpsertTime(); + } + } + } + return totalUpsertTime; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -236,7 +322,7 @@ public int hashCode() { public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException { return fromJsonString(new String(bytes, Charset.forName("utf-8"))); } - + private static ObjectMapper getObjectMapper() { ObjectMapper mapper = new ObjectMapper(); mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index 4706e25a3f260..e03e84e3a67da 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -16,6 +16,7 @@ package com.uber.hoodie.common.model; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import java.io.Serializable; import javax.annotation.Nullable; @@ -90,19 +91,46 @@ public class HoodieWriteStat implements Serializable { * Total number of log records that were compacted by a compaction operation */ @Nullable - private Long totalLogRecords; + private long totalLogRecords; /** - * Total number of log files that were compacted by a compaction operation + * Total number of log files compacted for a file slice with this base fileid */ @Nullable - private Long totalLogFiles; + private long totalLogFilesCompacted; + + /** + * Total size of all log files for a file slice with this base fileid + */ + @Nullable + private long totalLogSizeCompacted; /** * Total number of records updated by a compaction operation */ @Nullable - private Long totalRecordsToBeUpdate; + private long totalUpdatedRecordsCompacted; + + /** + * Total number of log blocks seen in a compaction operation + */ + @Nullable + private long totalLogBlocks; + + /** + * Total number of corrupt blocks seen in a compaction operation + */ + @Nullable + private long totalCorruptLogBlock; + + /** + * Total number of rollback blocks seen in a compaction operation + */ + private long totalRollbackBlocks; + + @Nullable + @JsonIgnore + private RuntimeStats runtimeStats; public HoodieWriteStat() { // called by jackson json lib @@ -180,28 +208,28 @@ public void setPartitionPath(String partitionPath) { this.partitionPath = partitionPath; } - public Long getTotalLogRecords() { + public long getTotalLogRecords() { return totalLogRecords; } - public void setTotalLogRecords(Long totalLogRecords) { + public void setTotalLogRecords(long totalLogRecords) { this.totalLogRecords = totalLogRecords; } - public Long getTotalLogFiles() { - return totalLogFiles; + public long getTotalLogFilesCompacted() { + return totalLogFilesCompacted; } - public void setTotalLogFiles(Long totalLogFiles) { - this.totalLogFiles = totalLogFiles; + public void setTotalLogFilesCompacted(long totalLogFilesCompacted) { + this.totalLogFilesCompacted = totalLogFilesCompacted; } - public Long getTotalRecordsToBeUpdate() { - return totalRecordsToBeUpdate; + public long getTotalUpdatedRecordsCompacted() { + return totalUpdatedRecordsCompacted; } - public void setTotalRecordsToBeUpdate(Long totalRecordsToBeUpdate) { - this.totalRecordsToBeUpdate = totalRecordsToBeUpdate; + public void setTotalUpdatedRecordsCompacted(long totalUpdatedRecordsCompacted) { + this.totalUpdatedRecordsCompacted = totalUpdatedRecordsCompacted; } public void setTempPath(String tempPath) { @@ -212,6 +240,47 @@ public String getTempPath() { return this.tempPath; } + public long getTotalLogSizeCompacted() { + return totalLogSizeCompacted; + } + + public void setTotalLogSizeCompacted(long totalLogSizeCompacted) { + this.totalLogSizeCompacted = totalLogSizeCompacted; + } + + public long getTotalLogBlocks() { + return totalLogBlocks; + } + + public void setTotalLogBlocks(long totalLogBlocks) { + this.totalLogBlocks = totalLogBlocks; + } + + public long getTotalCorruptLogBlock() { + return totalCorruptLogBlock; + } + + public void setTotalCorruptLogBlock(long totalCorruptLogBlock) { + this.totalCorruptLogBlock = totalCorruptLogBlock; + } + + public long getTotalRollbackBlocks() { + return totalRollbackBlocks; + } + + public void setTotalRollbackBlocks(Long totalRollbackBlocks) { + this.totalRollbackBlocks = totalRollbackBlocks; + } + + @Nullable + public RuntimeStats getRuntimeStats() { + return runtimeStats; + } + + public void setRuntimeStats(@Nullable RuntimeStats runtimeStats) { + this.runtimeStats = runtimeStats; + } + /** * Set path and tempPath relative to the given basePath. */ @@ -224,17 +293,25 @@ public void setPaths(Path basePath, Path path, Path tempPath) { @Override public String toString() { - return new StringBuilder() - .append("HoodieWriteStat {") - .append("path=" + path) - .append(", tempPath=" + tempPath) - .append(", prevCommit='" + prevCommit + '\'') - .append(", numWrites=" + numWrites) - .append(", numDeletes=" + numDeletes) - .append(", numUpdateWrites=" + numUpdateWrites) - .append(", numWriteBytes=" + totalWriteBytes) - .append('}') - .toString(); + return "HoodieWriteStat{" + + "fileId='" + fileId + '\'' + + ", path='" + path + '\'' + + ", prevCommit='" + prevCommit + '\'' + + ", numWrites=" + numWrites + + ", numDeletes=" + numDeletes + + ", numUpdateWrites=" + numUpdateWrites + + ", totalWriteBytes=" + totalWriteBytes + + ", totalWriteErrors=" + totalWriteErrors + + ", tempPath='" + tempPath + '\'' + + ", partitionPath='" + partitionPath + + '\'' + ", totalLogRecords=" + totalLogRecords + + ", totalLogFilesCompacted=" + totalLogFilesCompacted + + ", totalLogSizeCompacted=" + totalLogSizeCompacted + + ", totalUpdatedRecordsCompacted=" + totalUpdatedRecordsCompacted + + ", totalLogBlocks=" + totalLogBlocks + + ", totalCorruptLogBlock=" + totalCorruptLogBlock + + ", totalRollbackBlocks=" + totalRollbackBlocks + + '}'; } @Override @@ -260,4 +337,51 @@ public int hashCode() { result = 31 * result + prevCommit.hashCode(); return result; } + + public static class RuntimeStats implements Serializable { + /** + * Total time taken to read and merge logblocks in a log file + */ + @Nullable + private long totalScanTime; + + /** + * Total time taken by a Hoodie Merge for an existing file + */ + @Nullable + private long totalUpsertTime; + + /** + * Total time taken by a Hoodie Insert to a file + */ + @Nullable + private long totalCreateTime; + + @Nullable + public long getTotalScanTime() { + return totalScanTime; + } + + public void setTotalScanTime(@Nullable long totalScanTime) { + this.totalScanTime = totalScanTime; + } + + @Nullable + public long getTotalUpsertTime() { + return totalUpsertTime; + } + + public void setTotalUpsertTime(@Nullable long totalUpsertTime) { + this.totalUpsertTime = totalUpsertTime; + } + + @Nullable + public long getTotalCreateTime() { + return totalCreateTime; + } + + public void setTotalCreateTime(@Nullable long totalCreateTime) { + this.totalCreateTime = totalCreateTime; + } + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index d23792fc7fae8..6e9699442e893 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -29,6 +29,7 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.util.HoodieTimer; import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; @@ -73,8 +74,14 @@ public class HoodieCompactedLogRecordScanner implements private final Schema readerSchema; // Total log files read - for metrics private AtomicLong totalLogFiles = new AtomicLong(0); + // Total log blocks read - for metrics + private AtomicLong totalLogBlocks = new AtomicLong(0); // Total log records read - for metrics private AtomicLong totalLogRecords = new AtomicLong(0); + // Total number of rollbacks written across all log files + private AtomicLong totalRollbacks = new AtomicLong(0); + // Total number of corrupt blocks written across all log files + private AtomicLong totalCorruptBlocks = new AtomicLong(0); // Total final list of compacted/merged records private long totalRecordsToUpdate; // Latest valid instant time @@ -84,6 +91,10 @@ public class HoodieCompactedLogRecordScanner implements private String payloadClassFQN; // Store the last instant log blocks (needed to implement rollback) private Deque currentInstantLogBlocks = new ArrayDeque<>(); + // Stores the total time taken to perform reading and merging of log blocks + private long totalTimeTakenToReadAndMergeBlocks = 0L; + // A timer for calculating elapsed time in millis + public HoodieTimer timer = new HoodieTimer(); public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, @@ -93,6 +104,8 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List new HoodieLogFile(new Path(logFile))) .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize); + HoodieLogFile logFile; while (logFormatReaderWrapper.hasNext()) { - HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); + logFile = logFormatReaderWrapper.getLogFile(); log.info("Scanning log file " + logFile); - totalLogFiles.incrementAndGet(); // Use the HoodieLogFileReader to iterate through the blocks in the log file HoodieLogBlock r = logFormatReaderWrapper.next(); if (r.getBlockType() != CORRUPT_BLOCK @@ -164,6 +177,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List " + maxMemorySizeInBytes); - log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records - .getInMemoryMapNumEntries()); - log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records - .getCurrentInMemoryMapSize()); - log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records - .getDiskBasedMapNumEntries()); + log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); + log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); + log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); + log.debug("Total time taken for scanning and compacting log files => " + totalTimeTakenToReadAndMergeBlocks); } /** @@ -307,6 +321,10 @@ public long getTotalLogRecords() { return totalLogRecords.get(); } + public long getTotalLogBlocks() { + return totalLogBlocks.get(); + } + public Map> getRecords() { return records; } @@ -314,5 +332,17 @@ public Map> getRecords() { public long getTotalRecordsToUpdate() { return totalRecordsToUpdate; } + + public long getTotalRollbacks() { + return totalRollbacks.get(); + } + + public long getTotalCorruptBlocks() { + return totalCorruptBlocks.get(); + } + + public long getTotalTimeTakenToReadAndMergeBlocks() { + return totalTimeTakenToReadAndMergeBlocks; + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieTimer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieTimer.java new file mode 100644 index 0000000000000..6e87387a041e7 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieTimer.java @@ -0,0 +1,70 @@ +/* + * 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.common.util; + +import com.uber.hoodie.exception.HoodieException; +import java.util.ArrayDeque; +import java.util.Deque; + +/** + * Timing utility to help keep track of execution times of code blocks. This class helps to allow multiple + * timers started at the same time and automatically returns the execution time in the order in which the + * timers are stopped. + */ +public class HoodieTimer { + + // Ordered stack of TimeInfo's to make sure stopping the timer returns the correct elapsed time + Deque timeInfoDeque = new ArrayDeque<>(); + + class TimeInfo { + + // captures the startTime of the code block + long startTime; + // is the timing still running for the last started timer + boolean isRunning; + + public TimeInfo(long startTime) { + this.startTime = startTime; + this.isRunning = true; + } + + public long getStartTime() { + return startTime; + } + + public boolean isRunning() { + return isRunning; + } + + public long stop() { + this.isRunning = false; + return System.currentTimeMillis() - startTime; + } + } + + public HoodieTimer startTimer() { + timeInfoDeque.push(new TimeInfo(System.currentTimeMillis())); + return this; + } + + public long endTimer() { + if (timeInfoDeque.isEmpty()) { + throw new HoodieException("Timer was not started"); + } + return timeInfoDeque.pop().stop(); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 6aab1e1cd29b2..6a956f27c9766 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -27,6 +27,7 @@ import com.google.common.collect.Maps; import com.uber.hoodie.avro.model.HoodieCleanMetadata; import com.uber.hoodie.common.HoodieCleanStat; +import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.table.HoodieTableConfig; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -335,4 +336,25 @@ public static List monotonicIncreasingCommitTimestamps(int numTimestamps } return commits; } + + public static List generateFakeHoodieWriteStat(int limit) { + List writeStatList = new ArrayList<>(); + for (int i = 0; i < limit; i++) { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(UUID.randomUUID().toString()); + writeStat.setNumDeletes(0); + writeStat.setNumUpdateWrites(100); + writeStat.setNumWrites(100); + writeStat.setPath("/some/fake/path" + i); + writeStat.setPartitionPath("/some/fake/partition/path" + i); + writeStat.setTotalLogFilesCompacted(100L); + RuntimeStats runtimeStats = new RuntimeStats(); + runtimeStats.setTotalScanTime(100); + runtimeStats.setTotalCreateTime(100); + runtimeStats.setTotalUpsertTime(100); + writeStat.setRuntimeStats(runtimeStats); + writeStatList.add(writeStat); + } + return writeStatList; + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java new file mode 100644 index 0000000000000..5f92e83bf56e9 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java @@ -0,0 +1,42 @@ +/* + * 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.common.model; + +import java.util.List; +import org.junit.Assert; +import org.junit.Test; + +public class TestHoodieCommitMetadata { + + @Test + public void testPerfStatPresenceInHoodieMetadata() throws Exception { + + List fakeHoodieWriteStats = HoodieTestUtils.generateFakeHoodieWriteStat(100); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + fakeHoodieWriteStats.stream().forEach(stat -> commitMetadata.addWriteStat(stat.getPartitionPath(), stat)); + Assert.assertTrue(commitMetadata.getTotalCreateTime() > 0); + Assert.assertTrue(commitMetadata.getTotalUpsertTime() > 0); + Assert.assertTrue(commitMetadata.getTotalScanTime() > 0); + Assert.assertTrue(commitMetadata.getTotalLogFilesCompacted() > 0); + + String serializedCommitMetadata = commitMetadata.toJsonString(); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(serializedCommitMetadata); + // Make sure timing metrics are not written to instant file + Assert.assertTrue(metadata.getTotalScanTime() == 0); + Assert.assertTrue(metadata.getTotalLogFilesCompacted() > 0); + } +} From 8f1b94e17dbfb6226490b5a4d6eef472802800f8 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 2 May 2018 00:41:03 -0700 Subject: [PATCH 051/374] Adding a fix/workaround when fs.append() unable to return a valid outputstream --- .../table/log/HoodieLogFormatWriter.java | 23 ++- .../log/HoodieLogFormatAppendFailureTest.java | 141 ++++++++++++++++++ .../collection/TestExternalSpillableMap.java | 7 - 3 files changed, 161 insertions(+), 10 deletions(-) create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatAppendFailureTest.java diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index ee1dcf559fdd6..fa17711ba59db 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -46,6 +46,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { private final Integer bufferSize; private final Short replication; private FSDataOutputStream output; + private static final String APPEND_UNAVAILABLE_EXCEPTION_MESSAGE = "not sufficiently replicated yet"; /** * @param fs @@ -69,6 +70,18 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { try { this.output = fs.append(path, bufferSize); } catch (RemoteException e) { + if (e.getMessage().contains(APPEND_UNAVAILABLE_EXCEPTION_MESSAGE)) { + // This issue happens when all replicas for a file are down and/or being decommissioned. + // The fs.append() API could append to the last block for a file. If the last block is full, a new block is + // appended to. In a scenario when a lot of DN's are decommissioned, it can happen that DN's holding all + // replicas for a block/file are decommissioned together. During this process, all these blocks will start to + // get replicated to other active DataNodes but this process might take time (can be of the order of few + // hours). During this time, if a fs.append() API is invoked for a file whose last block is eligible to be + // appended to, then the NN will throw an exception saying that it couldn't find any active replica with the + // last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325 + log.warn("Failed to open an append stream to the log file. Opening a new log file..", e); + createNewFile(); + } // this happens when either another task executor writing to this file died or // data node is going down if (e.getClassName().equals(AlreadyBeingCreatedException.class.getName()) @@ -86,9 +99,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { } catch (IOException ioe) { if (ioe.getMessage().equalsIgnoreCase("Not supported")) { log.info("Append not supported. Opening a new log file.."); - this.logFile = logFile.rollOver(fs); - this.output = fs.create(this.logFile.getPath(), false, bufferSize, replication, - WriterBuilder.DEFAULT_SIZE_THRESHOLD, null); + createNewFile(); } else { throw ioe; } @@ -192,6 +203,12 @@ private Writer rolloverIfNeeded() throws IOException, InterruptedException { return this; } + private void createNewFile() throws IOException { + this.logFile = logFile.rollOver(fs); + this.output = fs.create(this.logFile.getPath(), false, bufferSize, replication, + WriterBuilder.DEFAULT_SIZE_THRESHOLD, null); + } + @Override public void close() throws IOException { flush(); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatAppendFailureTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatAppendFailureTest.java new file mode 100644 index 0000000000000..76a03a3477cf2 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatAppendFailureTest.java @@ -0,0 +1,141 @@ +/* + * 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.common.table.log; + +import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; + +import com.google.common.collect.Maps; +import com.uber.hoodie.common.minicluster.MiniClusterUtil; +import com.uber.hoodie.common.model.HoodieArchivedLogFile; +import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; +import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.util.SchemaTestUtil; +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeoutException; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * This class is intentionally using a different way of setting up the MiniDFSCluster and not relying on + * {@link MiniClusterUtil} to reproduce append() issue : https://issues.apache.org/jira/browse/HDFS-6325 + * Reference : https://issues.apache.org/jira/secure/attachment/12645053/HDFS-6325.patch + */ +public class HoodieLogFormatAppendFailureTest { + + private static File baseDir; + private static MiniDFSCluster cluster; + + @BeforeClass + public static void setUpClass() throws IOException { + // NOTE : The MiniClusterDFS leaves behind the directory under which the cluster was created + baseDir = new File("/tmp/" + UUID.randomUUID().toString()); + FileUtil.fullyDelete(baseDir); + // Append is not supported in LocalFileSystem. HDFS needs to be setup. + Configuration conf = new Configuration(); + // lower heartbeat interval for fast recognition of DN + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000); + conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); + conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 3000); + cluster = new MiniDFSCluster.Builder(conf).checkExitOnShutdown(true).numDataNodes(4).build(); + } + + @AfterClass + public static void tearDownClass() { + cluster.shutdown(true); + // Force clean up the directory under which the cluster was created + FileUtil.fullyDelete(baseDir); + } + + @Test(timeout = 60000) + public void testFailedToGetAppendStreamFromHDFSNameNode() throws IOException, URISyntaxException, + InterruptedException, TimeoutException { + + // Use some fs like LocalFileSystem, that does not support appends + String uuid = UUID.randomUUID().toString(); + Path localPartitionPath = new Path("/tmp/"); + FileSystem fs = cluster.getFileSystem(); + Path testPath = new Path(localPartitionPath, uuid); + fs.mkdirs(testPath); + + // Some data & append. + List records = SchemaTestUtil.generateTestRecords(0, 10); + Map header = Maps.newHashMap(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); + + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits" + + ".archive").overBaseCommit("") + .withFs(fs).build(); + + writer = writer.appendBlock(dataBlock); + // get the current log file version to compare later + int logFileVersion = writer.getLogFile().getLogVersion(); + Path logFilePath = writer.getLogFile().getPath(); + writer.close(); + + // Wait for 3 times replication of file + DFSTestUtil.waitReplication(fs, logFilePath, (short) 3); + // Shut down all DNs that have the last block location for the file + LocatedBlocks lbs = cluster.getFileSystem().getClient().getNamenode() + .getBlockLocations("/tmp/" + uuid + "/" + logFilePath.getName(), 0, Long.MAX_VALUE); + List dnsOfCluster = cluster.getDataNodes(); + DatanodeInfo[] dnsWithLocations = lbs.getLastLocatedBlock().getLocations(); + for (DataNode dn : dnsOfCluster) { + for (DatanodeInfo loc : dnsWithLocations) { + if (dn.getDatanodeId().equals(loc)) { + dn.shutdown(); + cluster.stopDataNode(dn.getDisplayName()); + DFSTestUtil.waitForDatanodeDeath(dn); + } + } + } + // Wait for the replication of this file to go down to 0 + DFSTestUtil.waitReplication(fs, logFilePath, (short) 0); + + // Opening a new Writer right now will throw IOException. The code should handle this, rollover the logfile and + // return a new writer with a bumped up logVersion + writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits" + ".archive") + .overBaseCommit("") + .withFs(fs).build(); + // The log version should be different for this new writer + Assert.assertFalse(writer.getLogFile().getLogVersion() == logFileVersion); + } + +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index c62aa1cc45ced..bb385dc1a098d 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -30,7 +30,6 @@ import com.uber.hoodie.common.util.SpillableMapTestUtils; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; import com.uber.hoodie.common.util.collection.converter.StringConverter; -import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; import java.net.URISyntaxException; @@ -182,12 +181,6 @@ public void simpleTestWithException() throws IOException, URISyntaxException { } } - @Test - public void simpleTestWithExceptionValidateFileIsRemoved() throws Exception { - File file = new File(FAILURE_OUTPUT_PATH); - assertFalse(file.exists()); - } - @Test public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException { From aaa5eb417cef1a7a7d1bb39c9f6bb1328f75bbcf Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Mon, 2 Apr 2018 22:53:28 -0700 Subject: [PATCH 052/374] Minor fixes for MergeOnRead MVP release readiness --- .../cli/commands/HoodieLogFileCommand.java | 9 ++- .../hoodie/config/HoodieMemoryConfig.java | 11 ++- .../uber/hoodie/config/HoodieWriteConfig.java | 4 ++ .../com/uber/hoodie/io/HoodieMergeHandle.java | 5 +- .../compact/HoodieRealtimeTableCompactor.java | 3 +- .../log/HoodieCompactedLogRecordScanner.java | 5 +- .../table/log/HoodieLogFormatWriter.java | 72 +++++++++++-------- .../common/util/collection/DiskBasedMap.java | 18 ++--- .../util/collection/ExternalSpillableMap.java | 3 +- .../common/table/log/HoodieLogFormatTest.java | 23 +++--- .../util/collection/TestDiskBasedMap.java | 12 ++-- .../collection/TestExternalSpillableMap.java | 23 ++++-- .../realtime/HoodieRealtimeRecordReader.java | 7 +- 13 files changed, 117 insertions(+), 78 deletions(-) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index b22d45d72f026..7115b67a2918f 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -82,6 +82,7 @@ public String showLogFileCommits( commitCountAndMetadata = Maps.newHashMap(); int totalEntries = 0; int numCorruptBlocks = 0; + int dummyInstantTimeCount = 0; for (String logFilePath : logFilePaths) { FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath)); @@ -108,6 +109,11 @@ public String showLogFileCommits( } } else { instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME); + if (instantTime == null) { + // This can happen when reading archived commit files since they were written without any instant time + dummyInstantTimeCount++; + instantTime = "dummy_instant_time_" + dummyInstantTimeCount; + } if (n instanceof HoodieAvroDataBlock) { recordCount = ((HoodieAvroDataBlock) n).getRecords().size(); } @@ -188,7 +194,8 @@ public String showLogFileRecords(@CliOption(key = { Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES), Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED), - Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); + Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), + HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH); for (HoodieRecord hoodieRecord : scanner) { Optional record = hoodieRecord.getData().getInsertValue(readerSchema); if (allRecords.size() >= limit) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java index 3fd6d6c4e6305..6efa70b0d4ab6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java @@ -48,7 +48,9 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig { // Property to set the max memory for dfs inputstream buffer size public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size"; public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 16 * 1024 * 1024; // 16MB - + public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path"; + // Default file path prefix for spillable file + public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/"; private HoodieMemoryConfig(Properties props) { super(props); @@ -77,13 +79,13 @@ public Builder fromProperties(Properties props) { return this; } - public Builder withMaxMemoryFractionPerPartitionMerge(long maxMemoryFractionPerPartitionMerge) { + public Builder withMaxMemoryFractionPerPartitionMerge(double maxMemoryFractionPerPartitionMerge) { props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, String.valueOf(maxMemoryFractionPerPartitionMerge)); return this; } - public Builder withMaxMemoryFractionPerCompaction(long maxMemoryFractionPerCompaction) { + public Builder withMaxMemoryFractionPerCompaction(double maxMemoryFractionPerCompaction) { props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, String.valueOf(maxMemoryFractionPerCompaction)); return this; @@ -155,6 +157,9 @@ public HoodieMemoryConfig build() { setDefaultOnCondition(props, !props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP), MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); + setDefaultOnCondition(props, + !props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP), + SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 633e5695b82c4..e0fe99379e0db 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -375,6 +375,10 @@ public int getMaxDFSStreamBufferSize() { props.getProperty(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP)); } + public String getSpillableMapBasePath() { + return props.getProperty(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP); + } + public static class Builder { private final Properties props = new Properties(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index d5c6a90de87b3..8e28b8d986336 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -141,8 +141,9 @@ private String init(String fileId, Iterator> newRecordsItr) { try { // Load the new records in a map logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); - this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), Optional.empty(), - new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass())); + this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), + config.getSpillableMapBasePath(), new StringConverter(), + new HoodieRecordConverter(schema, config.getPayloadClass())); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 8983605c5187e..cbf06d9366f5a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -118,7 +118,8 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(), - config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize()); + config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), + config.getSpillableMapBasePath()); if (!scanner.iterator().hasNext()) { return Lists.newArrayList(); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 6e9699442e893..86da05b737fab 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -42,7 +42,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.avro.Schema; @@ -98,7 +97,7 @@ public class HoodieCompactedLogRecordScanner implements public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, - boolean readBlocksLazily, boolean reverseReader, int bufferSize) { + boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); @@ -109,7 +108,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List(maxMemorySizeInBytes, Optional.empty(), + this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new StringConverter(), new HoodieRecordConverter(readerSchema, payloadClassFQN)); // iterate over the paths HoodieLogFormatReader logFormatReaderWrapper = diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index fa17711ba59db..e0644d69fd8ce 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -22,12 +22,14 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; +import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.ipc.RemoteException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -70,35 +72,11 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { try { this.output = fs.append(path, bufferSize); } catch (RemoteException e) { - if (e.getMessage().contains(APPEND_UNAVAILABLE_EXCEPTION_MESSAGE)) { - // This issue happens when all replicas for a file are down and/or being decommissioned. - // The fs.append() API could append to the last block for a file. If the last block is full, a new block is - // appended to. In a scenario when a lot of DN's are decommissioned, it can happen that DN's holding all - // replicas for a block/file are decommissioned together. During this process, all these blocks will start to - // get replicated to other active DataNodes but this process might take time (can be of the order of few - // hours). During this time, if a fs.append() API is invoked for a file whose last block is eligible to be - // appended to, then the NN will throw an exception saying that it couldn't find any active replica with the - // last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325 - log.warn("Failed to open an append stream to the log file. Opening a new log file..", e); - createNewFile(); - } - // this happens when either another task executor writing to this file died or - // data node is going down - if (e.getClassName().equals(AlreadyBeingCreatedException.class.getName()) - && fs instanceof DistributedFileSystem) { - log.warn("Trying to recover log on path " + path); - if (FSUtils.recoverDFSFileLease((DistributedFileSystem) fs, path)) { - log.warn("Recovered lease on path " + path); - // try again - this.output = fs.append(path, bufferSize); - } else { - log.warn("Failed to recover lease on path " + path); - throw new HoodieException(e); - } - } + handleAppendExceptionOrRecoverLease(path, e); } catch (IOException ioe) { if (ioe.getMessage().equalsIgnoreCase("Not supported")) { log.info("Append not supported. Opening a new log file.."); + this.logFile = logFile.rollOver(fs); createNewFile(); } else { throw ioe; @@ -107,8 +85,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { } else { log.info(logFile + " does not exist. Create a new file"); // Block size does not matter as we will always manually autoflush - this.output = fs.create(path, false, bufferSize, replication, - WriterBuilder.DEFAULT_SIZE_THRESHOLD, null); + createNewFile(); // TODO - append a file level meta block } } @@ -204,7 +181,6 @@ private Writer rolloverIfNeeded() throws IOException, InterruptedException { } private void createNewFile() throws IOException { - this.logFile = logFile.rollOver(fs); this.output = fs.create(this.logFile.getPath(), false, bufferSize, replication, WriterBuilder.DEFAULT_SIZE_THRESHOLD, null); } @@ -221,7 +197,9 @@ private void flush() throws IOException { return; // Presume closed } output.flush(); - output.hflush(); + // NOTE : the following API call makes sure that the data is flushed to disk on DataNodes (akin to POSIX fsync()) + // See more details here : https://issues.apache.org/jira/browse/HDFS-744 + output.hsync(); } public long getCurrentSize() throws IOException { @@ -232,4 +210,38 @@ public long getCurrentSize() throws IOException { return output.getPos(); } + private void handleAppendExceptionOrRecoverLease(Path path, RemoteException e) throws IOException, + InterruptedException { + if (e.getMessage().contains(APPEND_UNAVAILABLE_EXCEPTION_MESSAGE)) { + // This issue happens when all replicas for a file are down and/or being decommissioned. + // The fs.append() API could append to the last block for a file. If the last block is full, a new block is + // appended to. In a scenario when a lot of DN's are decommissioned, it can happen that DN's holding all + // replicas for a block/file are decommissioned together. During this process, all these blocks will start to + // get replicated to other active DataNodes but this process might take time (can be of the order of few + // hours). During this time, if a fs.append() API is invoked for a file whose last block is eligible to be + // appended to, then the NN will throw an exception saying that it couldn't find any active replica with the + // last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325 + log.warn("Failed to open an append stream to the log file. Opening a new log file..", e); + // Rollover the current log file (since cannot get a stream handle) and create new one + this.logFile = logFile.rollOver(fs); + createNewFile(); + } else if ((e.getClassName().contentEquals(AlreadyBeingCreatedException.class.getName()) || e.getClassName() + .contentEquals(RecoveryInProgressException.class.getName())) && (fs instanceof DistributedFileSystem)) { + // this happens when either another task executor writing to this file died or + // data node is going down. Note that we can only try to recover lease for a DistributedFileSystem. + // ViewFileSystem unfortunately does not support this operation + log.warn("Trying to recover log on path " + path); + if (FSUtils.recoverDFSFileLease((DistributedFileSystem) fs, path)) { + log.warn("Recovered lease on path " + path); + // try again + this.output = fs.append(path, bufferSize); + } else { + log.warn("Failed to recover lease on path " + path); + throw new HoodieException(e); + } + } else { + throw new HoodieIOException("Failed to open an append stream ", e); + } + } + } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java index 7ae620ef65c44..598025471d093 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -34,7 +34,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; @@ -49,8 +48,6 @@ public final class DiskBasedMap implements Map { private static final Logger log = LogManager.getLogger(DiskBasedMap.class); - // Default file path prefix to put the spillable file - private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; // Stores the key and corresponding value's latest metadata spilled to disk private final Map valueMetadataMap; // Key converter to convert key type to bytes @@ -70,17 +67,12 @@ public final class DiskBasedMap implements Map { private String filePath; - protected DiskBasedMap(Optional baseFilePath, + protected DiskBasedMap(String baseFilePath, Converter keyConverter, Converter valueConverter) throws IOException { this.valueMetadataMap = new HashMap<>(); - - if (!baseFilePath.isPresent()) { - baseFilePath = Optional.of(DEFAULT_BASE_FILE_PATH); - } - this.filePath = baseFilePath.get() + UUID.randomUUID().toString(); - File writeOnlyFileHandle = new File(filePath); + File writeOnlyFileHandle = new File(baseFilePath, UUID.randomUUID().toString()); + this.filePath = writeOnlyFileHandle.getPath(); initFile(writeOnlyFileHandle); - this.fileOutputStream = new FileOutputStream(writeOnlyFileHandle, true); this.writeOnlyFileHandle = new SizeAwareDataOutputStream(fileOutputStream); this.filePosition = new AtomicLong(0L); @@ -93,8 +85,10 @@ private void initFile(File writeOnlyFileHandle) throws IOException { if (writeOnlyFileHandle.exists()) { writeOnlyFileHandle.delete(); } + if (!writeOnlyFileHandle.getParentFile().exists()) { + writeOnlyFileHandle.getParentFile().mkdir(); + } writeOnlyFileHandle.createNewFile(); - log.info( "Spilling to file location " + writeOnlyFileHandle.getAbsolutePath() + " in host (" + InetAddress.getLocalHost() .getHostAddress() + ") with hostname (" + InetAddress.getLocalHost().getHostName() + ")"); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index b3ff515c04457..e061bf44453e5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -25,7 +25,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Map; -import java.util.Optional; import java.util.Set; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -64,7 +63,7 @@ public class ExternalSpillableMap implements Map { // Flag to determine whether to stop re-estimating payload size private boolean shouldEstimatePayloadSize = true; - public ExternalSpillableMap(Long maxInMemorySizeInBytes, Optional baseFilePath, + public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath, Converter keyConverter, Converter valueConverter) throws IOException { this.inMemoryMap = new HashMap<>(); this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 93eed42798956..192e6a51d2d7d 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -77,6 +77,7 @@ public class HoodieLogFormatTest { private Path partitionPath; private static String basePath; private int bufferSize = 4096; + private static final String BASE_OUTPUT_PATH = "/tmp/"; private Boolean readBlocksLazily = true; @@ -401,7 +402,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect // scan all log blocks (across multiple log files) HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", - 10240L, readBlocksLazily, false, bufferSize); + 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); List scannedRecords = new ArrayList<>(); for (HoodieRecord record : scanner) { @@ -527,7 +528,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false, bufferSize); + "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -587,7 +588,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "102", 10240L, readBlocksLazily, false, bufferSize); + "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -665,7 +666,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "103", 10240L, true, false, bufferSize); + "103", 10240L, true, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -719,7 +720,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "102", 10240L, readBlocksLazily, false, bufferSize); + "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -739,7 +740,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect readKeys.clear(); scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, - false, bufferSize); + false, bufferSize, BASE_OUTPUT_PATH); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); } @@ -800,7 +801,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect // all data must be rolled back before merge HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false, bufferSize); + "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(); @@ -849,7 +850,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false, bufferSize); + "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -881,7 +882,7 @@ public void testAvroLogRecordReaderWithInvalidRollback() "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false, bufferSize); + "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -931,7 +932,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "101", 10240L, readBlocksLazily, false, bufferSize); + "101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -1019,7 +1020,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, - "101", 10240L, readBlocksLazily, false, bufferSize); + "101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index ee2c77b5ee41b..fdf6ba7cdb7a2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -51,12 +51,14 @@ 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<>(Optional.empty(), + DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); @@ -78,7 +80,7 @@ public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISynta Schema schema = getSimpleSchema(); String payloadClazz = HoodieAvroPayload.class.getName(); - DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), + DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List hoodieRecords = SchemaTestUtil .generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); @@ -105,7 +107,7 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); - DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), + DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -191,9 +193,7 @@ public void testSizeEstimator() throws IOException, URISyntaxException { } /** - * vb - Disabled this test after talking to Nishanth as this relies on timing and sometimes fails in my laptop. - * This specific test sometime takes more than 100 ms (In one case, saw 122 ms), - * @na: TODO: Please check if this can be removed. + * @na: Leaving this test here for a quick performance test */ @Ignore @Test diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index bb385dc1a098d..419cd1b41e339 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.util.SpillableMapTestUtils; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; import com.uber.hoodie.common.util.collection.converter.StringConverter; +import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; import java.net.URISyntaxException; @@ -40,6 +41,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.junit.BeforeClass; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -48,13 +50,22 @@ public class TestExternalSpillableMap { private static final String FAILURE_OUTPUT_PATH = "/tmp/test_fail"; + private static final String BASE_OUTPUT_PATH = "/tmp/"; + + @BeforeClass + public static void cleanUp() { + File file = new File(BASE_OUTPUT_PATH); + file.delete(); + file = new File(FAILURE_OUTPUT_PATH); + file.delete(); + } @Test public void simpleInsertTest() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, Optional.empty(), new StringConverter(), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -76,7 +87,7 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -114,7 +125,7 @@ public void testAllMapOperations() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, Optional.empty(), new StringConverter(), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -169,7 +180,7 @@ public void simpleTestWithException() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -188,7 +199,7 @@ public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOExcept String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); @@ -241,7 +252,7 @@ public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISy String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index f96060b214554..163cbe8f85df6 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -83,6 +83,10 @@ public class HoodieRealtimeRecordReader implements RecordReader the commit we are trying to read (if using // readCommit() API) From 6e96be679c3a6ce5dc42d681efbb529a6e34e8ae Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 26 Apr 2018 10:18:05 -0700 Subject: [PATCH 053/374] Support union mode in HoodieRealtimeRecordReader for pure insert workloads Also Replace BufferedIteratorPayload abstraction with function passing --- .../cli/commands/HoodieLogFileCommand.java | 4 +- .../uber/hoodie/func/BufferedIterator.java | 209 ----------- .../hoodie/func/BufferedIteratorExecutor.java | 89 ----- .../uber/hoodie/func/LazyInsertIterable.java | 111 +++--- .../hoodie/func/ParquetReaderIterator.java | 3 +- .../func/SparkBoundedInMemoryExecutor.java | 57 +++ .../AbstractBufferedIteratorPayload.java | 42 --- .../HoodieRecordBufferedIteratorPayload.java | 47 --- .../uber/hoodie/io/HoodieCreateHandle.java | 8 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 5 +- .../compact/HoodieRealtimeTableCompactor.java | 6 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 73 ++-- ....java => TestBoundedInMemoryExecutor.java} | 65 ++-- .../hoodie/func/TestBoundedInMemoryQueue.java | 336 ++++++++++++++++++ .../hoodie/func/TestBufferedIterator.java | 203 ----------- ...va => AbstractHoodieLogRecordScanner.java} | 179 +++++----- .../log/HoodieMergedLogRecordScanner.java | 131 +++++++ .../log/HoodieUnMergedLogRecordScanner.java | 55 +++ .../table/log/block/HoodieLogBlock.java | 3 +- .../common/util/DefaultSizeEstimator.java | 18 +- .../util/HoodieRecordSizeEstimator.java | 52 +++ .../hoodie/common/util/SizeEstimator.java | 31 ++ .../hoodie/common/util/SpillableMapUtils.java | 5 +- .../util/collection/ExternalSpillableMap.java | 12 +- .../util/collection/converter/Converter.java | 5 - .../converter/HoodieRecordConverter.java | 13 - .../collection/converter/StringConverter.java | 6 - .../util/queue/BoundedInMemoryExecutor.java | 162 +++++++++ .../util/queue/BoundedInMemoryQueue.java | 273 ++++++++++++++ .../queue/BoundedInMemoryQueueConsumer.java | 63 ++++ .../queue/BoundedInMemoryQueueProducer.java | 35 ++ .../queue/FunctionBasedQueueProducer.java | 46 +++ .../queue/IteratorBasedQueueProducer.java | 49 +++ .../common/table/log/HoodieLogFormatTest.java | 31 +- .../util/collection/TestDiskBasedMap.java | 15 +- .../collection/TestExternalSpillableMap.java | 20 +- .../hadoop/RecordReaderValueIterator.java | 83 +++++ .../SafeParquetRecordReaderWrapper.java | 91 +++++ .../AbstractRealtimeRecordReader.java | 282 +++++++++++++++ .../realtime/HoodieRealtimeRecordReader.java | 320 ++--------------- .../RealtimeCompactedRecordReader.java | 129 +++++++ .../RealtimeUnmergedRecordReader.java | 142 ++++++++ .../hadoop/TestRecordReaderValueIterator.java | 105 ++++++ .../HoodieRealtimeRecordReaderTest.java | 102 +++++- 44 files changed, 2541 insertions(+), 1175 deletions(-) delete mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java delete mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java delete mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java delete mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java rename hoodie-client/src/test/java/com/uber/hoodie/func/{TestBufferedIteratorExecutor.java => TestBoundedInMemoryExecutor.java} (55%) create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java delete mode 100644 hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java rename hoodie-common/src/main/java/com/uber/hoodie/common/table/log/{HoodieCompactedLogRecordScanner.java => AbstractHoodieLogRecordScanner.java} (71%) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java rename hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java => hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java (55%) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java create mode 100644 hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java create mode 100644 hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java create mode 100644 hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java create mode 100644 hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java create mode 100644 hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java create mode 100644 hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 7115b67a2918f..8533d7afb2aee 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -24,8 +24,8 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; -import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; import com.uber.hoodie.common.table.log.HoodieLogFormat; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; @@ -187,7 +187,7 @@ public String showLogFileRecords(@CliOption(key = { if (shouldMerge) { System.out.println("===========================> MERGING RECORDS <==================="); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema, HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get() .getTimestamp(), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java deleted file mode 100644 index 58616408491da..0000000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java +++ /dev/null @@ -1,209 +0,0 @@ -/* - * 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.func; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.uber.hoodie.exception.HoodieException; -import java.util.Iterator; -import java.util.Optional; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.util.SizeEstimator; - -/** - * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It - * internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in - * buffer accordingly. This is done to ensure that we don't OOM. - * - * @param input payload data type - * @param output payload data type - */ -public class BufferedIterator implements Iterator { - - // interval used for polling records in the queue. - public static final int RECORD_POLL_INTERVAL_SEC = 5; - // rate used for sampling records to determine avg record size in bytes. - public static final int RECORD_SAMPLING_RATE = 64; - // maximum records that will be cached - private static final int RECORD_CACHING_LIMIT = 128 * 1024; - private static Logger logger = LogManager.getLogger(BufferedIterator.class); - // It indicates number of records to cache. We will be using sampled record's average size to - // determine how many - // records we should cache and will change (increase/decrease) permits accordingly. - @VisibleForTesting - public final Semaphore rateLimiter = new Semaphore(1); - // used for sampling records with "RECORD_SAMPLING_RATE" frequency. - public final AtomicLong samplingRecordCounter = new AtomicLong(-1); - // internal buffer to cache buffered records. - private final LinkedBlockingQueue> buffer = new - LinkedBlockingQueue<>(); - // maximum amount of memory to be used for buffering records. - private final long bufferMemoryLimit; - // original iterator from where records are read for buffering. - private final Iterator inputIterator; - // it holds the root cause of the exception in case either buffering records (reading from - // inputIterator) fails or - // thread reading records from buffer fails. - private final AtomicReference hasFailed = new AtomicReference(null); - // used for indicating that all the records from buffer are read successfully. - private final AtomicBoolean isDone = new AtomicBoolean(false); - // indicates rate limit (number of records to cache). it is updated whenever there is a change - // in avg record size. - @VisibleForTesting - public int currentRateLimit = 1; - // indicates avg record size in bytes. It is updated whenever a new record is sampled. - @VisibleForTesting - public long avgRecordSizeInBytes = 0; - // indicates number of samples collected so far. - private long numSamples = 0; - // next record to be read from buffer. - private O nextRecord; - // Function to transform the input payload to the expected output payload - private Function bufferedIteratorTransform; - - public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, - final Function bufferedIteratorTransform) { - this.inputIterator = iterator; - this.bufferMemoryLimit = bufferMemoryLimit; - this.bufferedIteratorTransform = bufferedIteratorTransform; - } - - @VisibleForTesting - public int size() { - return this.buffer.size(); - } - - // It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in - // bytes. It is used - // for determining how many maximum records to buffer. Based on change in avg size it may - // increase or decrease - // available permits. - private void adjustBufferSizeIfNeeded(final I record) throws InterruptedException { - if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { - return; - } - final long recordSizeInBytes = SizeEstimator.estimate(record); - final long newAvgRecordSizeInBytes = Math - .max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); - final int newRateLimit = (int) Math - .min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes)); - - // If there is any change in number of records to cache then we will either release (if it increased) or acquire - // (if it decreased) to adjust rate limiting to newly computed value. - if (newRateLimit > currentRateLimit) { - rateLimiter.release(newRateLimit - currentRateLimit); - } else if (newRateLimit < currentRateLimit) { - rateLimiter.acquire(currentRateLimit - newRateLimit); - } - currentRateLimit = newRateLimit; - avgRecordSizeInBytes = newAvgRecordSizeInBytes; - numSamples++; - } - - // inserts record into internal buffer. It also fetches insert value from the record to offload - // computation work on to - // buffering thread. - private void insertRecord(I t) throws Exception { - rateLimiter.acquire(); - adjustBufferSizeIfNeeded(t); - // We are retrieving insert value in the record buffering thread to offload computation - // around schema validation - // and record creation to it. - final O payload = bufferedIteratorTransform.apply(t); - buffer.put(Optional.of(payload)); - } - - private void readNextRecord() { - rateLimiter.release(); - Optional newRecord; - while (true) { - try { - throwExceptionIfFailed(); - newRecord = buffer.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS); - if (newRecord != null) { - break; - } - } catch (InterruptedException e) { - logger.error("error reading records from BufferedIterator", e); - throw new HoodieException(e); - } - } - if (newRecord.isPresent()) { - this.nextRecord = newRecord.get(); - } else { - // We are done reading all the records from internal iterator. - this.isDone.set(true); - this.nextRecord = null; - } - } - - public void startBuffering() throws Exception { - logger.info("starting to buffer records"); - try { - while (inputIterator.hasNext()) { - // We need to stop buffering if buffer-reader has failed and exited. - throwExceptionIfFailed(); - insertRecord(inputIterator.next()); - } - // done buffering records notifying buffer-reader. - buffer.put(Optional.empty()); - } catch (Exception e) { - logger.error("error buffering records", e); - // Used for notifying buffer-reader thread of the failed operation. - markAsFailed(e); - throw e; - } - logger.info("finished buffering records"); - } - - @Override - public boolean hasNext() { - if (this.nextRecord == null && !this.isDone.get()) { - readNextRecord(); - } - return !this.isDone.get(); - } - - @Override - public O next() { - Preconditions.checkState(hasNext() && this.nextRecord != null); - final O ret = this.nextRecord; - this.nextRecord = null; - return ret; - } - - private void throwExceptionIfFailed() { - if (this.hasFailed.get() != null) { - throw new HoodieException("operation has failed", this.hasFailed.get()); - } - } - - public void markAsFailed(Exception e) { - this.hasFailed.set(e); - // release the permits so that if the buffering thread is waiting for permits then it will - // get it. - this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); - } -} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java deleted file mode 100644 index fea0f28bbcce2..0000000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.func; - -import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieException; -import java.util.Iterator; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.function.Function; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; - -/** - * Executor for a BufferedIterator operation. This class takes as input the input iterator which - * needs to be buffered, the runnable function that needs to be executed in the reader thread and - * return the transformed output based on the writer function - */ -public class BufferedIteratorExecutor { - - private static Logger logger = LogManager.getLogger(BufferedIteratorExecutor.class); - - // Executor service used for launching writer thread. - final ExecutorService writerService; - // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - final BufferedIterator bufferedIterator; - // Need to set current spark thread's TaskContext into newly launched thread so that new - // thread can access - // TaskContext properties. - final TaskContext sparkThreadTaskContext; - - public BufferedIteratorExecutor(final HoodieWriteConfig hoodieConfig, final Iterator inputItr, - final Function bufferedIteratorTransform, - final ExecutorService writerService) { - this.sparkThreadTaskContext = TaskContext.get(); - this.writerService = writerService; - this.bufferedIterator = new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(), - bufferedIteratorTransform); - } - - /** - * Starts buffering and executing the writer function - */ - public Future start(Function writerFunction) { - try { - Future future = writerService.submit( - () -> { - logger.info("starting hoodie writer thread"); - // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext - // properties. - TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); - try { - E result = writerFunction.apply(bufferedIterator); - logger.info("hoodie write is done; notifying reader thread"); - return result; - } catch (Exception e) { - logger.error("error writing hoodie records", e); - bufferedIterator.markAsFailed(e); - throw e; - } - }); - bufferedIterator.startBuffering(); - return future; - } catch (Exception e) { - throw new HoodieException(e); - } - } - - public boolean isRemaining() { - return bufferedIterator.hasNext(); - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index 36aba53054e6f..6fdf41c3c514b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -19,27 +19,25 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; -import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload; import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.spark.TaskContext; +import scala.Tuple2; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -52,7 +50,6 @@ public class LazyInsertIterable extends private final String commitTime; private final HoodieTable hoodieTable; private Set partitionsCleaned; - private HoodieCreateHandle handle; public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { @@ -63,57 +60,68 @@ public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWrite this.hoodieTable = hoodieTable; } - @Override - protected void start() { - } - /** * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some * expensive operations of transformation to the reader thread. - * @param schema - * @param - * @return */ - public static Function, AbstractBufferedIteratorPayload> - bufferedItrPayloadTransform(Schema schema) { - return (hoodieRecord) -> new HoodieRecordBufferedIteratorPayload(hoodieRecord, schema); + static Function, + Tuple2, Optional>> getTransformFunction(Schema schema) { + return hoodieRecord -> { + try { + return new Tuple2, Optional>(hoodieRecord, + hoodieRecord.getData().getInsertValue(schema)); + } catch (IOException e) { + throw new HoodieException(e); + } + }; + } + + @Override + protected void start() { } @Override protected List computeNext() { // Executor service used for launching writer thread. - final ExecutorService writerService = Executors.newFixedThreadPool(1); + BoundedInMemoryExecutor, + Tuple2, Optional>, List> bufferedIteratorExecutor = null; try { - Function> function = (bufferedIterator) -> { - List statuses = new LinkedList<>(); - statuses.addAll(handleWrite(bufferedIterator)); - return statuses; - }; - BufferedIteratorExecutor, AbstractBufferedIteratorPayload, List> - bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieConfig, inputItr, - bufferedItrPayloadTransform(HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)), - writerService); - Future> writerResult = bufferedIteratorExecutor.start(function); - final List result = writerResult.get(); + final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); + bufferedIteratorExecutor = + new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, + new InsertHandler(), getTransformFunction(schema)); + final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); return result; } catch (Exception e) { throw new HoodieException(e); } finally { - writerService.shutdownNow(); + if (null != bufferedIteratorExecutor) { + bufferedIteratorExecutor.shutdownNow(); + } } } - private List handleWrite( - final BufferedIterator, AbstractBufferedIteratorPayload> bufferedIterator) { - List statuses = new ArrayList<>(); - while (bufferedIterator.hasNext()) { - final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator - .next(); - final HoodieRecord insertPayload = (HoodieRecord) payload.getInputPayload(); + @Override + protected void end() { + + } + + /** + * Consumes stream of hoodie records from in-memory queue and + * writes to one or more create-handles + */ + private class InsertHandler extends + BoundedInMemoryQueueConsumer, Optional>, List> { + + private final List statuses = new ArrayList<>(); + private HoodieCreateHandle handle; + + @Override + protected void consumeOneRecord(Tuple2, Optional> payload) { + final HoodieRecord insertPayload = payload._1(); // clean up any partial failures - if (!partitionsCleaned - .contains(insertPayload.getPartitionPath())) { + if (!partitionsCleaned.contains(insertPayload.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with // the same data again. Thus, before we open any files under a given partition, we // first delete any files in the same partitionPath written by same Spark partition @@ -127,33 +135,30 @@ private List handleWrite( handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); } - if (handle.canWrite(((HoodieRecord) payload.getInputPayload()))) { + if (handle.canWrite(payload._1())) { // write the payload, if the handle has capacity - handle.write(insertPayload, (Optional) payload.getOutputPayload(), payload.exception); + handle.write(insertPayload, payload._2()); } else { // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); - handle.write(insertPayload, - (Optional) payload.getOutputPayload(), - payload.exception); // we should be able to write 1 payload. + handle.write(insertPayload, payload._2()); // we should be able to write 1 payload. } } - // If we exited out, because we ran out of records, just close the pending handle. - if (!bufferedIterator.hasNext()) { + @Override + protected void finish() { if (handle != null) { statuses.add(handle.close()); } + handle = null; + assert statuses.size() > 0; } - assert statuses.size() > 0 && !bufferedIterator.hasNext(); // should never return empty statuses - return statuses; - } - - @Override - protected void end() { - + @Override + protected List getResult() { + return statuses; + } } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java index ed81da0359c32..2ccc743209fa4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java @@ -16,6 +16,7 @@ package com.uber.hoodie.func; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Iterator; @@ -23,7 +24,7 @@ /** * This class wraps a parquet reader and provides an iterator based api to - * read from a parquet file. This is used in {@link BufferedIterator} + * read from a parquet file. This is used in {@link BoundedInMemoryQueue} */ public class ParquetReaderIterator implements Iterator { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java b/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java new file mode 100644 index 0000000000000..76e35827f613d --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java @@ -0,0 +1,57 @@ +/* + * 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.func; + +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; +import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; +import com.uber.hoodie.config.HoodieWriteConfig; +import java.util.Iterator; +import java.util.Optional; +import java.util.function.Function; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; + +public class SparkBoundedInMemoryExecutor extends BoundedInMemoryExecutor { + + // Need to set current spark thread's TaskContext into newly launched thread so that new thread can access + // TaskContext properties. + final TaskContext sparkThreadTaskContext; + + public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator inputItr, + BoundedInMemoryQueueConsumer consumer, + Function bufferedIteratorTransform) { + this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform); + } + + public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, + BoundedInMemoryQueueProducer producer, + BoundedInMemoryQueueConsumer consumer, + Function bufferedIteratorTransform) { + super(hoodieConfig.getWriteBufferLimitBytes(), producer, + Optional.of(consumer), bufferedIteratorTransform); + this.sparkThreadTaskContext = TaskContext.get(); + } + + public void preExecute() { + // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties. + TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java deleted file mode 100644 index 4a7e322243535..0000000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.func.payload; - -/** - * @param Input data type for BufferedIterator - * @param Output data type for BufferedIterator - */ -public abstract class AbstractBufferedIteratorPayload { - - // input payload for iterator - protected I inputPayload; - // output payload for iterator, this is used in cases where the output payload is computed - // from the input payload and most of this computation is off-loaded to the reader - protected O outputPayload; - - public AbstractBufferedIteratorPayload(I record) { - this.inputPayload = record; - } - - public I getInputPayload() { - return inputPayload; - } - - public O getOutputPayload() { - return outputPayload; - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java deleted file mode 100644 index a79f0e01ab499..0000000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.func.payload; - -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieRecordPayload; -import java.util.Optional; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; - -/** - * BufferedIteratorPayload that takes HoodieRecord as input and transforms to output Optional - * @param - */ -public class HoodieRecordBufferedIteratorPayload - extends AbstractBufferedIteratorPayload, Optional> { - - // It caches the exception seen while fetching insert value. - public Optional exception = Optional.empty(); - - public HoodieRecordBufferedIteratorPayload(HoodieRecord record, Schema schema) { - super(record); - try { - this.outputPayload = record.getData().getInsertValue(schema); - } catch (Exception e) { - this.exception = Optional.of(e); - } - } - - public Optional getException() { - return exception; - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index c2ee5f8c0ed70..0384db71c6808 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -90,15 +90,9 @@ public boolean canWrite(HoodieRecord record) { /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional insertValue, - Optional getInsertValueException) { + public void write(HoodieRecord record, Optional avroRecord) { Optional recordMetadata = record.getData().getMetadata(); try { - // throws exception if there was any exception while fetching insert value - if (getInsertValueException.isPresent()) { - throw getInsertValueException.get(); - } - Optional avroRecord = insertValue; if (avroRecord.isPresent()) { storageWriter.writeAvroWithMetadata(avroRecord.get(), record); // update the new location of record, so we know where to find it next diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 8e28b8d986336..679d5fa8e1cc1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -24,7 +24,9 @@ import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.util.DefaultSizeEstimator; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; @@ -143,7 +145,8 @@ private String init(String fileId, Iterator> newRecordsItr) { logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), config.getSpillableMapBasePath(), new StringConverter(), - new HoodieRecordConverter(schema, config.getPayloadClass())); + new HoodieRecordConverter(schema, config.getPayloadClass()), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index cbf06d9366f5a..e3d00f75e4a34 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -28,7 +28,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; -import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -115,7 +115,7 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con .filterCompletedInstants().lastInstant().get().getTimestamp(); log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), @@ -131,7 +131,7 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { - s.getStat().setTotalUpdatedRecordsCompacted(scanner.getTotalRecordsToUpdate()); + s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog()); s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); s.getStat().setPartitionPath(operation.getPartitionPath()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index bdf40f79ab086..ab194c4f52b7a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -33,17 +33,16 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieUpsertException; -import com.uber.hoodie.func.BufferedIterator; -import com.uber.hoodie.func.BufferedIteratorExecutor; import com.uber.hoodie.func.LazyInsertIterable; import com.uber.hoodie.func.ParquetReaderIterator; -import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; -import com.uber.hoodie.func.payload.GenericRecordBufferedIteratorPayload; +import com.uber.hoodie.func.SparkBoundedInMemoryExecutor; import com.uber.hoodie.io.HoodieCleanHelper; import com.uber.hoodie.io.HoodieMergeHandle; import java.io.IOException; @@ -58,9 +57,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -182,16 +178,6 @@ public Iterator> handleUpdate(String commitTime, String fileLo return handleUpdateInternal(upsertHandle, commitTime, fileLoc); } - /** - * Transformer function to help transform a GenericRecord. This transformer is used by BufferedIterator to offload - * some expensive operations of transformation to the reader thread. - * - */ - public static java.util.function.Function - bufferedItrPayloadTransform() { - return (genericRecord) -> new GenericRecordBufferedIteratorPayload(genericRecord); - } - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc) throws IOException { @@ -202,23 +188,19 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) .withConf(getHadoopConf()).build(); - final ExecutorService writerService = Executors.newFixedThreadPool(1); + BoundedInMemoryExecutor wrapper = null; try { - java.util.function.Function runnableFunction = (bufferedIterator) -> { - handleWrite(bufferedIterator, upsertHandle); - return null; - }; - BufferedIteratorExecutor wrapper = - new BufferedIteratorExecutor(config, new ParquetReaderIterator(reader), bufferedItrPayloadTransform(), - writerService); - Future writerResult = wrapper.start(runnableFunction); - writerResult.get(); + wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader), + new UpdateHandler(upsertHandle), x -> x); + wrapper.execute(); } catch (Exception e) { throw new HoodieException(e); } finally { reader.close(); upsertHandle.close(); - writerService.shutdownNow(); + if (null != wrapper) { + wrapper.shutdownNow(); + } } } @@ -231,15 +213,6 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups .iterator(); } - private void handleWrite(final BufferedIterator bufferedIterator, - final HoodieMergeHandle upsertHandle) { - while (bufferedIterator.hasNext()) { - final GenericRecordBufferedIteratorPayload payload = (GenericRecordBufferedIteratorPayload) bufferedIterator - .next(); - upsertHandle.write(payload.getOutputPayload()); - } - } - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, Iterator> recordItr) { return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); @@ -493,6 +466,32 @@ enum BucketType { UPDATE, INSERT } + /** + * Consumer that dequeues records from queue and sends to Merge Handle + */ + private static class UpdateHandler extends BoundedInMemoryQueueConsumer { + + private final HoodieMergeHandle upsertHandle; + + private UpdateHandler(HoodieMergeHandle upsertHandle) { + this.upsertHandle = upsertHandle; + } + + @Override + protected void consumeOneRecord(GenericRecord record) { + upsertHandle.write(record); + } + + @Override + protected void finish() { + } + + @Override + protected Void getResult() { + return null; + } + } + private static class PartitionCleanStat implements Serializable { private final String partitionPath; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java similarity index 55% rename from hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java rename to hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java index 5c3695b8b1831..385afaabfadc9 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java @@ -16,39 +16,35 @@ package com.uber.hoodie.func; +import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.function.Function; +import java.util.Optional; +import org.apache.avro.generic.IndexedRecord; import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import scala.Tuple2; -public class TestBufferedIteratorExecutor { +public class TestBoundedInMemoryExecutor { private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); - private ExecutorService executorService = null; - - @Before - public void beforeTest() { - this.executorService = Executors.newFixedThreadPool(1); - } + private SparkBoundedInMemoryExecutor>, Integer> executor = null; @After public void afterTest() { - if (this.executorService != null) { - this.executorService.shutdownNow(); - this.executorService = null; + if (this.executor != null) { + this.executor.shutdownNow(); + this.executor = null; } } @@ -59,21 +55,32 @@ public void testExecutor() throws Exception { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); - BufferedIteratorExecutor bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieWriteConfig, - hoodieRecords.iterator(), LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema), - executorService); - Function function = (bufferedIterator) -> { - Integer count = 0; - while (bufferedIterator.hasNext()) { - count++; - bufferedIterator.next(); - } - return count; - }; - Future future = bufferedIteratorExecutor.start(function); + BoundedInMemoryQueueConsumer>, Integer> consumer = + new BoundedInMemoryQueueConsumer>, Integer>() { + + private int count = 0; + + @Override + protected void consumeOneRecord(Tuple2> record) { + count++; + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return count; + } + }; + + executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, + hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + int result = executor.execute(); // It should buffer and write 100 records - Assert.assertEquals((int) future.get(), 100); + Assert.assertEquals(result, 100); // There should be no remaining records in the buffer - Assert.assertFalse(bufferedIteratorExecutor.isRemaining()); + Assert.assertFalse(executor.isRemaining()); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java new file mode 100644 index 0000000000000..b555b81672470 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java @@ -0,0 +1,336 @@ +/* + * 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.func; + +import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.SizeEstimator; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; +import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer; +import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; +import com.uber.hoodie.exception.HoodieException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import scala.Tuple2; + +public class TestBoundedInMemoryQueue { + + private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); + private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); + private ExecutorService executorService = null; + + @Before + public void beforeTest() { + this.executorService = Executors.newFixedThreadPool(2); + } + + @After + public void afterTest() { + if (this.executorService != null) { + this.executorService.shutdownNow(); + this.executorService = null; + } + } + + // Test to ensure that we are reading all records from queue iterator in the same order + // without any exceptions. + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testRecordReading() throws Exception { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final BoundedInMemoryQueue>> queue = new BoundedInMemoryQueue(FileUtils.ONE_KB, + getTransformFunction(HoodieTestDataGenerator.avroSchema)); + // Produce + Future resFuture = + executorService.submit(() -> { + new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); + queue.close(); + return true; + }); + final Iterator originalRecordIterator = hoodieRecords.iterator(); + int recordsRead = 0; + while (queue.iterator().hasNext()) { + final HoodieRecord originalRecord = originalRecordIterator.next(); + final Optional originalInsertValue = originalRecord.getData() + .getInsertValue(HoodieTestDataGenerator.avroSchema); + final Tuple2> payload = queue.iterator().next(); + // Ensure that record ordering is guaranteed. + Assert.assertEquals(originalRecord, payload._1()); + // cached insert value matches the expected insert value. + Assert.assertEquals(originalInsertValue, + payload._1().getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); + recordsRead++; + } + Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext()); + // all the records should be read successfully. + Assert.assertEquals(numRecords, recordsRead); + // should not throw any exceptions. + resFuture.get(); + } + + /** + * Test to ensure that we are reading all records from queue iterator when we have multiple producers + */ + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testCompositeProducerRecordReading() throws Exception { + final int numRecords = 1000; + final int numProducers = 40; + final List> recs = new ArrayList<>(); + + final BoundedInMemoryQueue>> queue = + new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Record Key to + Map> keyToProducerAndIndexMap = new HashMap<>(); + + for (int i = 0; i < numProducers; i++) { + List pRecs = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + int j = 0; + for (HoodieRecord r : pRecs) { + Assert.assertTrue(!keyToProducerAndIndexMap.containsKey(r.getRecordKey())); + keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j)); + j++; + } + recs.add(pRecs); + } + + List> producers = new ArrayList<>(); + for (int i = 0; i < recs.size(); i++) { + final List r = recs.get(i); + // Alternate between pull and push based iterators + if (i % 2 == 0) { + producers.add(new IteratorBasedQueueProducer<>(r.iterator())); + } else { + producers.add(new FunctionBasedQueueProducer((buf) -> { + Iterator itr = r.iterator(); + while (itr.hasNext()) { + try { + buf.insertRecord(itr.next()); + } catch (Exception e) { + throw new HoodieException(e); + } + } + return true; + })); + } + } + + final List> futureList = producers.stream().map(producer -> { + return executorService.submit(() -> { + producer.produce(queue); + return true; + }); + }).collect(Collectors.toList()); + + // Close queue + Future closeFuture = executorService.submit(() -> { + try { + for (Future f : futureList) { + f.get(); + } + queue.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return true; + }); + + // Used to ensure that consumer sees the records generated by a single producer in FIFO order + Map lastSeenMap = IntStream.range(0, numProducers).boxed() + .collect(Collectors.toMap(Function.identity(), x -> -1)); + Map countMap = IntStream.range(0, numProducers).boxed() + .collect(Collectors.toMap(Function.identity(), x -> 0)); + + // Read recs and ensure we have covered all producer recs. + while (queue.iterator().hasNext()) { + final Tuple2> payload = queue.iterator().next(); + final HoodieRecord rec = payload._1(); + Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); + Integer lastSeenPos = lastSeenMap.get(producerPos._1()); + countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1); + lastSeenMap.put(producerPos._1(), lastSeenPos + 1); + // Ensure we are seeing the next record generated + Assert.assertEquals(lastSeenPos + 1, producerPos._2().intValue()); + } + + for (int i = 0; i < numProducers; i++) { + // Ensure we have seen all the records for each producers + Assert.assertEquals(Integer.valueOf(numRecords), countMap.get(i)); + } + + //Ensure Close future is done + closeFuture.get(); + } + + // Test to ensure that record queueing is throttled when we hit memory limit. + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testMemoryLimitForBuffering() throws Exception { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // maximum number of records to keep in memory. + final int recordLimit = 5; + final SizeEstimator>> sizeEstimator = + new DefaultSizeEstimator<>(); + final long objSize = sizeEstimator.sizeEstimate( + getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0))); + final long memoryLimitInBytes = recordLimit * objSize; + final BoundedInMemoryQueue>> queue = + new BoundedInMemoryQueue(memoryLimitInBytes, + getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Produce + Future resFuture = executorService.submit(() -> { + new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); + return true; + }); + // waiting for permits to expire. + while (!isQueueFull(queue.rateLimiter)) { + Thread.sleep(10); + } + Assert.assertEquals(0, queue.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, queue.currentRateLimit); + Assert.assertEquals(recordLimit, queue.size()); + Assert.assertEquals(recordLimit - 1, queue.samplingRecordCounter.get()); + + // try to read 2 records. + Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next()._1()); + Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next()._1()); + + // waiting for permits to expire. + while (!isQueueFull(queue.rateLimiter)) { + Thread.sleep(10); + } + // No change is expected in rate limit or number of queued records. We only expect + // queueing thread to read + // 2 more records into the queue. + Assert.assertEquals(0, queue.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, queue.currentRateLimit); + Assert.assertEquals(recordLimit, queue.size()); + Assert.assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get()); + } + + // Test to ensure that exception in either queueing thread or BufferedIterator-reader thread + // is propagated to + // another thread. + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testException() throws Exception { + final int numRecords = 256; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final SizeEstimator>> sizeEstimator = + new DefaultSizeEstimator<>(); + // queue memory limit + final long objSize = sizeEstimator.sizeEstimate( + getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0))); + final long memoryLimitInBytes = 4 * objSize; + + // first let us throw exception from queueIterator reader and test that queueing thread + // stops and throws + // correct exception back. + BoundedInMemoryQueue>> queue1 = + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Produce + Future resFuture = executorService.submit(() -> { + new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue1); + return true; + }); + + // waiting for permits to expire. + while (!isQueueFull(queue1.rateLimiter)) { + Thread.sleep(10); + } + // notify queueing thread of an exception and ensure that it exits. + final Exception e = new Exception("Failing it :)"); + queue1.markAsFailed(e); + try { + resFuture.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e1) { + Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); + Assert.assertEquals(e, e1.getCause().getCause()); + } + + // second let us raise an exception while doing record queueing. this exception should get + // propagated to + // queue iterator reader. + final RuntimeException expectedException = new RuntimeException("failing record reading"); + final Iterator mockHoodieRecordsIterator = mock(Iterator.class); + when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); + when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); + BoundedInMemoryQueue>> queue2 = + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Produce + Future res = executorService.submit(() -> { + try { + new IteratorBasedQueueProducer<>(mockHoodieRecordsIterator).produce(queue2); + } catch (Exception ex) { + queue2.markAsFailed(ex); + throw ex; + } + return true; + }); + + try { + queue2.iterator().hasNext(); + Assert.fail("exception is expected"); + } catch (Exception e1) { + Assert.assertEquals(expectedException, e1.getCause()); + } + // queueing thread should also have exited. make sure that it is not running. + try { + res.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e2) { + Assert.assertEquals(expectedException, e2.getCause()); + } + } + + private boolean isQueueFull(Semaphore rateLimiter) { + return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java deleted file mode 100644 index e55db1b05f856..0000000000000 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * 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.func; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import com.uber.hoodie.common.HoodieTestDataGenerator; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; -import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; -import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.Semaphore; -import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.io.FileUtils; -import org.apache.spark.util.SizeEstimator; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class TestBufferedIterator { - - private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); - private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); - private ExecutorService recordReader = null; - - @Before - public void beforeTest() { - this.recordReader = Executors.newFixedThreadPool(1); - } - - @After - public void afterTest() { - if (this.recordReader != null) { - this.recordReader.shutdownNow(); - this.recordReader = null; - } - } - - // Test to ensure that we are reading all records from buffered iterator in the same order - // without any exceptions. - @Test(timeout = 60000) - public void testRecordReading() throws IOException, ExecutionException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result = recordReader.submit(() -> { - bufferedIterator.startBuffering(); - return true; - }); - final Iterator originalRecordIterator = hoodieRecords.iterator(); - int recordsRead = 0; - while (bufferedIterator.hasNext()) { - final HoodieRecord originalRecord = originalRecordIterator.next(); - final Optional originalInsertValue = originalRecord.getData() - .getInsertValue(HoodieTestDataGenerator.avroSchema); - final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator.next(); - // Ensure that record ordering is guaranteed. - Assert.assertEquals(originalRecord, payload.getInputPayload()); - // cached insert value matches the expected insert value. - Assert.assertEquals(originalInsertValue, - ((HoodieRecord) payload.getInputPayload()).getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); - recordsRead++; - } - Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); - // all the records should be read successfully. - Assert.assertEquals(numRecords, recordsRead); - // should not throw any exceptions. - Assert.assertTrue(result.get()); - } - - // Test to ensure that record buffering is throttled when we hit memory limit. - @Test(timeout = 60000) - public void testMemoryLimitForBuffering() throws IOException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // maximum number of records to keep in memory. - final int recordLimit = 5; - final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); - final BufferedIterator bufferedIterator = - new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result = recordReader.submit(() -> { - bufferedIterator.startBuffering(); - return true; - }); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); - - // try to read 2 records. - Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().getInputPayload()); - Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().getInputPayload()); - - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - // No change is expected in rate limit or number of buffered records. We only expect - // buffering thread to read - // 2 more records into the buffer. - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); - } - - // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread - // is propagated to - // another thread. - @Test(timeout = 60000) - public void testException() throws IOException, InterruptedException { - final int numRecords = 256; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // buffer memory limit - final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); - - // first let us throw exception from bufferIterator reader and test that buffering thread - // stops and throws - // correct exception back. - BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result = recordReader.submit(() -> { - bufferedIterator1.startBuffering(); - return true; - }); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator1.rateLimiter)) { - Thread.sleep(10); - } - // notify buffering thread of an exception and ensure that it exits. - final Exception e = new Exception("Failing it :)"); - bufferedIterator1.markAsFailed(e); - try { - result.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e1) { - Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); - Assert.assertEquals(e, e1.getCause().getCause()); - } - - // second let us raise an exception while doing record buffering. this exception should get - // propagated to - // buffered iterator reader. - final RuntimeException expectedException = new RuntimeException("failing record reading"); - final Iterator mockHoodieRecordsIterator = mock(Iterator.class); - when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); - when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); - BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result2 = recordReader.submit(() -> { - bufferedIterator2.startBuffering(); - return true; - }); - try { - bufferedIterator2.hasNext(); - Assert.fail("exception is expected"); - } catch (Exception e1) { - Assert.assertEquals(expectedException, e1.getCause()); - } - // buffering thread should also have exited. make sure that it is not running. - try { - result2.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e2) { - Assert.assertEquals(expectedException, e2.getCause()); - } - } - - private boolean isQueueFull(Semaphore rateLimiter) { - return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); - } -} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java similarity index 71% rename from hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index 86da05b737fab..dd79fff921dc8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -19,7 +19,6 @@ import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; -import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -29,19 +28,14 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.util.HoodieTimer; import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.exception.HoodieIOException; -import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; import java.util.Deque; -import java.util.Iterator; +import java.util.HashSet; import java.util.List; -import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.avro.Schema; @@ -53,24 +47,38 @@ import org.apache.log4j.Logger; /** - * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will - * be used as a lookup table when merging the base columnar file with the redo log file. NOTE: If readBlockLazily is + * Implements logic to scan log blocks and expose valid and deleted log records to subclass implementation. + * Subclass is free to either apply merging or expose raw data back to the caller. + * + * NOTE: If readBlockLazily is * turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to * avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and * forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2 * Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | | * | Read Block N Metadata | | Read Block N Data |

This results in two I/O passes over the log file. */ +public abstract class AbstractHoodieLogRecordScanner { -public class HoodieCompactedLogRecordScanner implements - Iterable> { - - private static final Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); + private static final Logger log = LogManager.getLogger(AbstractHoodieLogRecordScanner.class); - // Final map of compacted/merged records - private final ExternalSpillableMap> records; // Reader schema for the records private final Schema readerSchema; + // Latest valid instant time + private final String latestInstantTime; + private final HoodieTableMetaClient hoodieTableMetaClient; + // Merge strategy to use when combining records from log + private final String payloadClassFQN; + // Log File Paths + private final List logFilePaths; + // Read Lazily flag + private final boolean readBlocksLazily; + // Reverse reader - Not implemented yet (NA -> Why do we need ?) + // but present here for plumbing for future implementation + private final boolean reverseReader; + // Buffer Size for log file reader + private final int bufferSize; + // FileSystem + private final FileSystem fs; // Total log files read - for metrics private AtomicLong totalLogFiles = new AtomicLong(0); // Total log blocks read - for metrics @@ -81,46 +89,47 @@ public class HoodieCompactedLogRecordScanner implements private AtomicLong totalRollbacks = new AtomicLong(0); // Total number of corrupt blocks written across all log files private AtomicLong totalCorruptBlocks = new AtomicLong(0); - // Total final list of compacted/merged records - private long totalRecordsToUpdate; - // Latest valid instant time - private String latestInstantTime; - private HoodieTableMetaClient hoodieTableMetaClient; - // Merge strategy to use when combining records from log - private String payloadClassFQN; // Store the last instant log blocks (needed to implement rollback) private Deque currentInstantLogBlocks = new ArrayDeque<>(); - // Stores the total time taken to perform reading and merging of log blocks - private long totalTimeTakenToReadAndMergeBlocks = 0L; - // A timer for calculating elapsed time in millis - public HoodieTimer timer = new HoodieTimer(); - public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, - boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) { + // Progress + private float progress = 0.0f; + + public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, + boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); // load class from the payload fully qualified class name this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); this.totalLogFiles.addAndGet(logFilePaths.size()); - timer.startTimer(); + this.logFilePaths = logFilePaths; + this.readBlocksLazily = readBlocksLazily; + this.reverseReader = reverseReader; + this.fs = fs; + this.bufferSize = bufferSize; + } + /** + * Scan Log files + */ + public void scan() { try { - // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize - this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, - new StringConverter(), new HoodieRecordConverter(readerSchema, payloadClassFQN)); // iterate over the paths HoodieLogFormatReader logFormatReaderWrapper = new HoodieLogFormatReader(fs, logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))) .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize); - HoodieLogFile logFile; + Set scannedLogFiles = new HashSet<>(); while (logFormatReaderWrapper.hasNext()) { - logFile = logFormatReaderWrapper.getLogFile(); + HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); log.info("Scanning log file " + logFile); + scannedLogFiles.add(logFile); + totalLogFiles.set(scannedLogFiles.size()); // Use the HoodieLogFileReader to iterate through the blocks in the log file HoodieLogBlock r = logFormatReaderWrapper.next(); + totalLogBlocks.incrementAndGet(); if (r.getBlockType() != CORRUPT_BLOCK && !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME), this.latestInstantTime, @@ -134,7 +143,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List " + maxMemorySizeInBytes); - log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); - log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); - log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); - log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); - log.debug("Total time taken for scanning and compacting log files => " + totalTimeTakenToReadAndMergeBlocks); } /** @@ -250,66 +253,69 @@ private boolean isNewInstantBlock(HoodieLogBlock logBlock) { } /** - * Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge with the application - * specific payload if the same key was found before. Sufficient to just merge the log records since the base data is - * merged on previous compaction. Finally, merge this log block with the accumulated records + * Iterate over the GenericRecord in the block, read the hoodie key and partition path and + * call subclass processors to handle it. */ - private Map> merge( - HoodieAvroDataBlock dataBlock) throws IOException { - // TODO (NA) - Implemnt getRecordItr() in HoodieAvroDataBlock and use that here + private void processAvroDataBlock(HoodieAvroDataBlock dataBlock) throws Exception { + // TODO (NA) - Implement getRecordItr() in HoodieAvroDataBlock and use that here List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); - recs.forEach(rec -> { - String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD) - .toString(); + for (IndexedRecord rec : recs) { HoodieRecord hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); - if (records.containsKey(key)) { - // Merge and store the merged record - HoodieRecordPayload combinedValue = records.get(key).getData() - .preCombine(hoodieRecord.getData()); - records - .put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), - combinedValue)); - } else { - // Put the record as is - records.put(key, hoodieRecord); - } - }); - return records; + processNextRecord(hoodieRecord); + } } /** - * Merge the last seen log blocks with the accumulated records + * Process next record + * + * @param hoodieRecord Hoodie Record to process */ - private void merge(Map> records, - Deque lastBlocks) throws IOException { + protected abstract void processNextRecord(HoodieRecord hoodieRecord) + throws Exception; + + /** + * Process next deleted key + * + * @param key Deleted record key + */ + protected abstract void processNextDeletedKey(String key); + + /** + * Process the set of log blocks belonging to the last instant which is read fully. + */ + private void processQueuedBlocksForInstant(Deque lastBlocks, int numLogFilesSeen) + throws Exception { while (!lastBlocks.isEmpty()) { log.info("Number of remaining logblocks to merge " + lastBlocks.size()); // poll the element at the bottom of the stack since that's the order it was inserted HoodieLogBlock lastBlock = lastBlocks.pollLast(); switch (lastBlock.getBlockType()) { case AVRO_DATA_BLOCK: - merge((HoodieAvroDataBlock) lastBlock); + processAvroDataBlock((HoodieAvroDataBlock) lastBlock); break; case DELETE_BLOCK: // TODO : If delete is the only block written and/or records are present in parquet file // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry - Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove); + Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey); break; case CORRUPT_BLOCK: log.warn("Found a corrupt block which was not rolled back"); break; default: - //TODO : Need to understand if COMMAND_BLOCK has to be handled? break; } } + // At this step the lastBlocks are consumed. We track approximate progress by number of log-files seen + progress = numLogFilesSeen - 1 / logFilePaths.size(); } - @Override - public Iterator> iterator() { - return records.iterator(); + /** + * Return progress of scanning as a float between 0.0 to 1.0 + */ + public float getProgress() { + return progress; } public long getTotalLogFiles() { @@ -324,12 +330,8 @@ public long getTotalLogBlocks() { return totalLogBlocks.get(); } - public Map> getRecords() { - return records; - } - - public long getTotalRecordsToUpdate() { - return totalRecordsToUpdate; + protected String getPayloadClassFQN() { + return payloadClassFQN; } public long getTotalRollbacks() { @@ -339,9 +341,4 @@ public long getTotalRollbacks() { public long getTotalCorruptBlocks() { return totalCorruptBlocks.get(); } - - public long getTotalTimeTakenToReadAndMergeBlocks() { - return totalTimeTakenToReadAndMergeBlocks; - } } - diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java new file mode 100644 index 0000000000000..ed976606958ef --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java @@ -0,0 +1,131 @@ +/* + * 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.table.log; + +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.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; +import com.uber.hoodie.common.util.HoodieTimer; +import com.uber.hoodie.common.util.collection.ExternalSpillableMap; +import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will + * be used as a lookup table when merging the base columnar file with the redo log file. + * + * NOTE: If readBlockLazily is + * turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to + * avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and + * forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2 + * Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | | + * | Read Block N Metadata | | Read Block N Data |

This results in two I/O passes over the log file. + */ + +public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner + implements Iterable> { + + private static final Logger log = LogManager.getLogger(HoodieMergedLogRecordScanner.class); + + // Final map of compacted/merged records + private final ExternalSpillableMap> records; + + // count of merged records in log + private long numMergedRecordsInLog; + + // Stores the total time taken to perform reading and merging of log blocks + private final long totalTimeTakenToReadAndMergeBlocks; + // A timer for calculating elapsed time in millis + public final HoodieTimer timer = new HoodieTimer(); + + @SuppressWarnings("unchecked") + public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, + boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize); + try { + // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize + this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, + new StringConverter(), new HoodieRecordConverter(readerSchema, getPayloadClassFQN()), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema)); + // Do the scan and merge + timer.startTimer(); + scan(); + this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); + this.numMergedRecordsInLog = records.size(); + log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); + log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records + .getInMemoryMapNumEntries()); + log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records + .getCurrentInMemoryMapSize()); + log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records + .getDiskBasedMapNumEntries()); + log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); + } catch (IOException e) { + throw new HoodieIOException("IOException when reading log file "); + } + } + + @Override + public Iterator> iterator() { + return records.iterator(); + } + + public Map> getRecords() { + return records; + } + + public long getNumMergedRecordsInLog() { + return numMergedRecordsInLog; + } + + @Override + protected void processNextRecord(HoodieRecord hoodieRecord) { + String key = hoodieRecord.getRecordKey(); + if (records.containsKey(key)) { + // Merge and store the merged record + HoodieRecordPayload combinedValue = records.get(key).getData().preCombine(hoodieRecord.getData()); + records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); + } else { + // Put the record as is + records.put(key, hoodieRecord); + } + } + + @Override + protected void processNextDeletedKey(String key) { + // TODO : If delete is the only block written and/or records are present in parquet file + // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry + records.remove(key); + } + + public long getTotalTimeTakenToReadAndMergeBlocks() { + return totalTimeTakenToReadAndMergeBlocks; + } +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java new file mode 100644 index 0000000000000..98264352b8a94 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -0,0 +1,55 @@ +/* + * 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.common.table.log; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import java.util.List; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; + +public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordScanner { + + private final LogRecordScannerCallback callback; + + public HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, + List logFilePaths, Schema readerSchema, String latestInstantTime, + boolean readBlocksLazily, boolean reverseReader, int bufferSize, + LogRecordScannerCallback callback) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize); + this.callback = callback; + } + + @Override + protected void processNextRecord(HoodieRecord hoodieRecord) throws Exception { + // Just call callback without merging + callback.apply(hoodieRecord); + } + + @Override + protected void processNextDeletedKey(String key) { + throw new IllegalStateException("Not expected to see delete records in this log-scan mode. Check Job Config"); + } + + @FunctionalInterface + public static interface LogRecordScannerCallback { + + public void apply(HoodieRecord record) throws Exception; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index 86b3f26984840..17475155eff2b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -18,6 +18,7 @@ import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import java.io.ByteArrayOutputStream; @@ -219,7 +220,7 @@ public static Map getLogMetadata(DataInputStream dis /** * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in - * {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner} + * {@link HoodieMergedLogRecordScanner} */ public static byte[] readOrSkipContent(FSDataInputStream inputStream, Integer contentLength, boolean readBlockLazily) throws IOException { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java similarity index 55% rename from hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java index 9d934d9ea7a8f..91145d08baade 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * 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. @@ -14,18 +14,18 @@ * limitations under the License. */ -package com.uber.hoodie.func.payload; +package com.uber.hoodie.common.util; -import org.apache.avro.generic.GenericRecord; +import com.twitter.common.objectsize.ObjectSizeCalculator; /** - * BufferedIteratorPayload that takes GenericRecord as input and GenericRecord as output + * Default implementation of size-estimator that uses Twitter's ObjectSizeCalculator + * @param */ -public class GenericRecordBufferedIteratorPayload - extends AbstractBufferedIteratorPayload { +public class DefaultSizeEstimator implements SizeEstimator { - public GenericRecordBufferedIteratorPayload(GenericRecord record) { - super(record); - this.outputPayload = record; + @Override + public long sizeEstimate(T t) { + return ObjectSizeCalculator.getObjectSize(t); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java new file mode 100644 index 0000000000000..72bcc4fcdcaba --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java @@ -0,0 +1,52 @@ +/* + * 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; + +import com.twitter.common.objectsize.ObjectSizeCalculator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import org.apache.avro.Schema; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Size Estimator for Hoodie record payload + * @param + */ +public class HoodieRecordSizeEstimator implements SizeEstimator> { + + private static Logger log = LogManager.getLogger(HoodieRecordSizeEstimator.class); + + // Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa + private final Schema schema; + + public HoodieRecordSizeEstimator(Schema schema) { + this.schema = schema; + } + + @Override + public long sizeEstimate(HoodieRecord hoodieRecord) { + // Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst + // all records in the JVM. Calculate and print the size of the Schema and of the Record to + // note the sizes and differences. A correct estimation in such cases is handled in + /** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/ + long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord); + long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema); + log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); + return sizeOfRecord; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java new file mode 100644 index 0000000000000..6a6f597bff845 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java @@ -0,0 +1,31 @@ +/* + * 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.common.util; + +/** + * An interface to estimate the size of payload in memory + * @param + */ +public interface SizeEstimator { + + /** + * This method is used to estimate the size of a payload in memory. + * The default implementation returns the total allocated size, in bytes, of the object + * and all other objects reachable from it + */ + long sizeEstimate(T t); +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index 94aa1758da444..79aa90ee65be9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -20,7 +20,6 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.collection.DiskBasedMap; -import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieCorruptedDataException; import java.io.IOException; @@ -99,8 +98,8 @@ public static long generateChecksum(byte[] data) { * Compute a bytes representation of the payload by serializing the contents This is used to estimate the size of the * payload (either in memory or when written to disk) */ - public static long computePayloadSize(R value, Converter valueConverter) throws IOException { - return valueConverter.sizeEstimate(value); + public static long computePayloadSize(R value, SizeEstimator valueSizeEstimator) throws IOException { + return valueSizeEstimator.sizeEstimate(value); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index e061bf44453e5..081a889e75a77 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -17,6 +17,7 @@ package com.uber.hoodie.common.util.collection; import com.twitter.common.objectsize.ObjectSizeCalculator; +import com.uber.hoodie.common.util.SizeEstimator; import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieNotSupportedException; import java.io.IOException; @@ -56,6 +57,10 @@ public class ExternalSpillableMap implements Map { private final Converter keyConverter; // Value converter to convert value type to bytes private final Converter valueConverter; + // Size Estimator for key type + private final SizeEstimator keySizeEstimator; + // Size Estimator for key types + private final SizeEstimator valueSizeEstimator; // current space occupied by this map in-memory private Long currentInMemoryMapSize; // An estimate of the size of each payload written to this map @@ -64,7 +69,8 @@ public class ExternalSpillableMap implements Map { private boolean shouldEstimatePayloadSize = true; public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath, - Converter keyConverter, Converter valueConverter) throws IOException { + Converter keyConverter, Converter valueConverter, + SizeEstimator keySizeEstimator, SizeEstimator valueSizeEstimator) throws IOException { this.inMemoryMap = new HashMap<>(); this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter); this.maxInMemorySizeInBytes = (long) Math @@ -72,6 +78,8 @@ public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath, this.currentInMemoryMapSize = 0L; this.keyConverter = keyConverter; this.valueConverter = valueConverter; + this.keySizeEstimator = keySizeEstimator; + this.valueSizeEstimator = valueSizeEstimator; } /** @@ -146,7 +154,7 @@ public R put(T key, R value) { // At first, use the sizeEstimate of a record being inserted into the spillable map. // Note, the converter may over estimate the size of a record in the JVM this.estimatedPayloadSize = - keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value); + keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value); log.info("Estimated Payload size => " + estimatedPayloadSize); } else if (shouldEstimatePayloadSize && inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java index b06651973cb5e..55168baa92589 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java @@ -31,9 +31,4 @@ public interface Converter { * This method is used to convert the serialized payload (in bytes) to the actual payload instance */ T getData(byte[] bytes); - - /** - * This method is used to estimate the size of a payload in memory - */ - long sizeEstimate(T t); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java index 88bf7fcd06065..ee6c90d9c64f2 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.util.collection.converter; -import com.twitter.common.objectsize.ObjectSizeCalculator; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -87,16 +86,4 @@ public HoodieRecord getData(byte[] bytes) { throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io); } } - - @Override - public long sizeEstimate(HoodieRecord hoodieRecord) { - // Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst - // all records in the JVM. Calculate and print the size of the Schema and of the Record to - // note the sizes and differences. A correct estimation in such cases is handled in - /** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/ - long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord); - long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema); - log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); - return sizeOfRecord; - } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java index ea84a7dd9467a..7855484dbec7a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.util.collection.converter; -import com.twitter.common.objectsize.ObjectSizeCalculator; import java.nio.charset.StandardCharsets; /** @@ -33,9 +32,4 @@ public byte[] getBytes(String s) { public String getData(byte[] bytes) { return new String(bytes); } - - @Override - public long sizeEstimate(String s) { - return ObjectSizeCalculator.getObjectSize(s); - } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java new file mode 100644 index 0000000000000..1e7664f481c08 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java @@ -0,0 +1,162 @@ +/* + * 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.common.util.queue; + +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.SizeEstimator; +import com.uber.hoodie.exception.HoodieException; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.concurrent.ConcurrentUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Executor which orchestrates concurrent producers and consumers communicating through a bounded in-memory queue. + * This class takes as input the size limit, queue producer(s), consumer and transformer + * and exposes API to orchestrate concurrent execution of these actors communicating through a central bounded queue + */ +public class BoundedInMemoryExecutor { + + private static Logger logger = LogManager.getLogger(BoundedInMemoryExecutor.class); + + // Executor service used for launching writer thread. + private final ExecutorService executorService; + // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. + private final BoundedInMemoryQueue queue; + // Producers + private final List> producers; + // Consumer + private final Optional> consumer; + + public BoundedInMemoryExecutor(final long bufferLimitInBytes, + BoundedInMemoryQueueProducer producer, + Optional> consumer, + final Function transformFunction) { + this(bufferLimitInBytes, Arrays.asList(producer), consumer, transformFunction, new DefaultSizeEstimator<>()); + } + + public BoundedInMemoryExecutor(final long bufferLimitInBytes, + List> producers, + Optional> consumer, + final Function transformFunction, + final SizeEstimator sizeEstimator) { + this.producers = producers; + this.consumer = consumer; + // Ensure single thread for each producer thread and one for consumer + this.executorService = Executors.newFixedThreadPool(producers.size() + 1); + this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); + } + + /** + * Callback to implement environment specific behavior before executors (producers/consumer) + * run. + */ + public void preExecute() { + // Do Nothing in general context + } + + /** + * Start all Producers + */ + public ExecutorCompletionService startProducers() { + // Latch to control when and which producer thread will close the queue + final CountDownLatch latch = new CountDownLatch(producers.size()); + final ExecutorCompletionService completionService = + new ExecutorCompletionService(executorService); + producers.stream().map(producer -> { + return completionService.submit(() -> { + try { + preExecute(); + producer.produce(queue); + } catch (Exception e) { + logger.error("error consuming records", e); + queue.markAsFailed(e); + throw e; + } finally { + synchronized (latch) { + latch.countDown(); + if (latch.getCount() == 0) { + // Mark production as done so that consumer will be able to exit + queue.close(); + } + } + } + return true; + }); + }).collect(Collectors.toList()); + return completionService; + } + + /** + * Start only consumer + */ + private Future startConsumer() { + return consumer.map(consumer -> { + return executorService.submit( + () -> { + logger.info("starting consumer thread"); + preExecute(); + try { + E result = consumer.consume(queue); + logger.info("Queue Consumption is done; notifying producer threads"); + return result; + } catch (Exception e) { + logger.error("error consuming records", e); + queue.markAsFailed(e); + throw e; + } + }); + }).orElse(ConcurrentUtils.constantFuture(null)); + } + + /** + * Main API to run both production and consumption + */ + public E execute() { + try { + ExecutorCompletionService producerService = startProducers(); + Future future = startConsumer(); + // Wait for consumer to be done + return future.get(); + } catch (Exception e) { + throw new HoodieException(e); + } + } + + + public boolean isRemaining() { + return queue.iterator().hasNext(); + } + + public void shutdownNow() { + executorService.shutdownNow(); + } + + public BoundedInMemoryQueue getQueue() { + return queue; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java new file mode 100644 index 0000000000000..401924e96f244 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java @@ -0,0 +1,273 @@ +/* + * 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.common.util.queue; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.SizeEstimator; +import com.uber.hoodie.exception.HoodieException; +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Used for enqueueing input records. Queue limit is controlled by {@link #memoryLimit}. + * Unlike standard bounded queue implementations, this queue bounds the size by memory bytes occupied by its + * tenants. The standard implementation bounds by the number of entries in the queue. + * + * It internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in + * queue accordingly. This is done to ensure that we don't OOM. + * + * This queue supports multiple producer single consumer pattern. + * + * @param input payload data type + * @param output payload data type + */ +public class BoundedInMemoryQueue implements Iterable { + + // interval used for polling records in the queue. + public static final int RECORD_POLL_INTERVAL_SEC = 1; + // rate used for sampling records to determine avg record size in bytes. + public static final int RECORD_SAMPLING_RATE = 64; + // maximum records that will be cached + private static final int RECORD_CACHING_LIMIT = 128 * 1024; + private static Logger logger = LogManager.getLogger(BoundedInMemoryQueue.class); + // It indicates number of records to cache. We will be using sampled record's average size to + // determine how many + // records we should cache and will change (increase/decrease) permits accordingly. + @VisibleForTesting + public final Semaphore rateLimiter = new Semaphore(1); + // used for sampling records with "RECORD_SAMPLING_RATE" frequency. + public final AtomicLong samplingRecordCounter = new AtomicLong(-1); + // internal queue for records. + private final LinkedBlockingQueue> queue = new + LinkedBlockingQueue<>(); + // maximum amount of memory to be used for queueing records. + private final long memoryLimit; + // it holds the root cause of the exception in case either queueing records (consuming from + // inputIterator) fails or + // thread reading records from queue fails. + private final AtomicReference hasFailed = new AtomicReference(null); + // used for indicating that all the records from queue are read successfully. + private final AtomicBoolean isReadDone = new AtomicBoolean(false); + // used for indicating that all records have been enqueued + private final AtomicBoolean isWriteDone = new AtomicBoolean(false); + // Function to transform the input payload to the expected output payload + private final Function transformFunction; + // Payload Size Estimator + private final SizeEstimator payloadSizeEstimator; + // Singleton (w.r.t this instance) Iterator for this queue + private final QueueIterator iterator; + // indicates rate limit (number of records to cache). it is updated whenever there is a change + // in avg record size. + @VisibleForTesting + public int currentRateLimit = 1; + // indicates avg record size in bytes. It is updated whenever a new record is sampled. + @VisibleForTesting + public long avgRecordSizeInBytes = 0; + // indicates number of samples collected so far. + private long numSamples = 0; + + /** + * Construct BoundedInMemoryQueue with default SizeEstimator + * + * @param memoryLimit MemoryLimit in bytes + * @param transformFunction Transformer Function to convert input payload type to stored payload type + */ + public BoundedInMemoryQueue(final long memoryLimit, final Function transformFunction) { + this(memoryLimit, transformFunction, new DefaultSizeEstimator() { + }); + } + + /** + * Construct BoundedInMemoryQueue with passed in size estimator + * + * @param memoryLimit MemoryLimit in bytes + * @param transformFunction Transformer Function to convert input payload type to stored payload type + * @param payloadSizeEstimator Payload Size Estimator + */ + public BoundedInMemoryQueue( + final long memoryLimit, + final Function transformFunction, + final SizeEstimator payloadSizeEstimator) { + this.memoryLimit = memoryLimit; + this.transformFunction = transformFunction; + this.payloadSizeEstimator = payloadSizeEstimator; + this.iterator = new QueueIterator(); + } + + @VisibleForTesting + public int size() { + return this.queue.size(); + } + + /** + * Samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in bytes. It is used + * for determining how many maximum records to queue. Based on change in avg size it ma increase or decrease + * available permits. + * + * @param payload Payload to size + */ + private void adjustBufferSizeIfNeeded(final O payload) throws InterruptedException { + if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { + return; + } + + final long recordSizeInBytes = payloadSizeEstimator.sizeEstimate(payload); + final long newAvgRecordSizeInBytes = Math + .max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); + final int newRateLimit = (int) Math + .min(RECORD_CACHING_LIMIT, Math.max(1, this.memoryLimit / newAvgRecordSizeInBytes)); + + // If there is any change in number of records to cache then we will either release (if it increased) or acquire + // (if it decreased) to adjust rate limiting to newly computed value. + if (newRateLimit > currentRateLimit) { + rateLimiter.release(newRateLimit - currentRateLimit); + } else if (newRateLimit < currentRateLimit) { + rateLimiter.acquire(currentRateLimit - newRateLimit); + } + currentRateLimit = newRateLimit; + avgRecordSizeInBytes = newAvgRecordSizeInBytes; + numSamples++; + } + + /** + * Inserts record into queue after applying transformation + * + * @param t Item to be queueed + */ + public void insertRecord(I t) throws Exception { + // If already closed, throw exception + if (isWriteDone.get()) { + throw new IllegalStateException("Queue closed for enqueueing new entries"); + } + + // We need to stop queueing if queue-reader has failed and exited. + throwExceptionIfFailed(); + + rateLimiter.acquire(); + // We are retrieving insert value in the record queueing thread to offload computation + // around schema validation + // and record creation to it. + final O payload = transformFunction.apply(t); + adjustBufferSizeIfNeeded(payload); + queue.put(Optional.of(payload)); + } + + /** + * Checks if records are either available in the queue or expected to be written in future + */ + private boolean expectMoreRecords() { + return !isWriteDone.get() || (isWriteDone.get() && !queue.isEmpty()); + } + + /** + * Reader interface but never exposed to outside world as this is a single consumer queue. + * Reading is done through a singleton iterator for this queue. + */ + private Optional readNextRecord() { + if (this.isReadDone.get()) { + return Optional.empty(); + } + + rateLimiter.release(); + Optional newRecord = Optional.empty(); + while (expectMoreRecords()) { + try { + throwExceptionIfFailed(); + newRecord = queue.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS); + if (newRecord != null) { + break; + } + } catch (InterruptedException e) { + logger.error("error reading records from queue", e); + throw new HoodieException(e); + } + } + if (newRecord != null && newRecord.isPresent()) { + return newRecord; + } else { + // We are done reading all the records from internal iterator. + this.isReadDone.set(true); + return Optional.empty(); + } + } + + /** + * Puts an empty entry to queue to denote termination + */ + public void close() throws InterruptedException { + // done queueing records notifying queue-reader. + isWriteDone.set(true); + } + + private void throwExceptionIfFailed() { + if (this.hasFailed.get() != null) { + throw new HoodieException("operation has failed", this.hasFailed.get()); + } + } + + /** + * API to allow producers and consumer to communicate termination due to failure + */ + public void markAsFailed(Exception e) { + this.hasFailed.set(e); + // release the permits so that if the queueing thread is waiting for permits then it will + // get it. + this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); + } + + @Override + public Iterator iterator() { + return iterator; + } + + /** + * Iterator for the memory bounded queue + */ + private final class QueueIterator implements Iterator { + + // next record to be read from queue. + private O nextRecord; + + @Override + public boolean hasNext() { + if (this.nextRecord == null) { + Optional res = readNextRecord(); + this.nextRecord = res.orElse(null); + } + return this.nextRecord != null; + } + + @Override + public O next() { + Preconditions.checkState(hasNext() && this.nextRecord != null); + final O ret = this.nextRecord; + this.nextRecord = null; + return ret; + } + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java new file mode 100644 index 0000000000000..e1f985c2c861b --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java @@ -0,0 +1,63 @@ +/* + * 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.common.util.queue; + +import java.util.Iterator; + + +/** + * Consume entries from queue and execute callback function + */ +public abstract class BoundedInMemoryQueueConsumer { + + /** + * API to de-queue entries to memory bounded queue + * + * @param queue In Memory bounded queue + */ + public O consume(BoundedInMemoryQueue queue) throws Exception { + Iterator iterator = queue.iterator(); + + while (iterator.hasNext()) { + consumeOneRecord(iterator.next()); + } + + // Notifies done + finish(); + + return getResult(); + } + + /** + * Consumer One record + */ + protected abstract void consumeOneRecord(I record); + + /** + * Notifies implementation that we have exhausted consuming records from queue + */ + protected abstract void finish(); + + /** + * Return result of consuming records so far + */ + protected abstract O getResult(); + + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java new file mode 100644 index 0000000000000..b7d9181581abb --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java @@ -0,0 +1,35 @@ +/* + * 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.common.util.queue; + +/** + * Producer for BoundedInMemoryQueue. Memory Bounded Buffer supports + * multiple producers single consumer pattern. + * + * @param Input type for buffer items produced + */ +public interface BoundedInMemoryQueueProducer { + + /** + * API to enqueue entries to memory bounded queue + * + * @param queue In Memory bounded queue + */ + void produce(BoundedInMemoryQueue queue) throws Exception; +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java new file mode 100644 index 0000000000000..3d4d0cee5fc92 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java @@ -0,0 +1,46 @@ +/* + * 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.common.util.queue; + +import java.util.function.Function; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Buffer producer which allows custom functions to insert entries to queue. + * + * @param Type of entry produced for queue + */ +public class FunctionBasedQueueProducer implements BoundedInMemoryQueueProducer { + + private static final Logger logger = LogManager.getLogger(FunctionBasedQueueProducer.class); + + private final Function, Boolean> producerFunction; + + public FunctionBasedQueueProducer(Function, Boolean> producerFunction) { + this.producerFunction = producerFunction; + } + + @Override + public void produce(BoundedInMemoryQueue queue) { + logger.info("starting function which will enqueue records"); + producerFunction.apply(queue); + logger.info("finished function which will enqueue records"); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java new file mode 100644 index 0000000000000..f8161199e72d6 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.common.util.queue; + +import java.util.Iterator; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Iterator based producer which pulls entry from iterator and produces items for the queue + * + * @param Item type produced for the buffer. + */ +public class IteratorBasedQueueProducer implements BoundedInMemoryQueueProducer { + + private static final Logger logger = LogManager.getLogger(IteratorBasedQueueProducer.class); + + // input iterator for producing items in the buffer. + private final Iterator inputIterator; + + public IteratorBasedQueueProducer(Iterator inputIterator) { + this.inputIterator = inputIterator; + } + + @Override + public void produce(BoundedInMemoryQueue queue) throws Exception { + logger.info("starting to buffer records"); + while (inputIterator.hasNext()) { + queue.insertRecord(inputIterator.next()); + } + logger.info("finished buffering records"); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 192e6a51d2d7d..dcc2c4996cc62 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -73,12 +73,11 @@ @RunWith(Parameterized.class) public class HoodieLogFormatTest { + private static final String BASE_OUTPUT_PATH = "/tmp/"; + private static String basePath; private FileSystem fs; private Path partitionPath; - private static String basePath; private int bufferSize = 4096; - private static final String BASE_OUTPUT_PATH = "/tmp/"; - private Boolean readBlocksLazily = true; public HoodieLogFormatTest(Boolean readBlocksLazily) { @@ -87,7 +86,7 @@ public HoodieLogFormatTest(Boolean readBlocksLazily) { @Parameterized.Parameters(name = "LogBlockReadMode") public static Collection data() { - return Arrays.asList(new Boolean[][] {{true}, {false}}); + return Arrays.asList(new Boolean[][]{{true}, {false}}); } @BeforeClass @@ -400,7 +399,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect writer.close(); // scan all log blocks (across multiple log files) - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); @@ -527,7 +526,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -587,7 +586,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -665,7 +664,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "103", 10240L, true, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -719,7 +718,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); @@ -739,8 +738,8 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect writer = writer.appendBlock(commandBlock); readKeys.clear(); - scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, - false, bufferSize, BASE_OUTPUT_PATH); + scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", + 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); } @@ -800,7 +799,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); // all data must be rolled back before merge - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); @@ -849,7 +848,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -881,7 +880,7 @@ public void testAvroLogRecordReaderWithInvalidRollback() List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); @@ -931,7 +930,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -1019,7 +1018,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback() List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index fdf6ba7cdb7a2..83d20bd2baaa7 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -27,6 +27,7 @@ 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; @@ -156,14 +157,14 @@ public void testSizeEstimator() throws IOException, URISyntaxException { List hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), - new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName())); + 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 HoodieRecordConverter(schema, HoodieAvroPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); // Following tests payloads without an Avro Schema in the Record @@ -175,7 +176,7 @@ public void testSizeEstimator() throws IOException, URISyntaxException { .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 HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); // Test sizeEstimator with hoodie metadata fields and without schema object in the payload @@ -188,7 +189,7 @@ public void testSizeEstimator() throws IOException, URISyntaxException { .of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata))))) .collect(Collectors.toList()); payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), - new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); } @@ -201,11 +202,11 @@ public void testSizeEstimatorPerformance() throws IOException, URISyntaxExceptio // Test sizeEstimatorPerformance with simpleSchema Schema schema = SchemaTestUtil.getSimpleSchema(); List hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); - HoodieRecordConverter converter = - new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()); + HoodieRecordSizeEstimator sizeEstimator = + new HoodieRecordSizeEstimator(schema); HoodieRecord record = hoodieRecords.remove(0); long startTime = System.currentTimeMillis(); - SpillableMapUtils.computePayloadSize(record, converter); + SpillableMapUtils.computePayloadSize(record, sizeEstimator); long timeTaken = System.currentTimeMillis() - startTime; System.out.println("Time taken :" + timeTaken); assertTrue(timeTaken < 100); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index 419cd1b41e339..7a5239f9d6993 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -25,7 +25,9 @@ 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.DefaultSizeEstimator; 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.collection.converter.HoodieRecordConverter; @@ -66,7 +68,8 @@ public void simpleInsertTest() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -88,7 +91,8 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -126,7 +130,8 @@ public void testAllMapOperations() throws IOException, URISyntaxException { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // insert a bunch of records so that values spill to disk too @@ -181,7 +186,8 @@ public void simpleTestWithException() throws IOException, URISyntaxException { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -200,7 +206,8 @@ public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOExcept ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk @@ -253,7 +260,8 @@ public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISy ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java new file mode 100644 index 0000000000000..59a83acef192d --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java @@ -0,0 +1,83 @@ +/* + * 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.hadoop; + +import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.RecordReader; + +/** + * Provides Iterator Interface to iterate value entries read from record reader + * + * @param Key Type + * @param Value Type + */ +public class RecordReaderValueIterator implements Iterator { + + public static final Log LOG = LogFactory.getLog(RecordReaderValueIterator.class); + + private final RecordReader reader; + private V nextVal = null; + + /** + * Construct RecordReaderValueIterator + * + * @param reader reader + */ + public RecordReaderValueIterator(RecordReader reader) { + this.reader = reader; + } + + @Override + public boolean hasNext() { + if (nextVal == null) { + K key = reader.createKey(); + V val = reader.createValue(); + try { + boolean notDone = reader.next(key, val); + if (!notDone) { + return false; + } + this.nextVal = val; + } catch (IOException e) { + LOG.error("Got error reading next record from record reader"); + throw new HoodieException(e); + } + } + return true; + } + + @Override + public V next() { + if (!hasNext()) { + throw new NoSuchElementException("Make sure you are following iterator contract."); + } + V retVal = this.nextVal; + this.nextVal = null; + return retVal; + } + + public void close() throws IOException { + this.reader.close(); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java new file mode 100644 index 0000000000000..7fb9d67d120d5 --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java @@ -0,0 +1,91 @@ +/* + * 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.hadoop; + +import java.io.IOException; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; + +/** + * Record Reader for parquet. Records read from this reader is safe to be + * buffered for concurrent processing. + * + * In concurrent producer/consumer pattern, where the record is read and buffered by one thread and processed in + * another thread, we need to ensure new instance of ArrayWritable is buffered. ParquetReader createKey/Value is unsafe + * as it gets reused for subsequent fetch. This wrapper makes ParquetReader safe for this use-case. + */ +public class SafeParquetRecordReaderWrapper implements RecordReader { + + // real Parquet reader to be wrapped + private final RecordReader parquetReader; + + // Value Class + private final Class valueClass; + + // Number of fields in Value Schema + private final int numValueFields; + + + public SafeParquetRecordReaderWrapper(RecordReader parquetReader) { + this.parquetReader = parquetReader; + ArrayWritable arrayWritable = parquetReader.createValue(); + this.valueClass = arrayWritable.getValueClass(); + this.numValueFields = arrayWritable.get().length; + } + + @Override + public boolean next(Void key, ArrayWritable value) throws IOException { + return parquetReader.next(key, value); + } + + @Override + public Void createKey() { + return parquetReader.createKey(); + } + + /** + * We could be in concurrent fetch and read env. + * We need to ensure new ArrayWritable as ParquetReader implementation reuses same + * ArrayWritable for all reads which will cause corruption when buffering. + * So, we create a new ArrayWritable here with Value class from parquetReader's value + * and an empty array. + */ + @Override + public ArrayWritable createValue() { + // Call createValue of parquetReader to get size and class type info only + Writable[] emptyWritableBuf = new Writable[numValueFields]; + return new ArrayWritable(valueClass, emptyWritableBuf); + } + + @Override + public long getPos() throws IOException { + return parquetReader.getPos(); + } + + @Override + public void close() throws IOException { + parquetReader.close(); + } + + @Override + public float getProgress() throws IOException { + return parquetReader.getProgress(); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java new file mode 100644 index 0000000000000..383ce0cd9832a --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -0,0 +1,282 @@ +/* + * 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.hadoop.realtime; + +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import parquet.avro.AvroSchemaConverter; +import parquet.hadoop.ParquetFileReader; +import parquet.schema.MessageType; + +/** + * Record Reader implementation to merge fresh avro data with base parquet data, to support real + * time queries. + */ +public abstract class AbstractRealtimeRecordReader { + + // Fraction of mapper/reducer task memory used for compaction of log files + public static final String COMPACTION_MEMORY_FRACTION_PROP = "compaction.memory.fraction"; + public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75"; + // used to choose a trade off between IO vs Memory when performing compaction process + // Depending on outputfile size and memory provided, choose true to avoid OOM for large file + // size + small memory + public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = + "compaction.lazy.block.read.enabled"; + public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true"; + + // Property to set the max memory for dfs inputstream buffer size + public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size"; + // Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper + public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB + // Property to set file path prefix for spillable file + public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path"; + // Default file path prefix for spillable file + public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/"; + + public static final Log LOG = LogFactory.getLog(AbstractRealtimeRecordReader.class); + protected final HoodieRealtimeFileSplit split; + protected final JobConf jobConf; + private final MessageType baseFileSchema; + + // Schema handles + private Schema readerSchema; + private Schema writerSchema; + + public AbstractRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job) { + this.split = split; + this.jobConf = job; + + LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); + try { + baseFileSchema = readSchema(jobConf, split.getPath()); + init(); + } catch (IOException e) { + throw new HoodieIOException( + "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); + } + } + + /** + * Reads the schema from the parquet file. This is different from ParquetUtils as it uses the + * twitter parquet to support hive 1.1.0 + */ + private static MessageType readSchema(Configuration conf, Path parquetFilePath) { + try { + return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema(); + } catch (IOException e) { + throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); + } + } + + protected static String arrayWritableToString(ArrayWritable writable) { + if (writable == null) { + return "null"; + } + + StringBuilder builder = new StringBuilder(); + Writable[] values = writable.get(); + builder.append(String.format("Size: %s,", values.length)); + for (Writable w : values) { + builder.append(w + " "); + } + return builder.toString(); + } + + /** + * Given a comma separated list of field names and positions at which they appear on Hive, return + * a ordered list of field names, that can be passed onto storage. + */ + public static List orderFields(String fieldNameCsv, String fieldOrderCsv, + String partitioningFieldsCsv) { + + String[] fieldOrders = fieldOrderCsv.split(","); + Set partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) + .collect(Collectors.toSet()); + List fieldNames = Arrays.stream(fieldNameCsv.split(",")) + .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); + + // Hive does not provide ids for partitioning fields, so check for lengths excluding that. + if (fieldNames.size() != fieldOrders.length) { + throw new HoodieException(String + .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", + fieldNames.size(), fieldOrders.length)); + } + TreeMap orderedFieldMap = new TreeMap<>(); + for (int ox = 0; ox < fieldOrders.length; ox++) { + orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox)); + } + return new ArrayList<>(orderedFieldMap.values()); + } + + /** + * Generate a reader schema off the provided writeSchema, to just project out the provided + * columns + */ + public static Schema generateProjectionSchema(Schema writeSchema, List fieldNames) { + List projectedFields = new ArrayList<>(); + for (String fn : fieldNames) { + Schema.Field field = writeSchema.getField(fn); + if (field == null) { + throw new HoodieException("Field " + fn + " not found log schema. Query cannot proceed!"); + } + projectedFields + .add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); + } + + return Schema.createRecord(projectedFields); + } + + /** + * Convert the projected read from delta record into an array writable + */ + public static Writable avroToArrayWritable(Object value, Schema schema) { + + // if value is null, make a NullWritable + if (value == null) { + return NullWritable.get(); + } + + switch (schema.getType()) { + case STRING: + return new Text(value.toString()); + case BYTES: + return new BytesWritable((byte[]) value); + case INT: + return new IntWritable((Integer) value); + case LONG: + return new LongWritable((Long) value); + case FLOAT: + return new FloatWritable((Float) value); + case DOUBLE: + return new DoubleWritable((Double) value); + case BOOLEAN: + return new BooleanWritable((Boolean) value); + case NULL: + return NullWritable.get(); + case RECORD: + GenericRecord record = (GenericRecord) value; + Writable[] values1 = new Writable[schema.getFields().size()]; + int index1 = 0; + for (Schema.Field field : schema.getFields()) { + values1[index1++] = avroToArrayWritable(record.get(field.name()), field.schema()); + } + return new ArrayWritable(Writable.class, values1); + case ENUM: + return new Text(value.toString()); + case ARRAY: + GenericArray arrayValue = (GenericArray) value; + Writable[] values2 = new Writable[arrayValue.size()]; + int index2 = 0; + for (Object obj : arrayValue) { + values2[index2++] = avroToArrayWritable(obj, schema.getElementType()); + } + return new ArrayWritable(Writable.class, values2); + case MAP: + Map mapValue = (Map) value; + Writable[] values3 = new Writable[mapValue.size()]; + int index3 = 0; + for (Object entry : mapValue.entrySet()) { + Map.Entry mapEntry = (Map.Entry) entry; + Writable[] mapValues = new Writable[2]; + mapValues[0] = new Text(mapEntry.getKey().toString()); + mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType()); + values3[index3++] = new ArrayWritable(Writable.class, mapValues); + } + return new ArrayWritable(Writable.class, values3); + case UNION: + List types = schema.getTypes(); + if (types.size() != 2) { + throw new IllegalArgumentException("Only support union with 2 fields"); + } + Schema s1 = types.get(0); + Schema s2 = types.get(1); + if (s1.getType() == Schema.Type.NULL) { + return avroToArrayWritable(value, s2); + } else if (s2.getType() == Schema.Type.NULL) { + return avroToArrayWritable(value, s1); + } else { + throw new IllegalArgumentException("Only support union with null"); + } + case FIXED: + return new BytesWritable(((GenericFixed) value).bytes()); + default: + return null; + } + } + + /** + * Goes through the log files and populates a map with latest version of each key logged, since + * the base split was written. + */ + private void init() throws IOException { + writerSchema = new AvroSchemaConverter().convert(baseFileSchema); + List projectionFields = orderFields( + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), + jobConf.get("partition_columns", "")); + // TODO(vc): In the future, the reader schema should be updated based on log files & be able + // to null out fields not present before + readerSchema = generateProjectionSchema(writerSchema, projectionFields); + + LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s", + split.getDeltaFilePaths(), split.getPath(), projectionFields)); + } + + public Schema getReaderSchema() { + return readerSchema; + } + + public Schema getWriterSchema() { + return writerSchema; + } + + public long getMaxCompactionMemoryInBytes() { + return (long) Math.ceil(Double + .valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) + * jobConf.getMemoryForMapTask()); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 163cbe8f85df6..88a5fdd1e9de6 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -18,339 +18,85 @@ package com.uber.hoodie.hadoop.realtime; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieRecordPayload; -import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; -import java.util.stream.Collectors; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericArray; -import org.apache.avro.generic.GenericFixed; -import org.apache.avro.generic.GenericRecord; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; -import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; -import parquet.avro.AvroSchemaConverter; -import parquet.hadoop.ParquetFileReader; -import parquet.schema.MessageType; /** - * Record Reader implementation to merge fresh avro data with base parquet data, to support real - * time queries. + * Realtime Record Reader which can do compacted (merge-on-read) record reading or + * unmerged reading (parquet and log files read in parallel) based on job configuration. */ public class HoodieRealtimeRecordReader implements RecordReader { - private final RecordReader parquetReader; - private final HoodieRealtimeFileSplit split; - private final JobConf jobConf; - - // Fraction of mapper/reducer task memory used for compaction of log files - public static final String COMPACTION_MEMORY_FRACTION_PROP = "compaction.memory.fraction"; - public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75"; - - // used to choose a trade off between IO vs Memory when performing compaction process - // Depending on outputfile size and memory provided, choose true to avoid OOM for large file - // size + small memory - public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = - "compaction.lazy.block.read.enabled"; - public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true"; - - // Property to set the max memory for dfs inputstream buffer size - public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size"; - // Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper - public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB - // Property to set file path prefix for spillable file - public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path"; - // Default file path prefix for spillable file - public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/"; - + // Property to enable parallel reading of parquet and log files without merging. + public static final String REALTIME_SKIP_MERGE_PROP = "hoodie.realtime.merge.skip"; + // By default, we do merged-reading + public static final String DEFAULT_REALTIME_SKIP_MERGE = "false"; public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class); - - private final HashMap deltaRecordMap; - private final MessageType baseFileSchema; + private final RecordReader reader; public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job, RecordReader realReader) { - this.split = split; - this.jobConf = job; - this.parquetReader = realReader; - this.deltaRecordMap = new HashMap<>(); - - LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); - try { - baseFileSchema = readSchema(jobConf, split.getPath()); - readAndCompactLog(jobConf); - } catch (IOException e) { - throw new HoodieIOException( - "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); - } - } - - /** - * Reads the schema from the parquet file. This is different from ParquetUtils as it uses the - * twitter parquet to support hive 1.1.0 - */ - private static MessageType readSchema(Configuration conf, Path parquetFilePath) { - try { - return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema(); - } catch (IOException e) { - throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); - } - } - - - /** - * Goes through the log files and populates a map with latest version of each key logged, since - * the base split was written. - */ - private void readAndCompactLog(JobConf jobConf) throws IOException { - Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema); - List projectionFields = orderFields( - jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), - jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), - jobConf.get("partition_columns", "")); - // TODO(vc): In the future, the reader schema should be updated based on log files & be able - // to null out fields not present before - Schema readerSchema = generateProjectionSchema(writerSchema, projectionFields); - - LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s", - split.getDeltaFilePaths(), split.getPath(), projectionFields)); - HoodieCompactedLogRecordScanner compactedLogRecordScanner = new HoodieCompactedLogRecordScanner( - FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), - split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime(), (long) Math.ceil(Double - .valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) - * jobConf.getMemoryForMapTask()), Boolean.valueOf(jobConf - .get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), - false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), - jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH)); - // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit - // but can return records for completed commits > the commit we are trying to read (if using - // readCommit() API) - for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { - GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema).get(); - String key = hoodieRecord.getRecordKey(); - // we assume, a later safe record in the log, is newer than what we have in the map & - // replace it. - // TODO : handle deletes here - ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema); - deltaRecordMap.put(key, aWritable); - if (LOG.isDebugEnabled()) { - LOG.debug("Log record : " + arrayWritableToString(aWritable)); - } - } - } - - private static String arrayWritableToString(ArrayWritable writable) { - if (writable == null) { - return "null"; - } - - StringBuilder builder = new StringBuilder(); - Writable[] values = writable.get(); - builder.append(String.format("Size: %s,", values.length)); - for (Writable w : values) { - builder.append(w + " "); - } - return builder.toString(); + this.reader = constructRecordReader(split, job, realReader); } - /** - * Given a comma separated list of field names and positions at which they appear on Hive, return - * a ordered list of field names, that can be passed onto storage. - */ - public static List orderFields(String fieldNameCsv, String fieldOrderCsv, - String partitioningFieldsCsv) { - - String[] fieldOrders = fieldOrderCsv.split(","); - Set partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) - .collect(Collectors.toSet()); - List fieldNames = Arrays.stream(fieldNameCsv.split(",")) - .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); - - // Hive does not provide ids for partitioning fields, so check for lengths excluding that. - if (fieldNames.size() != fieldOrders.length) { - throw new HoodieException(String - .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", - fieldNames.size(), fieldOrders.length)); - } - TreeMap orderedFieldMap = new TreeMap<>(); - for (int ox = 0; ox < fieldOrders.length; ox++) { - orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox)); - } - return new ArrayList<>(orderedFieldMap.values()); + public static boolean canSkipMerging(JobConf jobConf) { + return Boolean.valueOf(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE)); } /** - * Generate a reader schema off the provided writeSchema, to just project out the provided - * columns + * Construct record reader based on job configuration + * + * @param split File Split + * @param jobConf Job Configuration + * @param realReader Parquet Record Reader + * @return Realtime Reader */ - public static Schema generateProjectionSchema(Schema writeSchema, List fieldNames) { - List projectedFields = new ArrayList<>(); - for (String fn : fieldNames) { - Schema.Field field = writeSchema.getField(fn); - if (field == null) { - throw new HoodieException("Field " + fn + " not found log schema. Query cannot proceed!"); + private static RecordReader constructRecordReader(HoodieRealtimeFileSplit split, + JobConf jobConf, RecordReader realReader) { + try { + if (canSkipMerging(jobConf)) { + LOG.info("Enabling un-merged reading of realtime records"); + return new RealtimeUnmergedRecordReader(split, jobConf, realReader); } - projectedFields - .add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); - } - - return Schema.createRecord(projectedFields); - } - - /** - * Convert the projected read from delta record into an array writable - */ - public static Writable avroToArrayWritable(Object value, Schema schema) { - - // if value is null, make a NullWritable - if (value == null) { - return NullWritable.get(); - } - - switch (schema.getType()) { - case STRING: - return new Text(value.toString()); - case BYTES: - return new BytesWritable((byte[]) value); - case INT: - return new IntWritable((Integer) value); - case LONG: - return new LongWritable((Long) value); - case FLOAT: - return new FloatWritable((Float) value); - case DOUBLE: - return new DoubleWritable((Double) value); - case BOOLEAN: - return new BooleanWritable((Boolean) value); - case NULL: - return NullWritable.get(); - case RECORD: - GenericRecord record = (GenericRecord) value; - Writable[] values1 = new Writable[schema.getFields().size()]; - int index1 = 0; - for (Schema.Field field : schema.getFields()) { - values1[index1++] = avroToArrayWritable(record.get(field.name()), field.schema()); - } - return new ArrayWritable(Writable.class, values1); - case ENUM: - return new Text(value.toString()); - case ARRAY: - GenericArray arrayValue = (GenericArray) value; - Writable[] values2 = new Writable[arrayValue.size()]; - int index2 = 0; - for (Object obj : arrayValue) { - values2[index2++] = avroToArrayWritable(obj, schema.getElementType()); - } - return new ArrayWritable(Writable.class, values2); - case MAP: - Map mapValue = (Map) value; - Writable[] values3 = new Writable[mapValue.size()]; - int index3 = 0; - for (Object entry : mapValue.entrySet()) { - Map.Entry mapEntry = (Map.Entry) entry; - Writable[] mapValues = new Writable[2]; - mapValues[0] = new Text(mapEntry.getKey().toString()); - mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType()); - values3[index3++] = new ArrayWritable(Writable.class, mapValues); - } - return new ArrayWritable(Writable.class, values3); - case UNION: - List types = schema.getTypes(); - if (types.size() != 2) { - throw new IllegalArgumentException("Only support union with 2 fields"); - } - Schema s1 = types.get(0); - Schema s2 = types.get(1); - if (s1.getType() == Schema.Type.NULL) { - return avroToArrayWritable(value, s2); - } else if (s2.getType() == Schema.Type.NULL) { - return avroToArrayWritable(value, s1); - } else { - throw new IllegalArgumentException("Only support union with null"); - } - case FIXED: - return new BytesWritable(((GenericFixed) value).bytes()); - default: - return null; + return new RealtimeCompactedRecordReader(split, jobConf, realReader); + } catch (IOException ex) { + LOG.error("Got exception when constructing record reader", ex); + throw new HoodieException(ex); } } @Override - public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException { - // Call the underlying parquetReader.next - which may replace the passed in ArrayWritable - // with a new block of values - boolean result = this.parquetReader.next(aVoid, arrayWritable); - if (!result) { - // if the result is false, then there are no more records - return false; - } else { - // TODO(VC): Right now, we assume all records in log, have a matching base record. (which - // would be true until we have a way to index logs too) - // return from delta records map if we have some match. - String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS] - .toString(); - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("key %s, base values: %s, log values: %s", key, - arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); - } - if (deltaRecordMap.containsKey(key)) { - // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? - Writable[] replaceValue = deltaRecordMap.get(key).get(); - Writable[] originalValue = arrayWritable.get(); - System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); - arrayWritable.set(originalValue); - } - return true; - } + public boolean next(Void key, ArrayWritable value) throws IOException { + return this.reader.next(key, value); } @Override public Void createKey() { - return parquetReader.createKey(); + return this.reader.createKey(); } @Override public ArrayWritable createValue() { - return parquetReader.createValue(); + return this.reader.createValue(); } @Override public long getPos() throws IOException { - return parquetReader.getPos(); + return this.reader.getPos(); } @Override public void close() throws IOException { - parquetReader.close(); + this.reader.close(); } @Override public float getProgress() throws IOException { - return parquetReader.getProgress(); + return this.reader.getProgress(); } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java new file mode 100644 index 0000000000000..c64933389f53c --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -0,0 +1,129 @@ +/* + * 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.hadoop.realtime; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; +import com.uber.hoodie.common.util.FSUtils; +import java.io.IOException; +import java.util.HashMap; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; + +class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader implements + RecordReader { + + protected final RecordReader parquetReader; + private final HashMap deltaRecordMap; + + public RealtimeCompactedRecordReader(HoodieRealtimeFileSplit split, JobConf job, + RecordReader realReader) throws IOException { + super(split, job); + this.parquetReader = realReader; + this.deltaRecordMap = new HashMap<>(); + readAndCompactLog(); + } + + /** + * Goes through the log files and populates a map with latest version of each key logged, since + * the base split was written. + */ + private void readAndCompactLog() throws IOException { + HoodieMergedLogRecordScanner compactedLogRecordScanner = new HoodieMergedLogRecordScanner( + FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), + split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), getMaxCompactionMemoryInBytes(), + Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), + false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), + jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH)); + // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit + // but can return records for completed commits > the commit we are trying to read (if using + // readCommit() API) + for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { + GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(getReaderSchema()).get(); + String key = hoodieRecord.getRecordKey(); + // we assume, a later safe record in the log, is newer than what we have in the map & + // replace it. + // TODO : handle deletes here + ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema()); + deltaRecordMap.put(key, aWritable); + if (LOG.isDebugEnabled()) { + LOG.debug("Log record : " + arrayWritableToString(aWritable)); + } + } + } + + @Override + public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException { + // Call the underlying parquetReader.next - which may replace the passed in ArrayWritable + // with a new block of values + boolean result = this.parquetReader.next(aVoid, arrayWritable); + if (!result) { + // if the result is false, then there are no more records + return false; + } else { + // TODO(VC): Right now, we assume all records in log, have a matching base record. (which + // would be true until we have a way to index logs too) + // return from delta records map if we have some match. + String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS] + .toString(); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("key %s, base values: %s, log values: %s", key, + arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); + } + if (deltaRecordMap.containsKey(key)) { + // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? + Writable[] replaceValue = deltaRecordMap.get(key).get(); + Writable[] originalValue = arrayWritable.get(); + System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); + arrayWritable.set(originalValue); + } + return true; + } + } + + @Override + public Void createKey() { + return parquetReader.createKey(); + } + + @Override + public ArrayWritable createValue() { + return parquetReader.createValue(); + } + + @Override + public long getPos() throws IOException { + return parquetReader.getPos(); + } + + @Override + public void close() throws IOException { + parquetReader.close(); + } + + @Override + public float getProgress() throws IOException { + return parquetReader.getProgress(); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java new file mode 100644 index 0000000000000..afddefc6e28d5 --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -0,0 +1,142 @@ +/* + * 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.hadoop.realtime; + +import com.uber.hoodie.common.table.log.HoodieUnMergedLogRecordScanner; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; +import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer; +import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; +import com.uber.hoodie.hadoop.RecordReaderValueIterator; +import com.uber.hoodie.hadoop.SafeParquetRecordReaderWrapper; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; + +class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implements + RecordReader { + + // Log Record unmerged scanner + private final HoodieUnMergedLogRecordScanner logRecordScanner; + + // Parquet record reader + private final RecordReader parquetReader; + + // Parquet record iterator wrapper for the above reader + private final RecordReaderValueIterator parquetRecordsIterator; + + // Executor that runs the above producers in parallel + private final BoundedInMemoryExecutor executor; + + // Iterator for the buffer consumer + private final Iterator iterator; + + /** + * Construct a Unmerged record reader that parallely consumes both parquet and log records and buffers for upstream + * clients to consume + * + * @param split File split + * @param job Job Configuration + * @param realReader Parquet Reader + */ + public RealtimeUnmergedRecordReader(HoodieRealtimeFileSplit split, JobConf job, + RecordReader realReader) { + super(split, job); + this.parquetReader = new SafeParquetRecordReaderWrapper(realReader); + // Iterator for consuming records from parquet file + this.parquetRecordsIterator = new RecordReaderValueIterator<>(this.parquetReader); + this.executor = new BoundedInMemoryExecutor<>(getMaxCompactionMemoryInBytes(), getParallelProducers(), + Optional.empty(), x -> x, new DefaultSizeEstimator<>()); + // Consumer of this record reader + this.iterator = this.executor.getQueue().iterator(); + this.logRecordScanner = new HoodieUnMergedLogRecordScanner( + FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), + split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), Boolean.valueOf(jobConf + .get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), + false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), + record -> { + // convert Hoodie log record to Hadoop AvroWritable and buffer + GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get(); + ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema()); + this.executor.getQueue().insertRecord(aWritable); + }); + // Start reading and buffering + this.executor.startProducers(); + } + + /** + * Setup log and parquet reading in parallel. Both write to central buffer. + */ + @SuppressWarnings("unchecked") + private List> getParallelProducers() { + List> producers = new ArrayList<>(); + producers.add(new FunctionBasedQueueProducer<>(buffer -> { + logRecordScanner.scan(); + return null; + })); + producers.add(new IteratorBasedQueueProducer<>(parquetRecordsIterator)); + return producers; + } + + @Override + public boolean next(Void key, ArrayWritable value) throws IOException { + if (!iterator.hasNext()) { + return false; + } + // Copy from buffer iterator and set to passed writable + value.set(iterator.next().get()); + return true; + } + + @Override + public Void createKey() { + return parquetReader.createKey(); + } + + @Override + public ArrayWritable createValue() { + return parquetReader.createValue(); + } + + @Override + public long getPos() throws IOException { + //TODO: vb - No logical way to represent parallel stream pos in a single long. + // Should we just return invalid (-1). Where is it used ? + return 0; + } + + @Override + public void close() throws IOException { + this.parquetRecordsIterator.close(); + this.executor.shutdownNow(); + } + + @Override + public float getProgress() throws IOException { + return Math.min(parquetReader.getProgress(), logRecordScanner.getProgress()); + } +} diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java new file mode 100644 index 0000000000000..06fc41c99158f --- /dev/null +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java @@ -0,0 +1,105 @@ +/* + * 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.hadoop; + +import groovy.lang.Tuple2; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.RecordReader; +import org.junit.Assert; +import org.junit.Test; + +public class TestRecordReaderValueIterator { + + @Test + public void testValueIterator() { + String[] values = new String[]{ + "hoodie", + "efficient", + "new project", + "realtime", + "spark", + "dataset", + }; + List> entries = IntStream.range(0, values.length) + .boxed().map(idx -> new Tuple2<>(idx, values[idx])).collect(Collectors.toList()); + TestRecordReader reader = new TestRecordReader(entries); + RecordReaderValueIterator itr = new RecordReaderValueIterator(reader); + for (int i = 0; i < values.length; i++) { + Assert.assertTrue(itr.hasNext()); + Text val = itr.next(); + Assert.assertEquals(values[i], val.toString()); + } + Assert.assertFalse(itr.hasNext()); + } + + /** + * Simple replay record reader for unit-testing + */ + private static class TestRecordReader implements RecordReader { + + private final List> entries; + private int currIndex = 0; + + public TestRecordReader(List> entries) { + this.entries = entries; + } + + + @Override + public boolean next(IntWritable key, Text value) throws IOException { + if (currIndex >= entries.size()) { + return false; + } + key.set(entries.get(currIndex).getFirst()); + value.set(entries.get(currIndex).getSecond()); + currIndex++; + return true; + } + + @Override + public IntWritable createKey() { + return new IntWritable(); + } + + @Override + public Text createValue() { + return new Text(); + } + + @Override + public long getPos() throws IOException { + return currIndex; + } + + @Override + public void close() throws IOException { + + } + + @Override + public float getProgress() throws IOException { + return (currIndex * 1.0F) / entries.size(); + } + } +} diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index ea9016bb965f9..a889e1a6ec91c 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -35,8 +35,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -71,7 +73,7 @@ public class HoodieRealtimeRecordReaderTest { @Before public void setUp() { jobConf = new JobConf(); - jobConf.set(HoodieRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024)); + jobConf.set(AbstractRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024)); hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf); } @@ -82,12 +84,18 @@ public void setUp() { private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit, int numberOfRecords) throws InterruptedException, IOException { + return writeLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0); + } + + private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId, + String baseCommit, String newCommit, int numberOfRecords, int offset) + throws InterruptedException, IOException { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) .overBaseCommit(baseCommit).withFs(fs).build(); List records = new ArrayList<>(); - for (int i = 0; i < numberOfRecords; i++) { + for (int i = offset; i < offset + numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); } Schema writeSchema = records.get(0).getSchema(); @@ -142,8 +150,7 @@ public void testReader() throws Exception { jobConf.set("partition_columns", "datestr"); //validate record reader compaction - HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, - reader); + HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); //use reader to read base Parquet File and log file, merge in flight and return latest commit //here all 100 records should be updated, see above @@ -158,6 +165,90 @@ public void testReader() throws Exception { } } + @Test + public void testUnMergedReader() throws Exception { + // initial commit + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); + HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), + HoodieTableType.MERGE_ON_READ); + String commitTime = "100"; + final int numRecords = 1000; + final int firstBatchLastRecordKey = numRecords - 1; + final int secondBatchLastRecordKey = 2 * numRecords - 1; + File partitionDir = InputFormatTestUtil + .prepareParquetDataset(basePath, schema, 1, numRecords, commitTime); + InputFormatTestUtil.commit(basePath, commitTime); + // Add the paths + FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); + + // insert new records to log file + String newCommitTime = "101"; + HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime, + newCommitTime, numRecords, numRecords); + long size = writer.getCurrentSize(); + writer.close(); + assertTrue("block - size should be > 0", size > 0); + + //create a split with baseFile (parquet file written earlier) and new log file(s) + String logFilePath = writer.getLogFile().getPath().toString(); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, + jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime); + + //create a RecordReader to be used by HoodieRealtimeRecordReader + RecordReader reader = + new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), + jobConf, null); + JobConf jobConf = new JobConf(); + List fields = schema.getFields(); + String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); + String postions = fields.stream().map(f -> String.valueOf(f.pos())) + .collect(Collectors.joining(",")); + jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); + jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); + jobConf.set("partition_columns", "datestr"); + // Enable merge skipping. + jobConf.set("hoodie.realtime.merge.skip", "true"); + + //validate unmerged record reader + RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader); + + //use reader to read base Parquet File and log file + //here all records should be present. Also ensure log records are in order. + Void key = recordReader.createKey(); + ArrayWritable value = recordReader.createValue(); + int numRecordsAtCommit1 = 0; + int numRecordsAtCommit2 = 0; + Set seenKeys = new HashSet<>(); + Integer lastSeenKeyFromLog = firstBatchLastRecordKey; + while (recordReader.next(key, value)) { + Writable[] values = value.get(); + String gotCommit = values[0].toString(); + String keyStr = values[2].toString(); + Integer gotKey = Integer.parseInt(keyStr.substring("key".length())); + if (gotCommit.equals(newCommitTime)) { + numRecordsAtCommit2++; + Assert.assertTrue(gotKey > firstBatchLastRecordKey); + Assert.assertTrue(gotKey <= secondBatchLastRecordKey); + Assert.assertEquals(gotKey.intValue(), lastSeenKeyFromLog + 1); + lastSeenKeyFromLog++; + } else { + numRecordsAtCommit1++; + Assert.assertTrue(gotKey >= 0); + Assert.assertTrue(gotKey <= firstBatchLastRecordKey); + } + // Ensure unique key + Assert.assertFalse(seenKeys.contains(gotKey)); + seenKeys.add(gotKey); + key = recordReader.createKey(); + value = recordReader.createValue(); + } + Assert.assertEquals(numRecords, numRecordsAtCommit1); + Assert.assertEquals(numRecords, numRecordsAtCommit2); + Assert.assertEquals(2 * numRecords, seenKeys.size()); + } + @Test public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit @@ -203,8 +294,7 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { jobConf.set("partition_columns", "datestr"); // validate record reader compaction - HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, - reader); + HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); // use reader to read base Parquet File and log file, merge in flight and return latest commit // here the first 50 records should be updated, see above From e4b12248a3b2df9954bacc84c5ffbb753ca6c529 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 2 May 2018 00:52:37 -0700 Subject: [PATCH 054/374] enabling global index for MOR --- .../hoodie/table/HoodieMergeOnReadTable.java | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 79239ada8e8cd..857402564456e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -39,6 +39,7 @@ import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieUpsertException; +import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import java.io.IOException; @@ -154,11 +155,10 @@ public List rollback(JavaSparkContext jsc, List comm // Atomically un-publish all non-inflight commits commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue()) .filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight); - logger.info("Unpublished " + commits); - Long startTime = System.currentTimeMillis(); - + // TODO (NA) : remove this once HoodieIndex is a member of HoodieTable + HoodieIndex hoodieIndex = HoodieIndex.createIndex(config, jsc); List allRollbackStats = jsc.parallelize(FSUtils .getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) @@ -195,17 +195,24 @@ public List rollback(JavaSparkContext jsc, List comm // append rollback blocks for updates if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { + // This needs to be done since GlobalIndex at the moment does not store the latest commit time + Map fileIdToLatestCommitTimeMap = + hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null; commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { - return wStat != null - && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT + return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null; }).forEach(wStat -> { HoodieLogFormat.Writer writer = null; + String baseCommitTime = wStat.getPrevCommit(); + if (hoodieIndex.isGlobal()) { + baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId()); + } try { writer = HoodieLogFormat.newWriterBuilder().onParentPath( new Path(this.getMetaClient().getBasePath(), partitionPath)) - .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) + .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) .withFs(this.metaClient.getFs()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; From bd4273d88721ed5e1ce66cc299f1d5c6c668dab6 Mon Sep 17 00:00:00 2001 From: Sunil Ramaiah Date: Thu, 17 May 2018 15:40:47 -0700 Subject: [PATCH 055/374] Added a filter function to filter the record keys in a parquet file --- .../bloom/HoodieBloomIndexCheckFunction.java | 14 ++-- .../uber/hoodie/common/util/ParquetUtils.java | 42 +++++++++++- .../hoodie/common/util/TestParquetUtils.java | 64 ++++++++++++++----- 3 files changed, 93 insertions(+), 27 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index 074ec56dae863..a52582455f823 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -26,6 +26,7 @@ import com.uber.hoodie.func.LazyIterableIterator; import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -64,16 +65,9 @@ public static List checkCandidatesAgainstFile(Configuration configuratio try { // Load all rowKeys from the file, to double-confirm if (!candidateRecordKeys.isEmpty()) { - Set fileRowKeys = ParquetUtils.readRowKeysFromParquet(configuration, filePath); - logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath); - if (logger.isDebugEnabled()) { - logger.debug("Keys from " + filePath + " => " + fileRowKeys); - } - for (String rowKey : candidateRecordKeys) { - if (fileRowKeys.contains(rowKey)) { - foundRecordKeys.add(rowKey); - } - } + Set fileRowKeys = ParquetUtils.filterParquetRowKeys(configuration, filePath, + new HashSet<>(candidateRecordKeys)); + foundRecordKeys.addAll(fileRowKeys); logger.info("After checking with row keys, we have " + foundRecordKeys.size() + " results, for file " + filePath + " => " + foundRecordKeys); if (logger.isDebugEnabled()) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index f4215c8045f47..636bec583ec9c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -29,9 +29,12 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroParquetReader; @@ -52,8 +55,26 @@ public class ParquetUtils { * * @param filePath The parquet file path. * @param configuration configuration to build fs object + * @return Set Set of row keys */ public static Set readRowKeysFromParquet(Configuration configuration, Path filePath) { + return filterParquetRowKeys(configuration, filePath, new HashSet<>()); + } + + /** + * Read the rowKey list matching the given filter, from the given parquet file. If the filter is empty, + * then this will return all the rowkeys. + * + * @param filePath The parquet file path. + * @param configuration configuration to build fs object + * @param filter record keys filter + * @return Set Set of row keys matching candidateRecordKeys + */ + public static Set filterParquetRowKeys(Configuration configuration, Path filePath, Set filter) { + Optional filterFunction = Optional.empty(); + if (CollectionUtils.isNotEmpty(filter)) { + filterFunction = Optional.of(new RecordKeysFilterFunction(filter)); + } Configuration conf = new Configuration(configuration); conf.addResource(getFs(filePath.toString(), conf).getConf()); Schema readSchema = HoodieAvroUtils.getRecordKeySchema(); @@ -66,7 +87,10 @@ public static Set readRowKeysFromParquet(Configuration configuration, Pa Object obj = reader.read(); while (obj != null) { if (obj instanceof GenericRecord) { - rowKeys.add(((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()); + String recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + if (!filterFunction.isPresent() || filterFunction.get().apply(recordKey)) { + rowKeys.add(recordKey); + } } obj = reader.read(); } @@ -178,4 +202,20 @@ public static List readAvroRecords(Configuration configuration, P } return records; } + + static class RecordKeysFilterFunction implements Function { + private final Set candidateKeys; + + RecordKeysFilterFunction(Set candidateKeys) { + this.candidateKeys = candidateKeys; + } + + @Override + public Boolean apply(String recordKey) { + if (candidateKeys.contains(recordKey)) { + return true; + } + return false; + } + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java index 636df99e3d602..3d5949b6ab2ef 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java @@ -26,7 +26,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -54,28 +56,13 @@ public void setup() throws IOException { @Test public void testHoodieWriteSupport() throws Exception { - List rowKeys = new ArrayList<>(); for (int i = 0; i < 1000; i++) { rowKeys.add(UUID.randomUUID().toString()); } - // Write out a parquet file - Schema schema = HoodieAvroUtils.getRecordKeySchema(); - BloomFilter filter = new BloomFilter(1000, 0.0001); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, - filter); - String filePath = basePath + "/test.parquet"; - ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, - 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); - for (String rowKey : rowKeys) { - GenericRecord rec = new GenericData.Record(schema); - rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey); - writer.write(rec); - filter.add(rowKey); - } - writer.close(); + writeParquetFile(filePath, rowKeys); // Read and verify List rowKeysInFile = new ArrayList<>( @@ -90,4 +77,49 @@ public void testHoodieWriteSupport() throws Exception { assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); } } + + @Test + public void testFilterParquetRowKeys() throws Exception { + List rowKeys = new ArrayList<>(); + Set filter = new HashSet<>(); + for (int i = 0; i < 1000; i++) { + String rowKey = UUID.randomUUID().toString(); + rowKeys.add(rowKey); + if (i % 100 == 0) { + filter.add(rowKey); + } + } + + String filePath = basePath + "/test.parquet"; + writeParquetFile(filePath, rowKeys); + + // Read and verify + Set filtered = ParquetUtils.filterParquetRowKeys(HoodieTestUtils.getDefaultHadoopConf(), + new Path(filePath), + filter); + + assertEquals("Filtered count does not match", filter.size(), filtered.size()); + + for (String rowKey : filtered) { + assertTrue("filtered key must be in the given filter", filter.contains(rowKey)); + } + } + + private void writeParquetFile(String filePath, + List rowKeys) throws Exception { + // Write out a parquet file + Schema schema = HoodieAvroUtils.getRecordKeySchema(); + BloomFilter filter = new BloomFilter(1000, 0.0001); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, + filter); + ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, + 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); + for (String rowKey : rowKeys) { + GenericRecord rec = new GenericData.Record(schema); + rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey); + writer.write(rec); + filter.add(rowKey); + } + writer.close(); + } } From 9ebee855c27fb5813ce1212d408482418bf86970 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Fri, 5 Jan 2018 14:06:18 -0800 Subject: [PATCH 056/374] Improving out of box experience for data source - Fixes #246 - Bump up default parallelism to 1500, to handle large upserts - Add docs on s3 confuration & tuning tips with tested spark knobs - Fix bug to not duplicate hoodie metadata fields when input dataframe is another hoodie dataset - Improve speed of ROTablePathFilter by removing directory check - Move to spark-avro 4.0 to handle issue with nested fields with same name - Keep AvroConversionUtils in sync with spark-avro 4.0 --- docs/configurations.md | 44 ++++++++++++++++++- docs/s3_filesystem.md | 20 ++++++++- .../uber/hoodie/config/HoodieWriteConfig.java | 2 +- .../hoodie/common/util/HoodieAvroUtils.java | 11 ++++- .../hadoop/HoodieROTablePathFilter.java | 10 ++--- hoodie-spark/pom.xml | 2 +- .../com/uber/hoodie/AvroConversionUtils.scala | 25 +++++++++-- .../scala/com/uber/hoodie/DefaultSource.scala | 17 ++++--- 8 files changed, 112 insertions(+), 19 deletions(-) diff --git a/docs/configurations.md b/docs/configurations.md index bd4f4a27a2824..f8c94669870fe 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -7,6 +7,8 @@ toc: false summary: "Here we list all possible configurations and what they mean" --- +### Configuration + * [HoodieWriteConfig](#HoodieWriteConfig)
Top Level Config which is passed in when HoodieWriteClent is created. - [withPath](#withPath) (hoodie_base_path)
@@ -152,4 +154,44 @@ summary: "Here we list all possible configurations and what they mean" `instant_time <= END_INSTANTTIME` are fetched out. -{% include callout.html content="Hoodie is a young project. A lot of pluggable interfaces and configurations to support diverse workloads need to be created. Get involved [here](https://github.com/uber/hoodie)" type="info" %} +### Tuning + +Writing data via Hoodie happens as a Spark job and thus general rules of spark debugging applies here too. Below is a list of things to keep in mind, if you are looking to improving performance or reliability. + + - **Right operations** : Use `bulkinsert` to load new data into a table, and there on use `upsert`/`insert`. Difference between them is that bulk insert uses a disk based write path to scale to load large inputs without need to cache it. + - **Input Parallelism** : By default, Hoodie tends to over-partition input (i.e `withParallelism(1500)`), to ensure each Spark partition stays within the 2GB limit for inputs upto 500GB. Bump this up accordingly if you have larger inputs + - **Off-heap memory** : Hoodie writes parquet files and that needs good amount of off-heap memory proportional to schema width. Consider setting something like `spark.yarn.executor.memoryOverhead` or `spark.yarn.driver.memoryOverhead`, if you are running into such failures. + - **Spark Memory** : Typically, hoodie needs to be able to read a single file into memory to perform merges or compactions and thus the executor memory should be sufficient to accomodate this. In addition, Hoodie caches the input to be able to intelligently place data and thus leaving some `spark.storage.memoryFraction` will generally help boost performance. + - **Sizing files** : Set `limitFileSize` above judiciously, to balance ingest/write latency vs number of files & consequently metadata overhead associated with it. + - **Timeseries/Log data** : Default configs are tuned for database/nosql changelogs where individual record sizes are large. Another very popular class of data is timeseries/event/log data that tends to be more volumnious with lot more records per partition. In such cases + - Consider tuning the bloom filter accuracy via `.bloomFilterFPP()/bloomFilterNumEntries()` to achieve your target index look up time + - Consider making a key that is prefixed with time of the event, which will enable range pruning & significantly speeding up index lookup. + + Below is a full working production config + + ``` + spark.driver.extraClassPath /etc/hive/conf + spark.driver.extraJavaOptions -XX:+PrintTenuringDistribution -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCApplicationConcurrentTime -XX:+PrintGCTimeStamps -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/hoodie-heapdump.hprof + spark.driver.maxResultSize 2g + spark.driver.memory 4g + spark.executor.cores 1 + spark.executor.extraJavaOptions -XX:+PrintFlagsFinal -XX:+PrintReferenceGC -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintAdaptiveSizePolicy -XX:+UnlockDiagnosticVMOptions -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/hoodie-heapdump.hprof + spark.executor.id driver + spark.executor.instances 300 + spark.executor.memory 6g + spark.rdd.compress true + + spark.kryoserializer.buffer.max 512m + spark.serializer org.apache.spark.serializer.KryoSerializer + spark.shuffle.memoryFraction 0.2 + spark.shuffle.service.enabled true + spark.sql.hive.convertMetastoreParquet false + spark.storage.memoryFraction 0.6 + spark.submit.deployMode cluster + spark.task.cpus 1 + spark.task.maxFailures 4 + + spark.yarn.driver.memoryOverhead 1024 + spark.yarn.executor.memoryOverhead 3072 + spark.yarn.max.executor.failures 100 + ``` diff --git a/docs/s3_filesystem.md b/docs/s3_filesystem.md index adb1cefcb5a4f..09d70e62530e2 100644 --- a/docs/s3_filesystem.md +++ b/docs/s3_filesystem.md @@ -17,7 +17,9 @@ There are two configurations required for Hoodie-S3 compatibility: ### AWS Credentials -Add the required configs in your core-site.xml from where Hoodie can fetch them. Replace the `fs.defaultFS` with your S3 bucket name and Hoodie should be able to read/write from the bucket. +Simplest way to use Hoodie with S3, is to configure your `SparkSession` or `SparkContext` with S3 credentials. Hoodie will automatically pick this up and talk to S3. + +Alternatively, add the required configs in your core-site.xml from where Hoodie can fetch them. Replace the `fs.defaultFS` with your S3 bucket name and Hoodie should be able to read/write from the bucket. ``` @@ -51,6 +53,22 @@ Add the required configs in your core-site.xml from where Hoodie can fetch them. ``` + +Utilities such as hoodie-cli or deltastreamer tool, can pick up s3 creds via environmental variable prefixed with `HOODIE_ENV_`. For e.g below is a bash snippet to setup +such variables and then have cli be able to work on datasets stored in s3 + +``` +export HOODIE_ENV_fs_DOT_s3a_DOT_access_DOT_key=$accessKey +export HOODIE_ENV_fs_DOT_s3a_DOT_secret_DOT_key=$secretKey +export HOODIE_ENV_fs_DOT_s3_DOT_awsAccessKeyId=$accessKey +export HOODIE_ENV_fs_DOT_s3_DOT_awsSecretAccessKey=$secretKey +export HOODIE_ENV_fs_DOT_s3n_DOT_awsAccessKeyId=$accessKey +export HOODIE_ENV_fs_DOT_s3n_DOT_awsSecretAccessKey=$secretKey +export HOODIE_ENV_fs_DOT_s3n_DOT_impl=org.apache.hadoop.fs.s3a.S3AFileSystem +``` + + + ### AWS Libs AWS hadoop libraries to add to our classpath diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index e0fe99379e0db..ebec1dd7b8186 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -41,7 +41,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String TABLE_NAME = "hoodie.table.name"; private static final String BASE_PATH_PROP = "hoodie.base.path"; private static final String AVRO_SCHEMA = "hoodie.avro.schema"; - private static final String DEFAULT_PARALLELISM = "200"; + private static final String DEFAULT_PARALLELISM = "1500"; private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism"; private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism"; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index 09d1a2cab8078..45b69e0b6a829 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -74,6 +74,13 @@ public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOEx return reader.read(null, decoder); } + public static boolean isMetadataField(String fieldName) { + return HoodieRecord.COMMIT_TIME_METADATA_FIELD.equals(fieldName) + || HoodieRecord.COMMIT_SEQNO_METADATA_FIELD.equals(fieldName) + || HoodieRecord.RECORD_KEY_METADATA_FIELD.equals(fieldName) + || HoodieRecord.PARTITION_PATH_METADATA_FIELD.equals(fieldName) + || HoodieRecord.FILENAME_METADATA_FIELD.equals(fieldName); + } /** * Adds the Hoodie metadata fields to the given schema @@ -98,7 +105,9 @@ public static Schema addMetadataFields(Schema schema) { parentFields.add(partitionPathField); parentFields.add(fileNameField); for (Schema.Field field : schema.getFields()) { - parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null)); + if (!isMetadataField(field.name())) { + parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null)); + } } Schema mergedSchema = Schema diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index 44672ec4b6452..158aa00391dba 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -20,7 +20,6 @@ import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieException; import java.io.Serializable; @@ -61,6 +60,9 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable { private HashSet nonHoodiePathCache; + private transient FileSystem fs; + + public HoodieROTablePathFilter() { hoodiePathCache = new HashMap<>(); nonHoodiePathCache = new HashSet<>(); @@ -79,7 +81,6 @@ private Path safeGetParentsParent(Path path) { return null; } - @Override public boolean accept(Path path) { @@ -88,9 +89,8 @@ public boolean accept(Path path) { } Path folder = null; try { - FileSystem fs = path.getFileSystem(FSUtils.prepareHadoopConf(new Configuration())); - if (fs.isDirectory(path)) { - return true; + if (fs == null) { + fs = path.getFileSystem(new Configuration()); } // Assumes path is a file diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 3e8878ea94549..634dab06b6768 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -142,7 +142,7 @@ com.databricks spark-avro_2.11 - 3.2.0 + 4.0.0 com.fasterxml.jackson.core diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index 4312636fa2586..e92043490690d 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -43,6 +43,16 @@ object AvroConversionUtils { } } + def getNewRecordNamespace(elementDataType: DataType, + currentRecordNamespace: String, + elementName: String): String = { + + elementDataType match { + case StructType(_) => s"$currentRecordNamespace.$elementName" + case _ => currentRecordNamespace + } + } + def createConverterToAvro(dataType: DataType, structName: String, recordNamespace: String): (Any) => Any = { @@ -60,7 +70,10 @@ object AvroConversionUtils { case DateType => (item: Any) => if (item == null) null else item.asInstanceOf[Date].getTime case ArrayType(elementType, _) => - val elementConverter = createConverterToAvro(elementType, structName, recordNamespace) + val elementConverter = createConverterToAvro( + elementType, + structName, + getNewRecordNamespace(elementType, recordNamespace, structName)) (item: Any) => { if (item == null) { null @@ -77,7 +90,10 @@ object AvroConversionUtils { } } case MapType(StringType, valueType, _) => - val valueConverter = createConverterToAvro(valueType, structName, recordNamespace) + val valueConverter = createConverterToAvro( + valueType, + structName, + getNewRecordNamespace(valueType, recordNamespace, structName)) (item: Any) => { if (item == null) { null @@ -94,7 +110,10 @@ object AvroConversionUtils { val schema: Schema = SchemaConverters.convertStructToAvro( structType, builder, recordNamespace) val fieldConverters = structType.fields.map(field => - createConverterToAvro(field.dataType, field.name, recordNamespace)) + createConverterToAvro( + field.dataType, + field.name, + getNewRecordNamespace(field.dataType, recordNamespace, field.name))) (item: Any) => { if (item == null) { null diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index dec9160e22d9e..dd744da6d5079 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -134,11 +134,16 @@ class DefaultSource extends RelationProvider df: DataFrame): BaseRelation = { val parameters = parametersWithWriteDefaults(optParams).toMap + val sparkContext = sqlContext.sparkContext val path = parameters.get("path") val tblName = parameters.get(HoodieWriteConfig.TABLE_NAME) if (path.isEmpty || tblName.isEmpty) { throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.") } + val serializer = sparkContext.getConf.get("spark.serializer") + if (!serializer.equals("org.apache.spark.serializer.KryoSerializer")) { + throw new HoodieException(s"${serializer} serialization is not supported by hoodie. Please use kryo.") + } val storageType = parameters(STORAGE_TYPE_OPT_KEY) val operation = parameters(OPERATION_OPT_KEY) @@ -146,11 +151,12 @@ class DefaultSource extends RelationProvider // register classes & schemas val structName = s"${tblName.get}_record" val nameSpace = s"hoodie.${tblName.get}" - sqlContext.sparkContext.getConf.registerKryoClasses( + sparkContext.getConf.registerKryoClasses( Array(classOf[org.apache.avro.generic.GenericData], classOf[org.apache.avro.Schema])) val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) - sqlContext.sparkContext.getConf.registerAvroSchemas(schema) + sparkContext.getConf.registerAvroSchemas(schema) + log.info(s"Registered avro schema : ${schema.toString(true)}"); // Convert to RDD[HoodieRecord] val keyGenerator = DataSourceUtils.createKeyGenerator( @@ -167,7 +173,7 @@ class DefaultSource extends RelationProvider val basePath = new Path(parameters.get("path").get) - val fs = basePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) var exists = fs.exists(basePath) // Handle various save modes @@ -190,12 +196,11 @@ class DefaultSource extends RelationProvider properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tblName.get); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, storageType); properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived"); - HoodieTableMetaClient.initializePathAsHoodieDataset( - sqlContext.sparkContext.hadoopConfiguration, path.get, properties); + HoodieTableMetaClient.initializePathAsHoodieDataset(sparkContext.hadoopConfiguration, path.get, properties); } // Create a HoodieWriteClient & issue the write. - val client = DataSourceUtils.createHoodieClient(new JavaSparkContext(sqlContext.sparkContext), + val client = DataSourceUtils.createHoodieClient(new JavaSparkContext(sparkContext), schema.toString, path.get, tblName.get, From 2b8fb08fa9d807a4dbba8db36d68071b58b89b3a Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Sun, 10 Jun 2018 18:54:58 -0700 Subject: [PATCH 057/374] Fixing deps & serialization for RTView - hoodie-hadoop-mr now needs objectsize bundled - Also updated docs with additional tuning tips --- docs/configurations.md | 5 ++++- hoodie-hadoop-mr/pom.xml | 6 ++++++ .../hadoop/realtime/HoodieRealtimeFileSplit.java | 16 ++++++++-------- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/docs/configurations.md b/docs/configurations.md index f8c94669870fe..4639fb2e52378 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -159,13 +159,16 @@ summary: "Here we list all possible configurations and what they mean" Writing data via Hoodie happens as a Spark job and thus general rules of spark debugging applies here too. Below is a list of things to keep in mind, if you are looking to improving performance or reliability. - **Right operations** : Use `bulkinsert` to load new data into a table, and there on use `upsert`/`insert`. Difference between them is that bulk insert uses a disk based write path to scale to load large inputs without need to cache it. - - **Input Parallelism** : By default, Hoodie tends to over-partition input (i.e `withParallelism(1500)`), to ensure each Spark partition stays within the 2GB limit for inputs upto 500GB. Bump this up accordingly if you have larger inputs + - **Input Parallelism** : By default, Hoodie tends to over-partition input (i.e `withParallelism(1500)`), to ensure each Spark partition stays within the 2GB limit for inputs upto 500GB. Bump this up accordingly if you have larger inputs. We recommend having shuffle parallelism `hoodie.[insert|upsert|bulkinsert].shuffle.parallelism` such that its atleast input_data_size/500MB - **Off-heap memory** : Hoodie writes parquet files and that needs good amount of off-heap memory proportional to schema width. Consider setting something like `spark.yarn.executor.memoryOverhead` or `spark.yarn.driver.memoryOverhead`, if you are running into such failures. - **Spark Memory** : Typically, hoodie needs to be able to read a single file into memory to perform merges or compactions and thus the executor memory should be sufficient to accomodate this. In addition, Hoodie caches the input to be able to intelligently place data and thus leaving some `spark.storage.memoryFraction` will generally help boost performance. - **Sizing files** : Set `limitFileSize` above judiciously, to balance ingest/write latency vs number of files & consequently metadata overhead associated with it. - **Timeseries/Log data** : Default configs are tuned for database/nosql changelogs where individual record sizes are large. Another very popular class of data is timeseries/event/log data that tends to be more volumnious with lot more records per partition. In such cases - Consider tuning the bloom filter accuracy via `.bloomFilterFPP()/bloomFilterNumEntries()` to achieve your target index look up time - Consider making a key that is prefixed with time of the event, which will enable range pruning & significantly speeding up index lookup. + - **GC Tuning** : Please be sure to follow garbage collection tuning tips from Spark tuning guide to avoid OutOfMemory errors + - [Must] Use G1/CMS Collector. Sample CMS Flags to add to spark.executor.extraJavaOptions : ``-XX:NewSize=1g -XX:SurvivorRatio=2 -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:CMSInitiatingOccupancyFraction=70 -XX:+PrintTenuringDistribution -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCApplicationConcurrentTime -XX:+PrintTenuringDistribution -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/hoodie-heapdump.hprof` + - If it keeps OOMing still, reduce spark memory conservatively: `spark.memory.fraction=0.2, spark.memory.storageFraction=0.2` allowing it to spill rather than OOM. (reliably slow vs crashing intermittently) Below is a full working production config diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 7bf618011ea94..eae2eb8d74e51 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -79,6 +79,11 @@ com.twitter parquet-avro + + com.twitter.common + objectsize + 0.0.12 + org.apache.avro avro @@ -114,6 +119,7 @@ com.uber.hoodie:hoodie-common com.twitter:parquet-avro + com.twitter.common:objectsize diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeFileSplit.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeFileSplit.java index 5ba7545b713a7..36f6a54b7280a 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -63,22 +63,22 @@ public String getBasePath() { } private static void writeString(String str, DataOutput out) throws IOException { - byte[] pathBytes = str.getBytes(StandardCharsets.UTF_8); - out.writeInt(pathBytes.length); - out.write(pathBytes); + byte[] bytes = str.getBytes(StandardCharsets.UTF_8); + out.writeInt(bytes.length); + out.write(bytes); } private static String readString(DataInput in) throws IOException { - byte[] pathBytes = new byte[in.readInt()]; - in.readFully(pathBytes); - return new String(pathBytes, StandardCharsets.UTF_8); + byte[] bytes = new byte[in.readInt()]; + in.readFully(bytes); + return new String(bytes, StandardCharsets.UTF_8); } @Override public void write(DataOutput out) throws IOException { super.write(out); - + writeString(basePath, out); writeString(maxCommitTime, out); out.writeInt(deltaFilePaths.size()); for (String logFilePath : deltaFilePaths) { @@ -89,7 +89,7 @@ public void write(DataOutput out) throws IOException { @Override public void readFields(DataInput in) throws IOException { super.readFields(in); - + basePath = readString(in); maxCommitTime = readString(in); int totalLogFiles = in.readInt(); deltaFilePaths = new ArrayList<>(totalLogFiles); From 29de41fcc0c23e1f60251ee97d35970d7fed63e7 Mon Sep 17 00:00:00 2001 From: Xavier Jodoin Date: Wed, 28 Mar 2018 13:09:57 -0400 Subject: [PATCH 058/374] Fix wrong use of TemporaryFolder junit rule --- .../hoodie/common/model/HoodieTestUtils.java | 17 +++++++---------- .../common/table/HoodieTableMetaClientTest.java | 7 ++++++- .../table/string/HoodieActiveTimelineTest.java | 14 +++++--------- .../view/HoodieTableFileSystemViewTest.java | 7 ++++++- .../hadoop/TestHoodieROTablePathFilter.java | 13 ++++++++++++- 5 files changed, 36 insertions(+), 22 deletions(-) diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 6a956f27c9766..0fbb46cd28f20 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -69,7 +69,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.util.StringUtils; -import org.junit.rules.TemporaryFolder; public class HoodieTestUtils { @@ -83,7 +82,13 @@ public static Configuration getDefaultHadoopConf() { return new Configuration(); } - public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) throws IOException { + public static HoodieTableMetaClient init(String basePath) + throws IOException { + return initTableType(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE); + } + + public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) + throws IOException { return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); } @@ -96,14 +101,6 @@ public static HoodieTableMetaClient initTableType(Configuration hadoopConf, Stri return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); } - public static HoodieTableMetaClient initOnTemp() throws IOException { - // Create a temp folder as the base path - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - String basePath = folder.getRoot().getAbsolutePath(); - return HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath); - } - public static String makeNewCommitTime() { return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index 414166b99ed7a..b9c9f889b46c8 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -35,16 +35,21 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; public class HoodieTableMetaClientTest { private HoodieTableMetaClient metaClient; private String basePath; + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Before public void init() throws IOException { - metaClient = HoodieTestUtils.initOnTemp(); + metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath()); basePath = metaClient.getBasePath(); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 25a6a00f486bd..df642490b4975 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -28,12 +28,11 @@ import java.io.IOException; import java.util.Optional; import java.util.stream.Stream; -import org.apache.hadoop.fs.Path; -import org.junit.After; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; public class HoodieActiveTimelineTest { @@ -41,15 +40,12 @@ public class HoodieActiveTimelineTest { private HoodieTableMetaClient metaClient; @Rule public final ExpectedException exception = ExpectedException.none(); - + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Before public void setUp() throws Exception { - this.metaClient = HoodieTestUtils.initOnTemp(); - } - - @After - public void tearDown() throws Exception { - metaClient.getFs().delete(new Path(this.metaClient.getBasePath()), true); + this.metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath()); } @Test diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index f26f6d3cd8d74..6862412716963 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -44,7 +44,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; @SuppressWarnings("ResultOfMethodCallIgnored") public class HoodieTableFileSystemViewTest { @@ -54,10 +56,13 @@ public class HoodieTableFileSystemViewTest { private TableFileSystemView fsView; private TableFileSystemView.ReadOptimizedView roView; private TableFileSystemView.RealtimeView rtView; + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); @Before public void init() throws IOException { - metaClient = HoodieTestUtils.initOnTemp(); + metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());; basePath = metaClient.getBasePath(); fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java index c81f9fccbf98f..e139b737177dc 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java @@ -24,6 +24,8 @@ import java.io.File; import java.io.IOException; import org.apache.hadoop.fs.Path; +import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -32,10 +34,19 @@ */ public class TestHoodieROTablePathFilter { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private HoodieTableMetaClient metaClient; + + @Before + public void setUp() throws Exception { + this.metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath()); + } + @Test public void testHoodiePaths() throws IOException { // Create a temp folder as the base path - HoodieTableMetaClient metaClient = HoodieTestUtils.initOnTemp(); String basePath = metaClient.getBasePath(); HoodieTestUtils.createCommitFiles(basePath, "001", "002"); From 62eecb95f4f1807b8ccdd538b0f342fd847792b5 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 11 Jun 2018 08:41:11 -0700 Subject: [PATCH 059/374] Update Release notes for 0.4.2 release --- RELEASE_NOTES.md | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index da9bce008ed01..c8d0564550942 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -1,3 +1,41 @@ +Release 0.4.2 +------------------------------------ + +### Highlights + * Parallelize Parquet writing & input record read resulting in upto 2x performance improvement + * Better out-of-box configs to support upto 500GB upserts, improved ROPathFilter performance + * Added a union mode for RT View, that supports near-real time event ingestion without update semantics + * Added a tuning guide with suggestions for oft-encountered problems + * New configs for configs for compression ratio, index storage levels + +### Full PR List + + * **@jianxu** - Use hadoopConf in HoodieTableMetaClient and related tests #343 + * **@jianxu** - Add more options in HoodieWriteConfig #341 + * **@n3nash** - Adding a tool to read/inspect a HoodieLogFile #328 + * **@ovj** - Parallelizing parquet write and spark's external read operation. #294 + * **@n3nash** - Fixing memory leak due to HoodieLogFileReader holding on to a logblock #346 + * **@kaushikd49** - DeduplicateRecords based on recordKey if global index is used #345 + * **@jianxu** - Checking storage level before persisting preppedRecords #358 + * **@n3nash** - Adding config for parquet compression ratio #366 + * **@xjodoin** - Replace deprecated jackson version #367 + * **@n3nash** - Making ExternalSpillableMap generic for any datatype #350 + * **@bvaradar** - CodeStyle formatting to conform to basic Checkstyle rules. #360 + * **@vinothchandar** - Update release notes for 0.4.1 (post) #371 + * **@bvaradar** - Issue-329 : Refactoring TestHoodieClientOnCopyOnWriteStorage and adding test-cases #372 + * **@n3nash** - Parallelized read-write operations in Hoodie Merge phase #370 + * **@n3nash** - Using BufferedFsInputStream to wrap FSInputStream for FSDataInputStream #373 + * **@suniluber** - Fix for updating duplicate records in same/different files in same pa… #380 + * **@bvaradar** - Fixit : Add Support for ordering and limiting results in CLI show commands #383 + * **@n3nash** - Adding metrics for MOR and COW #365 + * **@n3nash** - Adding a fix/workaround when fs.append() unable to return a valid outputstream #388 + * **@n3nash** - Minor fixes for MergeOnRead MVP release readiness #387 + * **@bvaradar** - Issue-257: Support union mode in HoodieRealtimeRecordReader for pure insert workloads #379 + * **@n3nash** - Enabling global index for MOR #389 + * **@suniluber** - Added a new filter function to filter by record keys when reading parquet file #395 + * **@vinothchandar** - Improving out of box experience for data source #295 + * **@xjodoin** - Fix wrong use of TemporaryFolder junit rule #411 + Release 0.4.1 ------------------------------------ From 3af91c3bd18c802125f77ddd23ee71507c4c2c06 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Mon, 11 Jun 2018 08:59:02 -0700 Subject: [PATCH 060/374] [maven-release-plugin] prepare release hoodie-0.4.2 --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- pom.xml | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 5712dd42da064..251b6a0097799 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index b027b357e5905..86c10cc44fc97 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 7c5a9adb50445..f5e23e091f634 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index eae2eb8d74e51..4471e04021447 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index a4f084e91315b..67b5d9f8829ec 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 634dab06b6768..bfdb03d4baffb 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index f752e0f90a47e..e43601eae9cbf 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2-SNAPSHOT + 0.4.2 4.0.0 diff --git a/pom.xml b/pom.xml index f8240c4829289..a22fa1763bca8 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.2-SNAPSHOT + 0.4.2 Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - HEAD + hoodie-0.4.2 From c206ae128c16030636dc9c8b24399fcc1d31cb05 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Mon, 11 Jun 2018 08:59:13 -0700 Subject: [PATCH 061/374] [maven-release-plugin] prepare for next development iteration --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- pom.xml | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 251b6a0097799..c6fe41fe10e65 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index 86c10cc44fc97..6dff57c98247a 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index f5e23e091f634..bbf8447112410 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 4471e04021447..da902e883290a 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index 67b5d9f8829ec..cf5a979ebc941 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index bfdb03d4baffb..b9aa9d81c53d8 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index e43601eae9cbf..5aa00f1ac0d22 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.2 + 0.4.3-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index a22fa1763bca8..570a76924cd65 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.2 + 0.4.3-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - hoodie-0.4.2 + HEAD From fe513327d94ab95e707aa075a42f7888b07b7958 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 13 May 2018 16:25:11 -0700 Subject: [PATCH 062/374] Adding ability for inserts to be written to log files --- .../com/uber/hoodie/HoodieReadClient.java | 13 +- .../com/uber/hoodie/HoodieWriteClient.java | 82 +++---- .../java/com/uber/hoodie/WriteStatus.java | 12 +- .../hoodie/config/HoodieStorageConfig.java | 10 + .../uber/hoodie/config/HoodieWriteConfig.java | 6 +- .../hoodie/func/BulkInsertMapFunction.java | 2 +- ...ava => CopyOnWriteLazyInsertIterable.java} | 33 +-- .../func/MergeOnReadLazyInsertIterable.java | 73 +++++++ .../com/uber/hoodie/index/HoodieIndex.java | 22 +- .../uber/hoodie/index/InMemoryHashIndex.java | 10 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 44 ++-- .../bloom/HoodieBloomIndexCheckFunction.java | 16 +- .../hoodie/index/bucketed/BucketedIndex.java | 16 +- .../uber/hoodie/index/hbase/HBaseIndex.java | 26 +-- .../uber/hoodie/io/HoodieAppendHandle.java | 193 +++++++++++------ .../com/uber/hoodie/io/HoodieCleanHelper.java | 2 +- .../hoodie/io/HoodieCommitArchiveLog.java | 9 +- .../uber/hoodie/io/HoodieCreateHandle.java | 41 +++- .../com/uber/hoodie/io/HoodieIOHandle.java | 25 +++ .../com/uber/hoodie/io/HoodieMergeHandle.java | 5 +- .../io/compact/CompactionOperation.java | 34 +-- .../compact/HoodieRealtimeTableCompactor.java | 29 ++- .../compact/strategy/CompactionStrategy.java | 11 +- .../LogFileSizeBasedCompactionStrategy.java | 5 +- .../io/storage/HoodieParquetWriter.java | 2 +- .../storage/HoodieStorageWriterFactory.java | 11 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 50 +++-- .../hoodie/table/HoodieMergeOnReadTable.java | 86 ++++++-- .../com/uber/hoodie/table/HoodieTable.java | 65 ++---- .../java/com/uber/hoodie/TestCleaner.java | 62 +++--- .../com/uber/hoodie/TestClientRollback.java | 6 +- .../com/uber/hoodie/TestHoodieClientBase.java | 6 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 10 +- .../hoodie/common/HoodieClientTestUtils.java | 13 +- .../common/HoodieTestDataGenerator.java | 4 +- .../func/TestBoundedInMemoryExecutor.java | 6 +- .../hoodie/func/TestBoundedInMemoryQueue.java | 2 +- .../hoodie/func/TestUpdateMapFunction.java | 21 +- .../com/uber/hoodie/index/TestHbaseIndex.java | 51 +++-- .../uber/hoodie/index/TestHoodieIndex.java | 43 +++- .../index/bloom/TestHoodieBloomIndex.java | 63 +++--- .../hoodie/io/TestHoodieCommitArchiveLog.java | 32 ++- .../uber/hoodie/io/TestHoodieCompactor.java | 14 +- ...TestHoodieMergeHandleDuplicateRecords.java | 3 +- .../TestHoodieCompactionStrategy.java | 3 +- .../hoodie/table/TestCopyOnWriteTable.java | 22 +- .../hoodie/table/TestMergeOnReadTable.java | 202 +++++++++++++++--- .../common/table/HoodieTableMetaClient.java | 52 ++++- .../log/AbstractHoodieLogRecordScanner.java | 7 +- .../com/uber/hoodie/common/util/FSUtils.java | 7 + .../uber/hoodie/HoodieDataSourceHelpers.java | 11 +- .../com/uber/hoodie/IncrementalRelation.scala | 7 +- 52 files changed, 1061 insertions(+), 519 deletions(-) rename hoodie-client/src/main/java/com/uber/hoodie/func/{LazyInsertIterable.java => CopyOnWriteLazyInsertIterable.java} (85%) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 9a2bed3abfe10..8dac00573cea7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -94,10 +94,11 @@ public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { this.jsc = jsc; this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); // Create a Hoodie table which encapsulated the commits and files visible + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); this.hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), - clientConfig); - this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); + .getHoodieTable(metaClient, + clientConfig, jsc); + this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); this.index = HoodieIndex.createIndex(clientConfig, jsc); this.sqlContextOpt = Optional.absent(); } @@ -128,7 +129,7 @@ public Dataset read(JavaRDD hoodieKeys, int parallelism) throws assertSqlContext(); JavaPairRDD> keyToFileRDD = index - .fetchRecordLocation(hoodieKeys, hoodieTable); + .fetchRecordLocation(hoodieKeys, jsc, hoodieTable); List paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) .map(keyFileTuple -> keyFileTuple._2().get()).collect(); @@ -156,7 +157,7 @@ public Dataset read(JavaRDD hoodieKeys, int parallelism) throws * file */ public JavaPairRDD> checkExists(JavaRDD hoodieKeys) { - return index.fetchRecordLocation(hoodieKeys, hoodieTable); + return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable); } /** @@ -180,6 +181,6 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco */ public JavaRDD> tagLocation(JavaRDD> hoodieRecords) throws HoodieIndexException { - return index.tagLocation(hoodieRecords, hoodieTable); + return index.tagLocation(hoodieRecords, jsc, hoodieTable); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 16aba54513ab9..ef805a94f011e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -130,7 +130,7 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, public static SparkConf registerClasses(SparkConf conf) { conf.registerKryoClasses( - new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); + new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); return conf; } @@ -144,9 +144,9 @@ public static SparkConf registerClasses(SparkConf conf) { public JavaRDD> filterExists(JavaRDD> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); + JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, jsc, table); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); } @@ -161,7 +161,7 @@ public JavaRDD upsert(JavaRDD> records, final Strin config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism()); // perform index loop up to get existing location of records - JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, table); + JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, jsc, table); return upsertRecordsInternal(taggedRecords, commitTime, table, true); } catch (Throwable e) { if (e instanceof HoodieUpsertException) { @@ -178,7 +178,7 @@ public JavaRDD upsert(JavaRDD> records, final Strin * needed. * * @param preppedRecords Prepared HoodieRecords to upsert - * @param commitTime Commit Time handle + * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, @@ -202,7 +202,7 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe * This implementation skips the index check and is able to leverage benefits such as small file * handling/blocking alignment, as with upsert(), by profiling the workload * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ @@ -230,7 +230,7 @@ public JavaRDD insert(JavaRDD> records, final Strin * The prepared HoodieRecords should be de-duped if needed. * * @param preppedRecords HoodieRecords to insert - * @param commitTime Commit Time handle + * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, @@ -256,7 +256,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)} * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ @@ -276,10 +276,10 @@ public JavaRDD bulkInsert(JavaRDD> records, * partitioner. If specified then it will be used for repartitioning records. See {@link * UserDefinedBulkInsertPartitioner}. * - * @param records HoodieRecords to insert - * @param commitTime Commit Time handle + * @param records HoodieRecords to insert + * @param commitTime Commit Time handle * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. + * before they are inserted into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD bulkInsert(JavaRDD> records, final String commitTime, @@ -310,10 +310,10 @@ public JavaRDD bulkInsert(JavaRDD> records, final S * partitioner. If specified then it will be used for repartitioning records. See {@link * UserDefinedBulkInsertPartitioner}. * - * @param preppedRecords HoodieRecords to insert - * @param commitTime Commit Time handle + * @param preppedRecords HoodieRecords to insert + * @param commitTime Commit Time handle * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. + * before they are inserted into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, @@ -390,6 +390,7 @@ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, Hood partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> { HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(entry.getKey()); + // TODO : Write baseCommitTime is possible here ? writeStat.setPrevCommit(entry.getValue().getKey()); writeStat.setNumUpdateWrites(entry.getValue().getValue()); metadata.addWriteStat(path.toString(), writeStat); @@ -450,10 +451,12 @@ private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, Workload private JavaRDD updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieTable table, String commitTime) { // Update the index back - JavaRDD statuses = index.updateLocation(writeStatusRDD, table); + JavaRDD statuses = index.updateLocation(writeStatusRDD, jsc, table); // Trigger the insert and collect statuses statuses = statuses.persist(config.getWriteStatusStorageLevel()); - commitOnAutoCommit(commitTime, statuses, table.getCommitActionType()); + commitOnAutoCommit(commitTime, statuses, + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true) + .getCommitActionType()); return statuses; } @@ -476,9 +479,8 @@ public boolean commit(String commitTime, JavaRDD writeStatuses) { */ public boolean commit(String commitTime, JavaRDD writeStatuses, Optional> extraMetadata) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType()); } private boolean commit(String commitTime, JavaRDD writeStatuses, @@ -487,7 +489,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); @@ -531,7 +533,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, // We cannot have unbounded commit files. Archive commits if we have to archive HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); - archiveLog.archiveIfRequired(); + archiveLog.archiveIfRequired(jsc); if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, logger.info("Auto cleaning is enabled. Running cleaner now"); @@ -568,13 +570,13 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, *

* Savepoint should be on a commit that could not have been cleaned. * - * @param user - User creating the savepoint + * @param user - User creating the savepoint * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); if (table.getCompletedCommitTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -595,13 +597,13 @@ public boolean savepoint(String user, String comment) { * Savepoint should be on a commit that could not have been cleaned. * * @param commitTime - commit that should be savepointed - * @param user - User creating the savepoint - * @param comment - Comment for the savepoint + * @param user - User creating the savepoint + * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String commitTime, String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, @@ -662,7 +664,7 @@ public boolean savepoint(String commitTime, String user, String comment) { */ public void deleteSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, @@ -688,9 +690,9 @@ public void deleteSavepoint(String savepointTime) { */ public boolean rollbackToSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline commitTimeline = table.getCommitsTimeline(); + HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); @@ -737,7 +739,7 @@ private void rollback(List commits) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); @@ -841,7 +843,7 @@ private void clean(String startCleanTime) throws HoodieIOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); List cleanStats = table.clean(jsc); if (cleanStats.isEmpty()) { @@ -890,9 +892,9 @@ public String startCommit() { public void startCommitWithTime(String commitTime) { logger.info("Generate a new commit time " + commitTime); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = table.getCommitActionType(); + String commitActionType = table.getMetaClient().getCommitActionType(); activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); } @@ -912,7 +914,7 @@ public String startCompaction() { */ public void startCompactionWithTime(String commitTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = HoodieTimeline.COMMIT_ACTION; activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); @@ -925,7 +927,7 @@ public void startCompactionWithTime(String commitTime) { public JavaRDD compact(String commitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction writeContext = metrics.getCommitCtx(); JavaRDD statuses = table.compact(jsc, commitTime); @@ -961,7 +963,7 @@ private void forceCompact(String compactionCommitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and // then use getTableAndInitCtx Timer.Context writeContext = metrics.getCommitCtx(); @@ -1048,8 +1050,8 @@ JavaRDD> deduplicateRecords(JavaRDD> records, */ private void rollbackInflightCommits() { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); Collections.reverse(commits); @@ -1061,8 +1063,8 @@ private void rollbackInflightCommits() { private HoodieTable getTableAndInitCtx() { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - if (table.getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { writeContext = metrics.getCommitCtx(); } else { writeContext = metrics.getDeltaCommitCtx(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java index c28db2567b8c0..14a01775386d7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java @@ -53,10 +53,10 @@ public class WriteStatus implements Serializable { * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * objects are collected in Spark Driver. * - * @param record deflated {@code HoodieRecord} containing information that uniquely identifies - * it. + * @param record deflated {@code HoodieRecord} containing information that uniquely identifies + * it. * @param optionalRecordMetadata optional metadata related to data contained in {@link - * HoodieRecord} before deflation. + * HoodieRecord} before deflation. */ public void markSuccess(HoodieRecord record, Optional> optionalRecordMetadata) { @@ -69,10 +69,10 @@ public void markSuccess(HoodieRecord record, * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * objects are collected in Spark Driver. * - * @param record deflated {@code HoodieRecord} containing information that uniquely identifies - * it. + * @param record deflated {@code HoodieRecord} containing information that uniquely identifies + * it. * @param optionalRecordMetadata optional metadata related to data contained in {@link - * HoodieRecord} before deflation. + * HoodieRecord} before deflation. */ public void markFailure(HoodieRecord record, Throwable t, Optional> optionalRecordMetadata) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java index b72e6f7434530..0403b62ebb465 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java @@ -44,6 +44,9 @@ public class HoodieStorageConfig extends DefaultHoodieConfig { public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio"; // Default compression ratio for parquet public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1); + public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio"; + // Default compression ratio for log file to parquet, general 3x + public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35); private HoodieStorageConfig(Properties props) { super(props); @@ -102,6 +105,11 @@ public Builder parquetCompressionRatio(double parquetCompressionRatio) { return this; } + public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) { + props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio)); + return this; + } + public HoodieStorageConfig build() { HoodieStorageConfig config = new HoodieStorageConfig(props); setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), @@ -116,6 +124,8 @@ public HoodieStorageConfig build() { LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES); setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO), PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO); + setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO), + LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO); return config; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index ebec1dd7b8186..c974d2892f696 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -322,6 +322,10 @@ public double getParquetCompressionRatio() { return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO)); } + public double getLogFileToParquetCompressionRatio() { + return Double.valueOf(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO)); + } + /** * metrics properties **/ @@ -345,7 +349,7 @@ public int getGraphiteServerPort() { public String getGraphiteMetricPrefix() { return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX); } - + /** * memory configs */ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java index c8a39b6857cd5..e530e86bd04df 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java @@ -46,6 +46,6 @@ public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config, @Override public Iterator> call(Integer partition, Iterator> sortedRecordItr) throws Exception { - return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable); + return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java similarity index 85% rename from hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java rename to hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java index 6fdf41c3c514b..5311f07a2bfaf 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -43,15 +44,15 @@ * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new * files. */ -public class LazyInsertIterable extends +public class CopyOnWriteLazyInsertIterable extends LazyIterableIterator, List> { - private final HoodieWriteConfig hoodieConfig; - private final String commitTime; - private final HoodieTable hoodieTable; - private Set partitionsCleaned; + protected final HoodieWriteConfig hoodieConfig; + protected final String commitTime; + protected final HoodieTable hoodieTable; + protected Set partitionsCleaned; - public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, + public CopyOnWriteLazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { super(sortedRecordItr); this.partitionsCleaned = new HashSet<>(); @@ -89,7 +90,7 @@ protected List computeNext() { final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); bufferedIteratorExecutor = new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, - new InsertHandler(), getTransformFunction(schema)); + getInsertHandler(), getTransformFunction(schema)); final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); return result; @@ -107,15 +108,19 @@ protected void end() { } + protected CopyOnWriteInsertHandler getInsertHandler() { + return new CopyOnWriteInsertHandler(); + } + /** * Consumes stream of hoodie records from in-memory queue and * writes to one or more create-handles */ - private class InsertHandler extends + protected class CopyOnWriteInsertHandler extends BoundedInMemoryQueueConsumer, Optional>, List> { - private final List statuses = new ArrayList<>(); - private HoodieCreateHandle handle; + protected final List statuses = new ArrayList<>(); + protected HoodieIOHandle handle; @Override protected void consumeOneRecord(Tuple2, Optional> payload) { @@ -132,7 +137,8 @@ protected void consumeOneRecord(Tuple2, Optional> // lazily initialize the handle, for the first time if (handle == null) { - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID + .randomUUID().toString()); } if (handle.canWrite(payload._1())) { @@ -142,7 +148,8 @@ protected void consumeOneRecord(Tuple2, Optional> // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID + .randomUUID().toString()); handle.write(insertPayload, payload._2()); // we should be able to write 1 payload. } } @@ -161,4 +168,4 @@ protected List getResult() { return statuses; } } -} +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java new file mode 100644 index 0000000000000..a6b5889766a78 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java @@ -0,0 +1,73 @@ +/* + * 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.func; + +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.io.HoodieAppendHandle; +import com.uber.hoodie.table.HoodieTable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import org.apache.avro.generic.IndexedRecord; +import scala.Tuple2; + +/** + * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new + * log files. + */ +public class MergeOnReadLazyInsertIterable extends + CopyOnWriteLazyInsertIterable { + + public MergeOnReadLazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, + String commitTime, HoodieTable hoodieTable) { + super(sortedRecordItr, config, commitTime, hoodieTable); + } + + @Override + protected CopyOnWriteInsertHandler getInsertHandler() { + return new MergeOnReadInsertHandler(); + } + + protected class MergeOnReadInsertHandler extends CopyOnWriteInsertHandler { + + @Override + protected void consumeOneRecord(Tuple2, Optional> payload) { + final HoodieRecord insertPayload = payload._1(); + List statuses = new ArrayList<>(); + // lazily initialize the handle, for the first time + if (handle == null) { + handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable); + } + if (handle.canWrite(insertPayload)) { + // write the payload, if the handle has capacity + handle.write(insertPayload, payload._2); + } else { + // handle is full. + handle.close(); + statuses.add(handle.getWriteStatus()); + // Need to handle the rejected payload & open new handle + handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable); + handle.write(insertPayload, payload._2); // we should be able to write 1 payload. + } + } + } + +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java index 5733237b44c79..bc7dc7eb48a4c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java @@ -38,24 +38,23 @@ public abstract class HoodieIndex implements Serializable { protected final HoodieWriteConfig config; - protected transient JavaSparkContext jsc = null; - protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) { + protected HoodieIndex(HoodieWriteConfig config) { this.config = config; - this.jsc = jsc; } + public static HoodieIndex createIndex(HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException { switch (config.getIndexType()) { case HBASE: - return new HBaseIndex<>(config, jsc); + return new HBaseIndex<>(config); case INMEMORY: - return new InMemoryHashIndex<>(config, jsc); + return new InMemoryHashIndex<>(config); case BLOOM: - return new HoodieBloomIndex<>(config, jsc); + return new HoodieBloomIndex<>(config); case BUCKETED: - return new BucketedIndex<>(config, jsc); + return new BucketedIndex<>(config); default: throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); } @@ -68,22 +67,23 @@ public static HoodieIndex createIndex(HoodieW * file */ public abstract JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, final HoodieTable table); + JavaRDD hoodieKeys, final JavaSparkContext jsc, HoodieTable hoodieTable); /** * Looks up the index and tags each incoming record with a location of a file that contains the * row (if it is actually present) */ public abstract JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) throws HoodieIndexException; + JavaSparkContext jsc, HoodieTable hoodieTable) throws HoodieIndexException; /** * Extracts the location of written records, and updates the index. *

* TODO(vc): We may need to propagate the record as well in a WriteStatus class */ - public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) throws HoodieIndexException; + public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTable hoodieTable) + throws HoodieIndexException; /** * Rollback the efffects of the commit made at commitTime. diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java index 6338e60bb1a14..a3d0429ea6719 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java @@ -43,25 +43,25 @@ public class InMemoryHashIndex extends HoodieInde private static ConcurrentMap recordLocationMap; - public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) { - super(config, jsc); + public InMemoryHashIndex(HoodieWriteConfig config) { + super(config); recordLocationMap = new ConcurrentHashMap<>(); } @Override public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - final HoodieTable table) { + JavaSparkContext jsc, HoodieTable hoodieTable) { throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true); } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { return writeStatusRDD.map(new Function() { @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index b85f0fed979f9..b0f37e8ac3ce5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ParquetUtils; @@ -64,13 +65,13 @@ public class HoodieBloomIndex extends HoodieIndex private static int MAX_ITEMS_PER_SHUFFLE_PARTITION = SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET; - public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) { - super(config, jsc); + public HoodieBloomIndex(HoodieWriteConfig config) { + super(config); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - final HoodieTable hoodieTable) { + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, + HoodieTable hoodieTable) { // Step 0: cache the input record RDD if (config.getBloomIndexUseCaching()) { @@ -82,8 +83,7 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())); // Lookup indexes for all the partition/recordkey pair - JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, - hoodieTable); + JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable); // Cache the result, for subsequent stages. if (config.getBloomIndexUseCaching()) { @@ -108,13 +108,12 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, } public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - final HoodieTable table) { + JavaSparkContext jsc, HoodieTable hoodieTable) { JavaPairRDD partitionRecordKeyPairRDD = hoodieKeys .mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); // Lookup indexes for all the partition/recordkey pair - JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, - table); + JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable); JavaPairRDD rowKeyHoodieKeyPairRDD = hoodieKeys .mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); @@ -125,7 +124,7 @@ public JavaPairRDD> fetchRecordLocation(JavaRDD> fetchRecordLocation(JavaRDD lookupIndex( - JavaPairRDD partitionRecordKeyPairRDD, final HoodieTable hoodieTable) { + JavaPairRDD partitionRecordKeyPairRDD, final JavaSparkContext + jsc, final HoodieTable hoodieTable) { // Obtain records per partition, in the incoming records Map recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); // Step 2: Load all involved files as pairs - List> fileInfoList = loadInvolvedFiles( - affectedPartitionPathList, hoodieTable); + List> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc, + hoodieTable); final Map> partitionToFileInfo = fileInfoList.stream() .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); @@ -154,8 +154,8 @@ private JavaPairRDD lookupIndex( // that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); - return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo, - partitionRecordKeyPairRDD, parallelism); + return findMatchingFilesForRecordKeys(partitionToFileInfo, + partitionRecordKeyPairRDD, parallelism, hoodieTable.getMetaClient()); } /** @@ -232,12 +232,12 @@ private int determineParallelism(int inputParallelism, int totalSubPartitions) { * Load all involved files as pair RDD. */ @VisibleForTesting - List> loadInvolvedFiles(List partitions, - final HoodieTable hoodieTable) { + List> loadInvolvedFiles(List partitions, final JavaSparkContext jsc, + final HoodieTable hoodieTable) { // Obtain the latest data files from all the partitions. List> dataFilesList = jsc .parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> { - java.util.Optional latestCommitTime = hoodieTable.getCommitsTimeline() + java.util.Optional latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline() .filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { @@ -352,9 +352,9 @@ JavaPairRDD> explodeRecordRDDWithFileCompariso * Make sure the parallelism is atleast the groupby parallelism for tagging location */ @VisibleForTesting - JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTable, + JavaPairRDD findMatchingFilesForRecordKeys( final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions) { + JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) { int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions); @@ -367,7 +367,7 @@ JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTab .sortByKey(true, joinParallelism); return fileSortedTripletRDD.mapPartitionsWithIndex( - new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) + new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true) .flatMap(indexLookupResults -> indexLookupResults.iterator()) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { @@ -410,7 +410,7 @@ record = new HoodieRecord(record.getKey(), record.getData()); } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { return writeStatusRDD; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index a52582455f823..9c2519cb0de1f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -20,11 +20,11 @@ import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.func.LazyIterableIterator; -import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -49,10 +49,10 @@ public class HoodieBloomIndexCheckFunction implements private final String basePath; - private final HoodieTable table; + private final HoodieTableMetaClient metaClient; - public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) { - this.table = table; + public HoodieBloomIndexCheckFunction(HoodieTableMetaClient metaClient, String basePath) { + this.metaClient = metaClient; this.basePath = basePath; } @@ -115,7 +115,7 @@ private void initState(String fileName, String partitionPath) throws HoodieIndex try { Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName); bloomFilter = ParquetUtils - .readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath); + .readBloomFilterFromParquetMetadata(metaClient.getHadoopConf(), filePath); candidateRecordKeys = new ArrayList<>(); currentFile = fileName; currentParitionPath = partitionPath; @@ -163,7 +163,7 @@ protected List computeNext() { .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); } ret.add(new IndexLookupResult(currentFile, - checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); + checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath))); initState(fileName, partitionPath); if (bloomFilter.mightContain(recordKey)) { @@ -176,7 +176,7 @@ protected List computeNext() { } } - // handle case, where we ran out of input, finish pending work, update return val + // handle case, where we ran out of input, close pending work, update return val if (!inputItr.hasNext()) { Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile); logger.info( @@ -186,7 +186,7 @@ protected List computeNext() { logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); } ret.add(new IndexLookupResult(currentFile, - checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); + checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath))); } } catch (Throwable e) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java index a8e3921730c66..dcb84add464a2 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java @@ -49,8 +49,8 @@ public class BucketedIndex extends HoodieIndex private static Logger logger = LogManager.getLogger(BucketedIndex.class); - public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) { - super(config, jsc); + public BucketedIndex(HoodieWriteConfig config) { + super(config); } private String getBucket(String recordKey) { @@ -59,13 +59,14 @@ private String getBucket(String recordKey) { @Override public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - HoodieTable table) { + JavaSparkContext jsc, HoodieTable hoodieTable) { return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey())))); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) throws HoodieIndexException { + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, + HoodieTable hoodieTable) + throws HoodieIndexException { return recordRDD.map(record -> { String bucket = getBucket(record.getRecordKey()); //HACK(vc) a non-existent commit is provided here. @@ -75,8 +76,9 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) throws HoodieIndexException { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTable hoodieTable) + throws HoodieIndexException { return writeStatusRDD; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java index 4928d144055db..e929b81d0dcf6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +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.config.HoodieWriteConfig; @@ -69,15 +70,15 @@ public class HBaseIndex extends HoodieIndex { private static Connection hbaseConnection = null; private final String tableName; - public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { - super(config, jsc); + public HBaseIndex(HoodieWriteConfig config) { + super(config); this.tableName = config.getHbaseTableName(); addShutDownHook(); } @Override public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - HoodieTable table) { + JavaSparkContext jsc, HoodieTable hoodieTable) { //TODO : Change/Remove filterExists in HoodieReadClient() and revisit throw new UnsupportedOperationException("HBase index does not implement check exist"); } @@ -119,8 +120,8 @@ private Get generateStatement(String key) throws IOException { .addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN); } - private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) { - HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); + private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) { + HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); // Check if the last commit ts for this row is 1) present in the timeline or // 2) is less than the first commit ts in the timeline return !commitTimeline.empty() && (commitTimeline @@ -133,8 +134,8 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) /** * Function that tags each HoodieRecord with an existing location, if known. */ - private Function2>, Iterator>> - locationTagFunction(HoodieTable hoodieTable) { + private Function2>, + Iterator>> locationTagFunction(HoodieTableMetaClient metaClient) { return (Function2>, Iterator>>) (partitionNum, hoodieRecordIterator) -> { @@ -176,7 +177,7 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) String partitionPath = Bytes .toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - if (checkIfValidCommit(hoodieTable, commitTs)) { + if (checkIfValidCommit(metaClient, commitTs)) { currentRecord = new HoodieRecord( new HoodieKey(currentRecord.getRecordKey(), partitionPath), currentRecord.getData()); @@ -211,13 +212,12 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { - return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true); + return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true); } - private Function2, Iterator> - updateLocationFunction() { + private Function2, Iterator> updateLocationFunction() { return (Function2, Iterator>) (partition, statusIterator) -> { Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); @@ -306,7 +306,7 @@ private void doPutsAndDeletes(HTable hTable, List puts, List delete } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 7ead846deacf9..d53c19bc2db0a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -31,7 +31,6 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -45,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -60,21 +60,35 @@ public class HoodieAppendHandle extends HoodieIOHandle { private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); + // This acts as the sequenceID for records written private static AtomicLong recordIndex = new AtomicLong(1); private final WriteStatus writeStatus; private final String fileId; + // Buffer for holding records in memory before they are flushed to disk List recordList = new ArrayList<>(); + // Buffer for holding records (to be deleted) in memory before they are flushed to disk List keysToDelete = new ArrayList<>(); private TableFileSystemView.RealtimeView fileSystemView; private String partitionPath; private Iterator> recordItr; + // Total number of records written during an append private long recordsWritten = 0; + // Total number of records deleted during an append private long recordsDeleted = 0; + // Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk private long averageRecordSize = 0; private HoodieLogFile currentLogFile; private Writer writer; + // Flag used to initialize some metadata private boolean doInit = true; + // Total number of bytes written during this append phase (an estimation) private long estimatedNumberOfBytesWritten; + // Number of records that must be written to meet the max block size for a log block + private int numberOfRecords = 0; + // Max block size to limit to for a log block + private int maxBlockSize = config.getLogFileDataBlockMaxSize(); + // Header metadata for a log block + private Map header = Maps.newHashMap(); public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String fileId, Iterator> recordItr) { @@ -87,43 +101,46 @@ public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTab this.recordItr = recordItr; } - private void init(String partitionPath) { - - // extract some information from the first record - FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst() - .get(); - // HACK(vc) This also assumes a base file. It will break, if appending without one. - String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); - String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); - writeStatus.getStat().setPrevCommit(baseCommitTime); - writeStatus.setFileId(fileId); - writeStatus.setPartitionPath(partitionPath); - writeStatus.getStat().setFileId(fileId); - this.partitionPath = partitionPath; + public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { + this(config, commitTime, hoodieTable, UUID.randomUUID().toString(), null); + } - try { - this.writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) - .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion( - fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion()) - .max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) - .withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - this.currentLogFile = writer.getLogFile(); - ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); - ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); - } catch (Exception e) { - logger.error("Error in update task at commit " + commitTime, e); - writeStatus.setGlobalError(e); - throw new HoodieUpsertException( - "Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " - + commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() - + partitionPath, e); + private void init(HoodieRecord record) { + if (doInit) { + this.partitionPath = record.getPartitionPath(); + // extract some information from the first record + Optional fileSlice = fileSystemView.getLatestFileSlices(partitionPath) + .filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst(); + String baseInstantTime = commitTime; + if (fileSlice.isPresent()) { + baseInstantTime = fileSlice.get().getBaseCommitTime(); + } else { + // This means there is no base data file, start appending to a new log file + fileSlice = Optional.of(new FileSlice(baseInstantTime, this.fileId)); + logger.info("New InsertHandle for partition :" + partitionPath); + } + writeStatus.getStat().setPrevCommit(baseInstantTime); + writeStatus.setFileId(fileId); + writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setFileId(fileId); + averageRecordSize = SizeEstimator.estimate(record); + try { + this.writer = createLogWriter(fileSlice, baseInstantTime); + this.currentLogFile = writer.getLogFile(); + ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); + ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); + } catch (Exception e) { + logger.error("Error in update task at commit " + commitTime, e); + writeStatus.setGlobalError(e); + throw new HoodieUpsertException( + "Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit " + + commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + + partitionPath, e); + } + Path path = new Path(partitionPath, writer.getLogFile().getFileName()); + writeStatus.getStat().setPath(path.toString()); + doInit = false; } - Path path = new Path(partitionPath, - FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); - writeStatus.getStat().setPath(path.toString()); } private Optional getIndexedRecord(HoodieRecord hoodieRecord) { @@ -160,38 +177,11 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { // TODO (NA) - Perform a schema check of current input record with the last schema on log file // to make sure we don't append records with older (shorter) schema than already appended public void doAppend() { - - int maxBlockSize = config.getLogFileDataBlockMaxSize(); - int numberOfRecords = 0; - Map header = Maps.newHashMap(); - header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime); - header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); while (recordItr.hasNext()) { HoodieRecord record = recordItr.next(); - // update the new location of the record, so we know where to find it next - record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); - if (doInit) { - init(record.getPartitionPath()); - averageRecordSize = SizeEstimator.estimate(record); - doInit = false; - } - // Append if max number of records reached to achieve block size - if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { - // Recompute averageRecordSize before writing a new block and update existing value with - // avg of new and old - logger.info("AvgRecordSize => " + averageRecordSize); - averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2; - doAppend(header); - estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; - numberOfRecords = 0; - } - Optional indexedRecord = getIndexedRecord(record); - if (indexedRecord.isPresent()) { - recordList.add(indexedRecord.get()); - } else { - keysToDelete.add(record.getRecordKey()); - } - numberOfRecords++; + init(record); + flushToDiskIfRequired(record); + writeToBuffer(record); } doAppend(header); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; @@ -199,6 +189,8 @@ public void doAppend() { private void doAppend(Map header) { try { + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); if (recordList.size() > 0) { writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header)); recordList.clear(); @@ -214,11 +206,37 @@ private void doAppend(Map header) { } } - public void close() { + @Override + public boolean canWrite(HoodieRecord record) { + return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config + .getLogFileToParquetCompressionRatio(); + } + + @Override + public void write(HoodieRecord record, Optional insertValue) { + Optional recordMetadata = record.getData().getMetadata(); try { + init(record); + flushToDiskIfRequired(record); + writeToBuffer(record); + } catch (Throwable t) { + // Not throwing exception from here, since we don't want to fail the entire job + // for a single record + writeStatus.markFailure(record, t, recordMetadata); + logger.error("Error writing record " + record, t); + } + } + + @Override + public WriteStatus close() { + try { + // flush any remaining records to disk + doAppend(header); if (writer != null) { writer.close(); } + writeStatus.getStat().setPrevCommit(commitTime); + writeStatus.getStat().setFileId(this.fileId); writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten); @@ -226,13 +244,54 @@ public void close() { RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalUpsertTime(timer.endTimer()); writeStatus.getStat().setRuntimeStats(runtimeStats); + return writeStatus; } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } } + @Override public WriteStatus getWriteStatus() { return writeStatus; } + private Writer createLogWriter(Optional fileSlice, String baseCommitTime) + throws IOException, InterruptedException { + return HoodieLogFormat.newWriterBuilder() + .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) + .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion( + fileSlice.get().getLogFiles().map(logFile -> logFile.getLogVersion()) + .max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) + .withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + } + + private void writeToBuffer(HoodieRecord record) { + // update the new location of the record, so we know where to find it next + record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); + Optional indexedRecord = getIndexedRecord(record); + if (indexedRecord.isPresent()) { + recordList.add(indexedRecord.get()); + } else { + keysToDelete.add(record.getRecordKey()); + } + numberOfRecords++; + } + + /** + * Checks if the number of records have reached the set threshold and then flushes the records to disk + */ + private void flushToDiskIfRequired(HoodieRecord record) { + // Append if max number of records reached to achieve block size + if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { + // Recompute averageRecordSize before writing a new block and update existing value with + // avg of new and old + logger.info("AvgRecordSize => " + averageRecordSize); + averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2; + doAppend(header); + estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; + numberOfRecords = 0; + } + } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index c748dfe849ab7..2f7006fe4e57f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -38,7 +38,7 @@ /** * Cleaner is responsible for garbage collecting older files in a given partition path, such that - *

1) It provides sufficient time for existing queries running on older versions, to finish

+ *

1) It provides sufficient time for existing queries running on older versions, to close

* 2) It bounds the growth of the files in the file system

TODO: Should all cleaning be done * based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata} */ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 10d9f94cc0019..bf78467ffe384 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -52,6 +52,7 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; /** * Archiver to bound the growth of .commit files @@ -99,9 +100,9 @@ private void close() { /** * Check if commits need to be archived. If yes, archive commits. */ - public boolean archiveIfRequired() { + public boolean archiveIfRequired(final JavaSparkContext jsc) { try { - List instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); + List instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList()); boolean success = true; if (instantsToArchive.iterator().hasNext()) { this.writer = openWriter(); @@ -117,13 +118,13 @@ public boolean archiveIfRequired() { } } - private Stream getInstantsToArchive() { + private Stream getInstantsToArchive(JavaSparkContext jsc) { // TODO : rename to max/minInstantsToKeep int maxCommitsToKeep = config.getMaxCommitsToKeep(); int minCommitsToKeep = config.getMinCommitsToKeep(); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); // GroupBy each action and limit each action timeline to maxCommitsToKeep // TODO: Handle ROLLBACK_ACTION in future diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 0384db71c6808..e05e34d2fef39 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -31,8 +31,8 @@ import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.util.Iterator; import java.util.Optional; -import java.util.UUID; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -49,12 +49,13 @@ public class HoodieCreateHandle extends HoodieIOH private Path tempPath = null; private long recordsWritten = 0; private long recordsDeleted = 0; + private Iterator> recordIterator; public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, - String partitionPath) { + String partitionPath, String fileId) { super(config, commitTime, hoodieTable); this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName()); - status.setFileId(UUID.randomUUID().toString()); + status.setFileId(fileId); status.setPartitionPath(partitionPath); final int sparkPartitionId = TaskContext.getPartitionId(); @@ -77,12 +78,13 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTab logger.info("New InsertHandle for partition :" + partitionPath); } - /** - * Determines whether we can accept the incoming records, into the current file, depending on - *

- * - Whether it belongs to the same partitionPath as existing records - Whether the current file - * written bytes lt max file size - */ + public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Iterator> recordIterator) { + this(config, commitTime, hoodieTable, partitionPath, fileId); + this.recordIterator = recordIterator; + } + + @Override public boolean canWrite(HoodieRecord record) { return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath()); } @@ -114,9 +116,30 @@ public void write(HoodieRecord record, Optional avroRecord) { } } + /** + * Writes all records passed + */ + public void write() { + try { + while (recordIterator.hasNext()) { + HoodieRecord record = recordIterator.next(); + write(record, record.getData().getInsertValue(schema)); + } + } catch (IOException io) { + throw new HoodieInsertException( + "Failed to insert records for path " + getStorageWriterPath(), io); + } + } + + @Override + public WriteStatus getWriteStatus() { + return status; + } + /** * Performs actions to durably, persist the current changes and returns a WriteStatus object */ + @Override public WriteStatus close() { logger.info("Closing the file " + status.getFileId() + " as we are done with all the records " + recordsWritten); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index e4a43af1998b0..3e3318161ceb4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -16,6 +16,8 @@ package com.uber.hoodie.io; +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -26,7 +28,9 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.util.Optional; import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -103,4 +107,25 @@ public Path makeTempPath(String partitionPath, int taskPartitionId, String fileN public Schema getSchema() { return schema; } + + /** + * Determines whether we can accept the incoming records, into the current file, depending on + *

+ * - Whether it belongs to the same partitionPath as existing records - Whether the current file + * written bytes lt max file size + */ + public boolean canWrite(HoodieRecord record) { + return false; + } + + /** + * Perform the actual writing of the given record into the backing file. + */ + public void write(HoodieRecord record, Optional insertValue) { + // NO_OP + } + + public abstract WriteStatus close(); + + public abstract WriteStatus getWriteStatus(); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 679d5fa8e1cc1..fbc2b2bf5b8e8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -243,7 +243,8 @@ public void write(GenericRecord oldRecord) { } } - public void close() { + @Override + public WriteStatus close() { try { // write out any pending records (this can happen when inserts are turned into updates) Iterator pendingRecordsItr = keyToNewRecords.keySet().iterator(); @@ -269,6 +270,7 @@ public void close() { RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalUpsertTime(timer.endTimer()); writeStatus.getStat().setRuntimeStats(runtimeStats); + return writeStatus; } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } @@ -283,6 +285,7 @@ private Path getStorageWriterPath() { return (this.tempPath == null) ? this.newFilePath : this.tempPath; } + @Override public WriteStatus getWriteStatus() { return writeStatus; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java index 8180f7dc18a43..3d5ffc40cb38f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java @@ -18,11 +18,13 @@ import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -33,10 +35,10 @@ */ public class CompactionOperation implements Serializable { - private String dataFileCommitTime; - private long dataFileSize; + private Optional dataFileCommitTime; + private Optional dataFileSize; private List deltaFilePaths; - private String dataFilePath; + private Optional dataFilePath; private String fileId; private String partitionPath; private Map metrics; @@ -46,24 +48,32 @@ public class CompactionOperation implements Serializable { public CompactionOperation() { } - public CompactionOperation(HoodieDataFile dataFile, String partitionPath, + public CompactionOperation(Optional dataFile, String partitionPath, List logFiles, HoodieWriteConfig writeConfig) { - this.dataFilePath = dataFile.getPath(); - this.fileId = dataFile.getFileId(); + if (dataFile.isPresent()) { + this.dataFilePath = Optional.of(dataFile.get().getPath()); + this.fileId = dataFile.get().getFileId(); + this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime()); + this.dataFileSize = Optional.of(dataFile.get().getFileSize()); + } else { + assert logFiles.size() > 0; + this.dataFilePath = Optional.empty(); + this.fileId = FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()); + this.dataFileCommitTime = Optional.empty(); + this.dataFileSize = Optional.empty(); + } this.partitionPath = partitionPath; - this.dataFileCommitTime = dataFile.getCommitTime(); - this.dataFileSize = dataFile.getFileSize(); this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) .collect(Collectors.toList()); this.metrics = writeConfig.getCompactionStrategy() - .captureMetrics(dataFile, partitionPath, logFiles); + .captureMetrics(writeConfig, dataFile, partitionPath, logFiles); } - public String getDataFileCommitTime() { + public Optional getDataFileCommitTime() { return dataFileCommitTime; } - public long getDataFileSize() { + public Optional getDataFileSize() { return dataFileSize; } @@ -71,7 +81,7 @@ public List getDeltaFilePaths() { return deltaFilePaths; } - public String getDataFilePath() { + public Optional getDataFilePath() { return dataFilePath; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index e3d00f75e4a34..fc03b6cad8737 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -86,16 +86,19 @@ public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig conf private JavaRDD executeCompaction(JavaSparkContext jsc, List operations, HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) throws IOException { - + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + // Compacting is very similar to applying updates to existing file + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); log.info("After filtering, Compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) - .map(s -> compact(hoodieTable, config, s, compactionCommitTime)) + .map(s -> compact(table, metaClient, config, s, compactionCommitTime)) .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); } - private List compact(HoodieTable hoodieTable, HoodieWriteConfig config, + private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient, + HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException { - FileSystem fs = hoodieTable.getMetaClient().getFs(); + FileSystem fs = metaClient.getFs(); Schema readerSchema = HoodieAvroUtils .addMetadataFields(new Schema.Parser().parse(config.getSchema())); @@ -107,7 +110,6 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con // Load all the delta commits since the last compaction commit and get all the blocks to be // loaded and load it using CompositeAvroLogReader // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon. - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); String maxInstantTime = metaClient.getActiveTimeline() .getTimelineOfActions( Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, @@ -125,9 +127,16 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con } // Compacting is very similar to applying updates to existing file - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); - Iterator> result = table - .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); + Iterator> result; + // If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a + // new base parquet file. + if (operation.getDataFilePath().isPresent()) { + result = hoodieCopyOnWriteTable + .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); + } else { + result = hoodieCopyOnWriteTable + .handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator()); + } Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { @@ -176,7 +185,7 @@ private List getCompactionWorkload(JavaSparkContext jsc, .getLogVersionComparator().reversed()).collect(Collectors.toList()); totalLogFiles.add((long) logFiles.size()); totalFileSlices.add(1L); - return new CompactionOperation(s.getDataFile().get(), partitionPath, logFiles, config); + return new CompactionOperation(s.getDataFile(), partitionPath, logFiles, config); }) .filter(c -> !c.getDeltaFilePaths().isEmpty()) .collect(toList()).iterator()).collect(); @@ -195,4 +204,4 @@ private List getCompactionWorkload(JavaSparkContext jsc, return operations; } -} +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index 75d62e803368e..2a7e659545b45 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -54,16 +54,19 @@ public abstract class CompactionStrategy implements Serializable { * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, - List logFiles) { + public Map captureMetrics(HoodieWriteConfig writeConfig, Optional dataFile, String + partitionPath, List logFiles) { Map metrics = Maps.newHashMap(); + Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); // Total size of all the log files Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent) .map(Optional::get).reduce((size1, size2) -> size1 + size2).orElse(0L); // Total read will be the base file + all the log files - Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize); + Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L) + + totalLogFileSize); // Total write will be similar to the size of the base file - Long totalIOWrite = FSUtils.getSizeInMB(dataFile.getFileSize()); + Long totalIOWrite = FSUtils + .getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize); // Total IO will the the IO for read + write Long totalIO = totalIORead + totalIOWrite; // Save these metrics and we will use during the filter diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java index a7f729e36cf9b..39f66b2fe528c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -39,10 +39,11 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; @Override - public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, + public Map captureMetrics(HoodieWriteConfig config, Optional dataFile, String + partitionPath, List logFiles) { - Map metrics = super.captureMetrics(dataFile, partitionPath, logFiles); + Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); // Total size of all the log files Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java index edd2c062038e5..bc8b491703e23 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java @@ -72,7 +72,7 @@ public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig par this.schema = schema; } - private static Configuration registerFileSystem(Path file, Configuration conf) { + public static Configuration registerFileSystem(Path file, Configuration conf) { Configuration returnConf = new Configuration(conf); String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl", diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java index a5430cf89f637..2e83108e434bf 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java @@ -30,18 +30,17 @@ public class HoodieStorageWriterFactory { - public static HoodieStorageWriter - getStorageWriter(String commitTime, Path path, HoodieTable hoodieTable, + public static HoodieStorageWriter getStorageWriter( + String commitTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, Schema schema) throws IOException { //TODO - based on the metadata choose the implementation of HoodieStorageWriter // Currently only parquet is supported return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable); } - private static HoodieStorageWriter - newParquetStorageWriter( - String commitTime, Path path, HoodieWriteConfig config, Schema schema, - HoodieTable hoodieTable) throws IOException { + private static HoodieStorageWriter newParquetStorageWriter(String commitTime, Path path, + HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException { BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP()); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index ab194c4f52b7a..25ba78421f45d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -40,10 +40,11 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieUpsertException; -import com.uber.hoodie.func.LazyInsertIterable; +import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable; import com.uber.hoodie.func.ParquetReaderIterator; import com.uber.hoodie.func.SparkBoundedInMemoryExecutor; import com.uber.hoodie.io.HoodieCleanHelper; +import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieMergeHandle; import java.io.IOException; import java.io.Serializable; @@ -64,6 +65,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroParquetReader; @@ -90,8 +92,8 @@ public class HoodieCopyOnWriteTable extends Hoodi private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); - public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - super(config, metaClient); + public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) { + super(config, jsc); } private static PairFlatMapFunction>, String, @@ -225,7 +227,15 @@ protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, public Iterator> handleInsert(String commitTime, Iterator> recordItr) throws Exception { - return new LazyInsertIterable<>(recordItr, config, commitTime, this); + return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this); + } + + public Iterator> handleInsert(String commitTime, String partitionPath, String fileId, + Iterator> recordItr) { + HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId, + recordItr); + createHandle.write(); + return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator(); } @SuppressWarnings("unchecked") @@ -289,17 +299,29 @@ public List clean(JavaSparkContext jsc) { */ protected Map deleteCleanedFiles(String partitionPath, List commits) throws IOException { + Map results = Maps.newHashMap(); + // PathFilter to get all parquet files and log files that need to be deleted + PathFilter filter = (path) -> { + if (path.toString().contains(".parquet")) { + String fileCommitTime = FSUtils.getCommitTime(path.getName()); + return commits.contains(fileCommitTime); + } + return false; + }; + deleteCleanedFiles(results, partitionPath, filter); + return results; + } + + /** + * Common method used for cleaning out parquet files under a partition path during rollback of a + * set of commits + */ + protected Map deleteCleanedFiles(Map results, String partitionPath, + PathFilter filter) + throws IOException { logger.info("Cleaning path " + partitionPath); FileSystem fs = getMetaClient().getFs(); - FileStatus[] toBeDeleted = fs - .listStatus(new Path(config.getBasePath(), partitionPath), path -> { - if (!path.toString().contains(".parquet")) { - return false; - } - String fileCommitTime = FSUtils.getCommitTime(path.getName()); - return commits.contains(fileCommitTime); - }); - Map results = Maps.newHashMap(); + FileStatus[] toBeDeleted = fs.listStatus(new Path(config.getBasePath(), partitionPath), filter); for (FileStatus file : toBeDeleted) { boolean success = fs.delete(file.getPath(), false); results.put(file, success); @@ -311,7 +333,7 @@ protected Map deleteCleanedFiles(String partitionPath, List @Override public List rollback(JavaSparkContext jsc, List commits) throws IOException { - String actionType = this.getCommitActionType(); + String actionType = metaClient.getCommitActionType(); HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); List inflights = this.getInflightCommitTimeline().getInstants() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 857402564456e..8ae0684b4bdc6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -27,11 +27,12 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; -import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; +import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; -import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; @@ -39,6 +40,7 @@ import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieUpsertException; +import com.uber.hoodie.func.MergeOnReadLazyInsertIterable; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; @@ -55,6 +57,7 @@ import java.util.stream.Collectors; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; @@ -78,8 +81,8 @@ public class HoodieMergeOnReadTable extends // UpsertPartitioner for MergeOnRead table type private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner; - public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - super(config, metaClient); + public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) { + super(config, jsc); } @Override @@ -110,6 +113,17 @@ public Iterator> handleUpdate(String commitTime, String fileId } } + @Override + public Iterator> handleInsert(String commitTime, + Iterator> recordItr) throws Exception { + // If canIndexLogFiles, write inserts to log files else write inserts to parquet files + if (index.canIndexLogFiles()) { + return new MergeOnReadLazyInsertIterable<>(recordItr, config, commitTime, this); + } else { + return super.handleInsert(commitTime, recordItr); + } + } + @Override public JavaRDD compact(JavaSparkContext jsc, String compactionCommitTime) { logger.info("Checking if compaction needs to be run on " + config.getBasePath()); @@ -166,6 +180,20 @@ public List rollback(JavaSparkContext jsc, List comm return commits.stream().map(commit -> { HoodieInstant instant = commitsAndCompactions.get(commit); HoodieRollbackStat hoodieRollbackStats = null; + // Need to put the path filter here since Filter is not serializable + // PathFilter to get all parquet files and log files that need to be deleted + PathFilter filter = (path) -> { + if (path.toString().contains(".parquet")) { + String fileCommitTime = FSUtils.getCommitTime(path.getName()); + return commits.contains(fileCommitTime); + } else if (path.toString().contains(".log")) { + // Since the baseCommitTime is the only commit for new log files, it's okay here + String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path); + return commits.contains(fileCommitTime); + } + return false; + }; + switch (instant.getAction()) { case HoodieTimeline.COMMIT_ACTION: try { @@ -180,18 +208,18 @@ public List rollback(JavaSparkContext jsc, List comm case HoodieTimeline.DELTA_COMMIT_ACTION: try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - this.getCommitTimeline().getInstantDetails( + metaClient.getCommitTimeline().getInstantDetails( new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) .get()); // read commit file and (either append delete blocks or delete file) - Map filesToDeletedStatus = new HashMap<>(); + final Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); - // we do not know fileIds for inserts (first inserts are parquet files), delete - // all parquet files for the corresponding failed commit, if present (same as COW) - filesToDeletedStatus = super - .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); + // In case all data was inserts and the commit failed, there is no partition stats + if (commitMetadata.getPartitionToWriteStats().size() == 0) { + super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); + } // append rollback blocks for updates if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { @@ -201,15 +229,31 @@ public List rollback(JavaSparkContext jsc, List comm .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null; commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { - return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT - && wStat.getPrevCommit() != null; - }).forEach(wStat -> { - HoodieLogFormat.Writer writer = null; + if (wStat != null + && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT + && wStat.getPrevCommit() != null) { + return true; + } + // we do not know fileIds for inserts (first inserts are either log files or parquet files), + // delete all files for the corresponding failed commit, if present (same as COW) + try { + super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); + } catch (IOException io) { + throw new UncheckedIOException(io); + } + return false; + }) + .forEach(wStat -> { + Writer writer = null; String baseCommitTime = wStat.getPrevCommit(); if (hoodieIndex.isGlobal()) { baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId()); } try { + // TODO : wStat.getPrevCommit() might not give the right commit time in the following + // scenario if a compaction was scheduled, the new commitTime will be used to write the + // new log file. In this case, the commit time for the log file is the + // getBaseCommitTime() writer = HoodieLogFormat.newWriterBuilder().onParentPath( new Path(this.getMetaClient().getBasePath(), partitionPath)) .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) @@ -217,15 +261,15 @@ public List rollback(JavaSparkContext jsc, List comm .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; // generate metadata - Map header = + Map header = Maps.newHashMap(); - header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, + header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, + header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit); - header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String + header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String .valueOf( - HoodieCommandBlock.HoodieCommandBlockTypeEnum + HoodieCommandBlockTypeEnum .ROLLBACK_PREVIOUS_BLOCK .ordinal())); // if update belongs to an existing log file @@ -263,7 +307,6 @@ public List rollback(JavaSparkContext jsc, List comm commitsAndCompactions.entrySet().stream().map( entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight); - logger .debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); @@ -298,6 +341,8 @@ protected List getSmallFiles(String partitionPath) { if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); // find smallest file in partition and append to it + + // TODO - check if index.isglobal then small files are log files too Optional smallFileSlice = getRTFileSystemView() .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( fileSlice -> fileSlice.getLogFiles().count() < 1 @@ -327,4 +372,5 @@ public List getSmallFileIds() { .collect(Collectors.toList()); } } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 6f62533c33f13..383743a22da8b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -31,9 +31,9 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieSavepointException; +import com.uber.hoodie.index.HoodieIndex; import java.io.IOException; import java.io.Serializable; import java.util.Iterator; @@ -54,19 +54,21 @@ public abstract class HoodieTable implements Seri protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; + protected final HoodieIndex index; - protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { + protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) { this.config = config; - this.metaClient = metaClient; + this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + this.index = HoodieIndex.createIndex(config, jsc); } public static HoodieTable getHoodieTable( - HoodieTableMetaClient metaClient, HoodieWriteConfig config) { + HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) { switch (metaClient.getTableType()) { case COPY_ON_WRITE: - return new HoodieCopyOnWriteTable<>(config, metaClient); + return new HoodieCopyOnWriteTable<>(config, jsc); case MERGE_ON_READ: - return new HoodieMergeOnReadTable<>(config, metaClient); + return new HoodieMergeOnReadTable<>(config, jsc); default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); } @@ -124,21 +126,21 @@ public TableFileSystemView.RealtimeView getRTFileSystemView() { * Get the completed (commit + compaction) view of the file system for this table */ public TableFileSystemView getCompletedFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCommitsTimeline()); + return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline()); } /** * Get only the completed (no-inflights) commit timeline */ public HoodieTimeline getCompletedCommitTimeline() { - return getCommitsTimeline().filterCompletedInstants(); + return metaClient.getCommitsTimeline().filterCompletedInstants(); } /** * Get only the inflights (no-completed) commit timeline */ public HoodieTimeline getInflightCommitTimeline() { - return getCommitsTimeline().filterInflights(); + return metaClient.getCommitsTimeline().filterInflights(); } /** @@ -190,49 +192,10 @@ public HoodieActiveTimeline getActiveTimeline() { } /** - * Get the commit timeline visible for this table + * Return the index */ - public HoodieTimeline getCommitsTimeline() { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - return getActiveTimeline().getCommitTimeline(); - case MERGE_ON_READ: - // We need to include the parquet files written out in delta commits - // Include commit action to be able to start doing a MOR over a COW dataset - no - // migration required - return getActiveTimeline().getCommitsTimeline(); - default: - throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); - } - } - - /** - * Get the compacted commit timeline visible for this table - */ - public HoodieTimeline getCommitTimeline() { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - case MERGE_ON_READ: - // We need to include the parquet files written out in delta commits in tagging - return getActiveTimeline().getCommitTimeline(); - default: - throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); - } - } - - /** - * Gets the commit action type - */ - public String getCommitActionType() { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - return HoodieActiveTimeline.COMMIT_ACTION; - case MERGE_ON_READ: - return HoodieActiveTimeline.DELTA_COMMIT_ACTION; - default: - throw new HoodieCommitException( - "Could not commit on unknown storage type " + metaClient.getTableType()); - } + public HoodieIndex getIndex() { + return index; } /** diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 086818274c938..a05cbc7b6eb6a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -76,10 +76,10 @@ public class TestCleaner extends TestHoodieClientBase { /** * Helper method to do first batch of insert for clean by versions/commits tests * - * @param cfg Hoodie Write Config - * @param client Hoodie Client + * @param cfg Hoodie Write Config + * @param client Hoodie Client * @param recordGenFunction Function to generate records for insertion - * @param insertFn Insertion API for testing + * @param insertFn Insertion API for testing * @throws Exception in case of error */ private void insertFirstBigBatchForClientCleanerTest( @@ -93,7 +93,6 @@ private void insertFirstBigBatchForClientCleanerTest( * (this is basically same as insert part of upsert, just adding it here so we can * catch breakages in insert(), if the implementation diverges.) */ - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); String newCommitTime = client.startCommit(); List records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE); @@ -108,7 +107,7 @@ private void insertFirstBigBatchForClientCleanerTest( HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); // Should have 100 records in table (check using Index), all in locations marked at commit - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); assertFalse(table.getCompletedCommitTimeline().empty()); String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); @@ -116,7 +115,8 @@ private void insertFirstBigBatchForClientCleanerTest( assertEquals("The clean instant should be the same as the commit instant", commitTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); + HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); } @@ -158,10 +158,10 @@ public void testBulkInsertPreppedAndCleanByVersions() throws Exception { /** * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective * - * @param insertFn Insert API to be tested - * @param upsertFn Upsert API to be tested + * @param insertFn Insert API to be tested + * @param upsertFn Upsert API to be tested * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during - * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) * @throws Exception in case of errors */ private void testInsertAndCleanByVersions( @@ -198,8 +198,8 @@ private void testInsertAndCleanByVersions( assertNoWriteErrors(statuses); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig()); - HoodieTimeline timeline = table.getCommitsTimeline(); + HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); + HoodieTimeline timeline = metadata.getCommitsTimeline(); TableFileSystemView fsView = table.getFileSystemView(); // Need to ensure the following @@ -280,10 +280,10 @@ public void testBulkInsertAndCleanByCommits() throws Exception { /** * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective * - * @param insertFn Insert API to be tested - * @param upsertFn Upsert API to be tested + * @param insertFn Insert API to be tested + * @param upsertFn Upsert API to be tested * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during - * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) * @throws Exception in case of errors */ private void testInsertAndCleanByCommits( @@ -318,7 +318,7 @@ private void testInsertAndCleanByCommits( assertNoWriteErrors(statuses); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); + HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); @@ -365,7 +365,8 @@ public void testKeepLatestFileVersions() throws IOException { String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -377,7 +378,8 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config, + jsc); String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert @@ -397,7 +399,7 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + config, jsc); HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update @@ -452,7 +454,7 @@ public void testKeepLatestFileVersionsMOR() throws IOException { // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStats = table.clean(jsc); assertEquals("Must clean three files, one parquet and 2 log files", 3, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); @@ -479,7 +481,8 @@ public void testKeepLatestCommits() throws IOException { String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -492,7 +495,7 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + config, jsc); String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert @@ -512,7 +515,7 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + config, jsc); HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update @@ -527,7 +530,7 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "003"); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + config, jsc); HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update @@ -568,14 +571,15 @@ public void testCleanTemporaryDataFilesOnRollback() throws IOException { .withUseTempFolderCopyOnWriteForCreate(false) .withUseTempFolderCopyOnWriteForMerge(false).build(); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); table.rollback(jsc, Collections.emptyList()); assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) .withUseTempFolderCopyOnWriteForMerge(false).build(); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + config, jsc); table.rollback(jsc, Collections.emptyList()); assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); } @@ -595,7 +599,8 @@ public void testCleaningWithZeroPartitonPaths() throws IOException { HoodieTestUtils.createCommitFiles(basePath, "000"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStatsOne = table.clean(jsc); assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); @@ -655,7 +660,8 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "003"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStats = table.clean(jsc); assertEquals(100, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); @@ -677,7 +683,7 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { * Utility method to create temporary data files * * @param commitTime Commit Timestamp - * @param numFiles Number for files to be generated + * @param numFiles Number for files to be generated * @return generated files * @throws IOException in case of error */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java index 6d9dea7b4803d..68e3e9f0b81ee 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java @@ -96,7 +96,7 @@ public void testSavepointAndRollback() throws Exception { List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); List dataFiles = partitionPaths.stream().flatMap(s -> { @@ -121,7 +121,7 @@ public void testSavepointAndRollback() throws Exception { assertNoWriteErrors(statuses); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); + table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); dataFiles = partitionPaths.stream().flatMap(s -> { @@ -142,7 +142,7 @@ public void testSavepointAndRollback() throws Exception { client.rollbackToSavepoint(savepoint.getTimestamp()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); + table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); dataFiles = partitionPaths.stream().flatMap(s -> { return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java index a9e374c403850..0840f7ddc0323 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -184,9 +184,9 @@ private Function2, String, Integer> wrapRecordsGenFunctionFor final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc); List records = recordGenFunction.apply(commit, numRecords); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); - HoodieTable.getHoodieTable(metaClient, writeConfig); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc); JavaRDD taggedRecords = - index.tagLocation(jsc.parallelize(records, 1), HoodieTable.getHoodieTable(metaClient, writeConfig)); + index.tagLocation(jsc.parallelize(records, 1), jsc, table); return taggedRecords.collect(); }; } @@ -348,7 +348,7 @@ JavaRDD writeBatch( fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords, - HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); + HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count()); // Check that the incremental consumption from prevCommitTime assertEquals("Incremental consumption from " + prevCommitTime diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 99790947e1adb..9fbd0871a41d8 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -415,7 +415,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { assertEquals("2 files needs to be committed.", 2, statuses.size()); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView(); List files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) .collect(Collectors.toList()); @@ -519,7 +519,7 @@ public void testSmallInsertHandlingForInserts() throws Exception { assertEquals("2 files needs to be committed.", 2, statuses.size()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List files = table.getROFileSystemView() .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) .collect(Collectors.toList()); @@ -544,7 +544,7 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); String commitTime = "000"; client.startCommitWithTime(commitTime); @@ -559,9 +559,9 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieTestUtils.doesCommitExist(basePath, commitTime)); // Get parquet file paths from commit metadata - String actionType = table.getCommitActionType(); + String actionType = metaClient.getCommitActionType(); HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); - HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); String basePath = table.getMetaClient().getBasePath(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index d092c8304e040..a85bd3c23bff1 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -28,7 +28,6 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.table.HoodieTable; import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; @@ -43,6 +42,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -152,14 +152,17 @@ public static Dataset readSince(String basePath, SQLContext sqlContext, Hoo /** * Reads the paths under the a hoodie dataset out as a DataFrame */ - public static Dataset read(String basePath, SQLContext sqlContext, FileSystem fs, String... paths) { + public static Dataset read(JavaSparkContext jsc, String basePath, SQLContext + sqlContext, + FileSystem + fs, String... + paths) { List filteredPaths = new ArrayList<>(); try { - HoodieTable hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); for (String path : paths) { TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( - hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); + metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); List latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList()); for (HoodieDataFile file : latestFiles) { filteredPaths.add(file.getPath()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index 99af0e2e31643..83161aa2c3205 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -213,10 +213,10 @@ public List generateUpdates(String commitTime, List /** * Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list + * * @param commitTime Commit Timestamp * @param n Number of updates (including dups) * @return list of hoodie record updates - * @throws IOException */ public List generateUpdates(String commitTime, Integer n) throws IOException { List updates = new ArrayList<>(); @@ -230,10 +230,10 @@ public List generateUpdates(String commitTime, Integer n) throws I /** * Generates deduped updates of keys previously inserted, randomly distributed across the keys above. + * * @param commitTime Commit Timestamp * @param n Number of unique records * @return list of hoodie record updates - * @throws IOException */ public List generateUniqueUpdates(String commitTime, Integer n) throws IOException { List updates = new ArrayList<>(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java index 385afaabfadc9..b629b837fe53c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java @@ -16,7 +16,7 @@ package com.uber.hoodie.func; -import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; +import static com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.getTransformFunction; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -38,7 +38,7 @@ public class TestBoundedInMemoryExecutor { private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); private SparkBoundedInMemoryExecutor>, Integer> executor = null; + Tuple2>, Integer> executor = null; @After public void afterTest() { @@ -83,4 +83,4 @@ protected Integer getResult() { // There should be no remaining records in the buffer Assert.assertFalse(executor.isRemaining()); } -} +} \ No newline at end of file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java index b555b81672470..4092f10a451a3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java @@ -16,7 +16,7 @@ package com.uber.hoodie.func; -import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; +import static com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.getTransformFunction; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index dcd5a94a5eb45..20a14ea2c1c71 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -19,6 +19,7 @@ import static org.junit.Assert.fail; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -29,11 +30,14 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieCopyOnWriteTable; +import java.io.File; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -41,6 +45,7 @@ public class TestUpdateMapFunction { private String basePath = null; + private transient JavaSparkContext jsc = null; @Before public void init() throws Exception { @@ -49,6 +54,18 @@ public void init() throws Exception { folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath); + // Initialize a local spark env + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestUpdateMapFunction")); + } + + @After + public void clean() { + if (basePath != null) { + new File(basePath).delete(); + } + if (jsc != null) { + jsc.stop(); + } } @Test @@ -56,7 +73,7 @@ public void testSchemaEvolutionOnUpdate() throws Exception { // Create a bunch of records with a old version of schema HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; @@ -82,7 +99,7 @@ public void testSchemaEvolutionOnUpdate() throws Exception { String fileId = insertResult.next().get(0).getFileId(); System.out.println(fileId); - table = new HoodieCopyOnWriteTable(config, metaClient); + table = new HoodieCopyOnWriteTable(config, jsc); // New content with values for the newly added field recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java index eb4a878f903a8..6f681564bc3f9 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -26,8 +26,7 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieTableType; -import com.uber.hoodie.common.table.HoodieTableConfig; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; @@ -74,7 +73,6 @@ public class TestHbaseIndex { private static String tableName = "test_table"; private String basePath = null; private transient FileSystem fs; - private HoodieTableMetaClient metaClient; public TestHbaseIndex() throws Exception { } @@ -117,9 +115,7 @@ public void before() throws Exception { folder.create(); basePath = folder.getRoot().getAbsolutePath(); // Initialize table - metaClient = HoodieTableMetaClient - .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName, - HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); } @Test @@ -132,13 +128,14 @@ public void testSimpleTagLocationAndUpdate() throws Exception { // Load to memory HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config, jsc); + HBaseIndex index = new HBaseIndex(config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); writeClient.startCommit(); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Test tagLocation without any entries in index - JavaRDD javaRDD = index.tagLocation(writeRecords, hoodieTable); + JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Insert 200 records @@ -147,14 +144,15 @@ public void testSimpleTagLocationAndUpdate() throws Exception { // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed // commit - javaRDD = index.tagLocation(writeRecords, hoodieTable); + javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Now commit this & update location of records inserted and validate no errors writeClient.commit(newCommitTime, writeStatues); - // Now tagLocation for these records, hbaseIndex should tag them correctly - javaRDD = index.tagLocation(writeRecords, hoodieTable); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); + javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); assertTrue(javaRDD.filter( @@ -169,14 +167,14 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); // Load to memory HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config, jsc); + HBaseIndex index = new HBaseIndex(config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); @@ -184,9 +182,9 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // commit this upsert writeClient.commit(newCommitTime, writeStatues); - + hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Now tagLocation for these records, hbaseIndex should tag them - JavaRDD javaRDD = index.tagLocation(writeRecords, hoodieTable); + JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); // check tagged records are tagged with correct fileIds @@ -199,9 +197,10 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Rollback the last commit writeClient.rollback(newCommitTime); + hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit - javaRDD = index.tagLocation(writeRecords, hoodieTable); + javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); } @@ -211,7 +210,7 @@ public void testTotalGetsBatching() throws Exception { HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config, jsc); + HBaseIndex index = new HBaseIndex(config); // Mock hbaseConnection and related entities Connection hbaseConnection = Mockito.mock(Connection.class); @@ -228,15 +227,15 @@ public void testTotalGetsBatching() throws Exception { String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 250); JavaRDD writeRecords = jsc.parallelize(records, 1); - - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 250 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, hbaseIndex should tag them - index.tagLocation(writeRecords, hoodieTable); + index.tagLocation(writeRecords, jsc, hoodieTable); // 3 batches should be executed given batchSize = 100 and parallelism = 1 Mockito.verify(table, times(3)).get((List) anyObject()); @@ -248,15 +247,15 @@ public void testTotalPutsBatching() throws Exception { HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); HoodieWriteConfig config = getConfig(); - HBaseIndex index = new HBaseIndex(config, jsc); + HBaseIndex index = new HBaseIndex(config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); // start a commit and generate test data String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 250); JavaRDD writeRecords = jsc.parallelize(records, 1); - - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); @@ -276,7 +275,7 @@ public void testTotalPutsBatching() throws Exception { // Get all the files generated int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count(); - index.updateLocation(writeStatues, hoodieTable); + index.updateLocation(writeStatues, jsc, hoodieTable); // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated, // so each fileId ideally gets updates Mockito.verify(table, atMost(numberOfDataFileIds)).put((List) anyObject()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java index 16f754a4bd491..002c1e4351ca2 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java @@ -18,27 +18,58 @@ import static org.junit.Assert.assertTrue; +import com.uber.hoodie.common.HoodieClientTestUtils; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.bloom.HoodieBloomIndex; import com.uber.hoodie.index.hbase.HBaseIndex; +import java.io.File; +import java.io.IOException; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.After; +import org.junit.Before; import org.junit.Test; +import org.junit.rules.TemporaryFolder; public class TestHoodieIndex { + private transient JavaSparkContext jsc = null; + private String basePath = null; + + @Before + public void init() throws IOException { + // Initialize a local spark env + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieIndex")); + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + basePath = folder.getRoot().getAbsolutePath(); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + } + + @After + public void clean() { + if (basePath != null) { + new File(basePath).delete(); + } + if (jsc != null) { + jsc.stop(); + } + } + @Test public void testCreateIndex() throws Exception { HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); // Different types - HoodieWriteConfig config = clientConfigBuilder.withPath("").withIndexConfig( + HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig( indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build(); - assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex); - config = clientConfigBuilder.withPath("") + assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex); + config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex); - config = clientConfigBuilder.withPath("") + assertTrue(HoodieIndex.createIndex(config, jsc) instanceof InMemoryHashIndex); + config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); - assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex); + assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HoodieBloomIndex); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 8e649c73c2076..82a4c1a29f855 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -95,6 +95,16 @@ public void init() throws IOException { schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); } + @After + public void clean() { + if (basePath != null) { + new File(basePath).delete(); + } + if (jsc != null) { + jsc.stop(); + } + } + @Test public void testLoadUUIDsInMemory() throws IOException { // Create one RDD of hoodie record @@ -135,7 +145,7 @@ record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())).groupB @Test public void testLoadInvolvedFiles() throws IOException { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); + HoodieBloomIndex index = new HoodieBloomIndex(config); // Create some partitions, and put some files // "2016/01/21": 0 file @@ -171,8 +181,8 @@ public void testLoadInvolvedFiles() throws IOException { List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); - List> filesList = index.loadInvolvedFiles(partitions, table); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); + List> filesList = index.loadInvolvedFiles(partitions, jsc, table); // Still 0, as no valid commit assertEquals(filesList.size(), 0); @@ -181,7 +191,8 @@ public void testLoadInvolvedFiles() throws IOException { new File(basePath + "/.hoodie/20160401010101.commit").createNewFile(); new File(basePath + "/.hoodie/20150312101010.commit").createNewFile(); - filesList = index.loadInvolvedFiles(partitions, table); + table = HoodieTable.getHoodieTable(metadata, config, jsc); + filesList = index.loadInvolvedFiles(partitions, jsc, table); assertEquals(filesList.size(), 4); // these files will not have the key ranges assertNull(filesList.get(0)._2().getMaxRecordKey()); @@ -205,7 +216,7 @@ public void testLoadInvolvedFiles() throws IOException { public void testRangePruning() { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); + HoodieBloomIndex index = new HoodieBloomIndex(config); final Map> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), @@ -290,13 +301,13 @@ public void testTagLocationWithEmptyRDD() throws Exception { // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); try { - bloomIndex.tagLocation(recordRDD, table); + bloomIndex.tagLocation(recordRDD, jsc, table); } catch (IllegalArgumentException e) { fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required"); } @@ -332,11 +343,11 @@ public void testTagLocation() throws Exception { // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); + JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table); // Should not find any files for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -350,9 +361,9 @@ public void testTagLocation() throws Exception { // We do the tag again metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metadata, config); + table = HoodieTable.getHoodieTable(metadata, config, jsc); - taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); + taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -397,11 +408,11 @@ public void testCheckExists() throws Exception { // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaPairRDD> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); + JavaPairRDD> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table); // Should not find any files for (Tuple2> record : taggedRecordRDD.collect()) { @@ -415,8 +426,8 @@ public void testCheckExists() throws Exception { // We do the tag again metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metadata, config); - taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); + table = HoodieTable.getHoodieTable(metadata, config, jsc); + taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table); // Check results for (Tuple2> record : taggedRecordRDD.collect()) { @@ -465,10 +476,10 @@ public void testBloomFilterFalseError() throws IOException, InterruptedException JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2)); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); - HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); + JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -525,14 +536,4 @@ private String writeParquetFile(String partitionPath, String filename, List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieIndex index = new HoodieBloomIndex<>(config, jsc); - updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); + HoodieIndex index = new HoodieBloomIndex<>(config); + updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect(); // Write them to corresponding avro logfiles HoodieTestUtils @@ -158,7 +158,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) .collect(Collectors.toList()); @@ -169,7 +169,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Do a compaction metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); JavaRDD result = compactor .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java index bbb826c101304..e5523f8beaad4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java @@ -247,7 +247,8 @@ private Dataset getRecords() { for (int i = 0; i < fullPartitionPaths.length; i++) { fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } - Dataset dataSet = HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths); + Dataset dataSet = HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, + fullPartitionPaths); return dataSet; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index c3aee4f2bb041..9965cfcba29fa 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -30,6 +30,7 @@ import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Random; import java.util.stream.Collectors; import org.junit.Test; @@ -127,7 +128,7 @@ private List createCompactionOperations(HoodieWriteConfig c Map> sizesMap) { List operations = Lists.newArrayList(sizesMap.size()); sizesMap.forEach((k, v) -> { - operations.add(new CompactionOperation(TestHoodieDataFile.newDataFile(k), + operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(k)), partitionPaths[new Random().nextInt(partitionPaths.length - 1)], v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config)); }); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index ebd31b25a13d7..b9be42b715549 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -77,6 +77,7 @@ public void init() throws Exception { folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + } @Test @@ -90,9 +91,10 @@ public void testMakeNewPath() throws Exception { String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieWriteConfig config = makeHoodieClientConfig(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); - HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath); + HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath, + UUID.randomUUID().toString()); Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName); assertTrue(newPath.toString().equals( this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName))); @@ -117,7 +119,7 @@ public void testUpdateRecords() throws Exception { HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); String partitionPath = "/2016/01/31"; - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); // Get some records belong to the same partition (2016/01/31) String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -187,7 +189,7 @@ public void testUpdateRecords() throws Exception { Thread.sleep(1000); String newCommitTime = HoodieTestUtils.makeNewCommitTime(); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = new HoodieCopyOnWriteTable(config, metadata); + table = new HoodieCopyOnWriteTable(config, jsc); Iterator> iter = table .handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator()); @@ -255,7 +257,7 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); // Get some records belong to the same partition (2016/01/31) String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -290,7 +292,7 @@ public void testInsertWithPartialFailures() throws Exception { String commitTime = HoodieTestUtils.makeNewCommitTime(); FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); // Write a few records, and get atleast one file // 10 records for partition 1, 1 record for partition 2. @@ -324,7 +326,7 @@ public void testInsertRecords() throws Exception { HoodieWriteConfig config = makeHoodieClientConfig(); String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); // Case 1: // 10 records for partition 1, 1 record for partition 2. @@ -372,7 +374,7 @@ public void testFileSizeUpsertRecords() throws Exception { .build()).build(); String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB @@ -411,9 +413,9 @@ private List testUpsertPartitioner(int smal HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath}); + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{testPartitionPath}); List insertRecords = dataGenerator.generateInserts("001", numInserts); List updateRecords = dataGenerator.generateUpdates("001", numUpdates); for (HoodieRecord updateRec : updateRecords) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index a57069d1dc8cb..7b033d121f14a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -47,6 +47,7 @@ import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.index.HoodieIndex.IndexType; import com.uber.hoodie.index.bloom.HoodieBloomIndex; import java.io.File; import java.io.IOException; @@ -70,7 +71,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -155,7 +155,7 @@ public void testSimpleInsertAndUpdate() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -166,7 +166,7 @@ public void testSimpleInsertAndUpdate() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -209,9 +209,8 @@ public void testSimpleInsertAndUpdate() throws Exception { assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false)); - HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true); + HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); @@ -263,7 +262,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -274,7 +273,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -372,7 +371,7 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { client.rollback(newCommitTime); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); @@ -408,7 +407,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -419,7 +418,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -456,7 +455,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { client.rollback(newCommitTime); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); @@ -482,8 +481,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); + hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() .getTimestamp(); @@ -500,8 +499,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); + hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); assertFalse(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -531,7 +530,7 @@ public void testUpsertPartitioner() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -542,7 +541,7 @@ public void testUpsertPartitioner() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); Map parquetFileIdToSize = dataFilesToRead.collect( Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); @@ -591,7 +590,6 @@ public void testUpsertPartitioner() throws Exception { } @Test - @Ignore public void testLogFileCountsAfterCompaction() throws Exception { // insert 100 records HoodieWriteConfig config = getConfig(true); @@ -606,15 +604,15 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Update all the 100 records HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); newCommitTime = "101"; writeClient.startCommitWithTime(newCommitTime); List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieIndex index = new HoodieBloomIndex<>(config, jsc); - updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); + HoodieIndex index = new HoodieBloomIndex<>(config); + updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect(); // Write them to corresponding avro logfiles HoodieTestUtils @@ -623,7 +621,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) .collect(Collectors.toList()); @@ -634,14 +632,14 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Do a compaction metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); String commitTime = writeClient.startCompaction(); JavaRDD result = writeClient.compact(commitTime); // Verify that recently written compacted data file has no log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps( @@ -677,7 +675,7 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception // total time taken for creating files should be greater than 0 long totalCreateTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalCreateTime()) - .reduce((a,b) -> a + b).intValue(); + .reduce((a, b) -> a + b).intValue(); Assert.assertTrue(totalCreateTime > 0); // Update all the 100 records @@ -690,7 +688,7 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception writeClient.commit(newCommitTime, statuses); // total time taken for upsert all records should be greater than 0 long totalUpsertTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalUpsertTime()) - .reduce((a,b) -> a + b).intValue(); + .reduce((a, b) -> a + b).intValue(); Assert.assertTrue(totalUpsertTime > 0); // Do a compaction @@ -699,22 +697,168 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception writeClient.commitCompaction(commitTime, statuses); // total time taken for scanning log files should be greater than 0 long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime()) - .reduce((a,b) -> a + b).longValue(); + .reduce((a, b) -> a + b).longValue(); Assert.assertTrue(timeTakenForScanner > 0); } + @Test + public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception { + // insert 100 records + // Setting IndexType to be InMemory to simulate Global Index nature + HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build(); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD statuses = writeClient.insert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + + HoodieTable table = HoodieTable + .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, + jsc); + TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); + + long numLogFiles = 0; + for (String partitionPath : dataGen.getPartitionPaths()) { + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getDataFile().isPresent()).count() == 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count() > 0); + numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count(); + } + + Assert.assertTrue(numLogFiles > 0); + // Do a compaction + String commitTime = writeClient.startCompaction(); + statuses = writeClient.compact(commitTime); + Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); + Assert.assertEquals(statuses.count(), numLogFiles); + writeClient.commitCompaction(commitTime, statuses); + } + + @Test + public void testInsertsGeneratedIntoLogFilesRollback() throws Exception { + // insert 100 records + // Setting IndexType to be InMemory to simulate Global Index nature + HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build(); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD statuses = writeClient.insert(recordsRDD, newCommitTime); + // trigger an action + List writeStatuses = statuses.collect(); + + // Ensure that inserts are written to only log files + Assert.assertEquals(writeStatuses.stream().filter(writeStatus -> !writeStatus.getStat().getPath().contains("log") + ).count(), 0); + Assert.assertTrue(writeStatuses.stream().filter(writeStatus -> writeStatus.getStat().getPath().contains("log") + ).count() > 0); + + // rollback a failed commit + boolean rollback = writeClient.rollback(newCommitTime); + Assert.assertTrue(rollback); + newCommitTime = "101"; + writeClient.startCommitWithTime(newCommitTime); + + // insert 100 records + records = dataGen.generateInserts(newCommitTime, 100); + recordsRDD = jsc.parallelize(records, 1); + statuses = writeClient.insert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + + // rollback a successful commit + writeClient.rollback(newCommitTime); + final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); + + long numLogFiles = 0; + for (String partitionPath : dataGen.getPartitionPaths()) { + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getDataFile().isPresent()).count() == 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count() == 0); + numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count(); + } + Assert.assertTrue(numLogFiles == 0); + } + + @Test + public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exception { + // insert 100 records + // Setting IndexType to be InMemory to simulate Global Index nature + HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build(); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD statuses = writeClient.insert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + // trigger an action + statuses.collect(); + + HoodieTable table = HoodieTable + .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, + jsc); + TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); + + long numLogFiles = 0; + for (String partitionPath : dataGen.getPartitionPaths()) { + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getDataFile().isPresent()).count() == 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count() > 0); + numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count(); + } + + Assert.assertTrue(numLogFiles > 0); + // Do a compaction + newCommitTime = writeClient.startCompaction(); + statuses = writeClient.compact(newCommitTime); + // Ensure all log files have been compacted into parquet files + Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); + Assert.assertEquals(statuses.count(), numLogFiles); + writeClient.commitCompaction(newCommitTime, statuses); + // Trigger a rollback of compaction + writeClient.rollback(newCommitTime); + for (String partitionPath : dataGen.getPartitionPaths()) { + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getDataFile().isPresent()).count() == 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count() > 0); + } + } + + private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit).build(); } private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { + return getConfigBuilder(autoCommit, IndexType.BLOOM); + } + + private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig( HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) .forTable("test-trip-table") - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()); } private void assertNoWriteErrors(List statuses) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 9462c301db349..3a89f96d07a63 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -16,12 +16,15 @@ package com.uber.hoodie.common.table; +import static com.uber.hoodie.common.model.HoodieTableType.MERGE_ON_READ; + import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.DatasetNotFoundException; +import com.uber.hoodie.exception.HoodieException; import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -198,7 +201,7 @@ public static HoodieTableMetaClient initTableType(Configuration hadoopConf, Stri Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); - if (tableType == HoodieTableType.MERGE_ON_READ) { + if (tableType == MERGE_ON_READ) { properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName); } return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); @@ -254,6 +257,53 @@ public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter na return fs.listStatus(metaPath, nameFilter); } + /** + * Get the commit timeline visible for this table + */ + public HoodieTimeline getCommitsTimeline() { + switch (this.getTableType()) { + case COPY_ON_WRITE: + return getActiveTimeline().getCommitTimeline(); + case MERGE_ON_READ: + // We need to include the parquet files written out in delta commits + // Include commit action to be able to start doing a MOR over a COW dataset - no + // migration required + return getActiveTimeline().getCommitsTimeline(); + default: + throw new HoodieException("Unsupported table type :" + this.getTableType()); + } + } + + /** + * Get the compacted commit timeline visible for this table + */ + public HoodieTimeline getCommitTimeline() { + switch (this.getTableType()) { + case COPY_ON_WRITE: + case MERGE_ON_READ: + // We need to include the parquet files written out in delta commits in tagging + return getActiveTimeline().getCommitTimeline(); + default: + throw new HoodieException("Unsupported table type :" + this.getTableType()); + } + } + + /** + * Gets the commit action type + */ + public String getCommitActionType() { + switch (this.getTableType()) { + case COPY_ON_WRITE: + return HoodieActiveTimeline.COMMIT_ACTION; + case MERGE_ON_READ: + return HoodieActiveTimeline.DELTA_COMMIT_ACTION; + default: + throw new HoodieException( + "Could not commit on unknown storage type " + this.getTableType()); + } + } + + @Override public boolean equals(Object o) { if (this == o) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index dd79fff921dc8..a1309c73f1061 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -91,13 +91,12 @@ public abstract class AbstractHoodieLogRecordScanner { private AtomicLong totalCorruptBlocks = new AtomicLong(0); // Store the last instant log blocks (needed to implement rollback) private Deque currentInstantLogBlocks = new ArrayDeque<>(); - // Progress private float progress = 0.0f; - public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, - boolean readBlocksLazily, boolean reverseReader, int bufferSize) { + // TODO (NA) - Change this to a builder, this constructor is too long + public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, + String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index e837285b1f2e0..437a8cfd20e57 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -247,6 +247,13 @@ public static String maskWithoutLogVersion(String commitTime, String fileId, return LOG_FILE_PREFIX + String.format("%s_%s%s*", fileId, commitTime, logFileExtension); } + public static boolean isLogFile(Path logPath) { + Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName()); + if (!matcher.find()) { + return false; + } + return true; + } /** * Get the latest log file written from the list of log files passed in diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java index f3fe941000f75..f2f6e4636ce3e 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -24,7 +24,6 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.table.HoodieTable; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; @@ -66,15 +65,13 @@ public static String latestCommit(FileSystem fs, String basePath) { * could be fed into the datasource options. */ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), - null); - if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { - return table.getActiveTimeline().getTimelineOfActions( + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); + if (metaClient.getTableType().equals(HoodieTableType.MERGE_ON_READ)) { + return metaClient.getActiveTimeline().getTimelineOfActions( Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION)); } else { - return table.getCommitTimeline().filterCompletedInstants(); + return metaClient.getCommitTimeline().filterCompletedInstants(); } } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index cf0a2c1aa8082..cace267a9bcdd 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -21,6 +21,7 @@ package com.uber.hoodie import com.uber.hoodie.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieTableType} import com.uber.hoodie.common.table.HoodieTableMetaClient import com.uber.hoodie.common.util.ParquetUtils +import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException import com.uber.hoodie.table.HoodieTable import org.apache.hadoop.fs.Path @@ -52,8 +53,10 @@ class IncrementalRelation(val sqlContext: SQLContext, if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") } - val hoodieTable = HoodieTable.getHoodieTable(metaClient, null) - val commitTimeline = hoodieTable.getCommitTimeline.filterCompletedInstants(); + // TODO : Figure out a valid HoodieWriteConfig + val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), + sqlContext.sparkContext) + val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants(); if (commitTimeline.empty()) { throw new HoodieException("No instants to incrementally pull") } From 7d7ada8410def90fd0eeac2cd5c9a9e607ca1d77 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 23 May 2018 14:26:11 -0700 Subject: [PATCH 063/374] Changing Day based compaction strategy to be IO agnostic --- .../hoodie/config/HoodieCompactionConfig.java | 12 +++++++++ .../uber/hoodie/config/HoodieWriteConfig.java | 5 ++++ .../strategy/DayBasedCompactionStrategy.java | 26 ++++++++++++------- .../TestHoodieCompactionStrategy.java | 25 +++++++++++++++--- 4 files changed, 55 insertions(+), 13 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index 3d10d1b73e292..d1584aac02649 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -104,6 +104,10 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24"; private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128); private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96); + public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = "hoodie.compaction.daybased.target" + + ".partitions"; + // 500GB of target IO per compaction (both read and write) + public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10); private HoodieCompactionConfig(Properties props) { super(props); @@ -230,6 +234,12 @@ public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogR return this; } + public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPerCompaction) { + props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, + String.valueOf(targetPartitionsPerCompaction)); + return this; + } + public HoodieCompactionConfig build() { HoodieCompactionConfig config = new HoodieCompactionConfig(props); setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, @@ -269,6 +279,8 @@ public HoodieCompactionConfig build() { COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED); setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP), COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED); + setDefaultOnCondition(props, !props.containsKey(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP), + TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION); HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP)); Preconditions.checkArgument(Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index c974d2892f696..6c1e394df59cf 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -237,6 +237,11 @@ public String getPayloadClass() { return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP); } + public int getTargetPartitionsPerDayBasedCompaction() { + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP)); + } + /** * index properties **/ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java index 3e50459521481..714dad54e6031 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java @@ -17,6 +17,7 @@ package com.uber.hoodie.io.compact.strategy; +import com.google.common.annotations.VisibleForTesting; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.io.compact.CompactionOperation; @@ -26,39 +27,46 @@ import java.util.Date; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.stream.Collectors; /** * This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to * compact data in latest partitions first and then older capped at the Total_IO allowed. */ -public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy { +public class DayBasedCompactionStrategy extends CompactionStrategy { // For now, use SimpleDateFormat as default partition format private static String datePartitionFormat = "yyyy/MM/dd"; // Sorts compaction in LastInFirstCompacted order - private static Comparator comparator = (CompactionOperation leftC, - CompactionOperation rightC) -> { + private static Comparator comparator = (String leftPartition, + String rightPartition) -> { try { Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH) - .parse(leftC.getPartitionPath()); + .parse(leftPartition); Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH) - .parse(rightC.getPartitionPath()); + .parse(rightPartition); return left.after(right) ? -1 : right.after(left) ? 1 : 0; } catch (ParseException e) { throw new HoodieException("Invalid Partition Date Format", e); } }; - public Comparator getComparator() { + @VisibleForTesting + public Comparator getComparator() { return comparator; } @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations) { - // Iterate through the operations and accept operations as long as we are within the IO limit - return super.orderAndFilter(writeConfig, - operations.stream().sorted(comparator).collect(Collectors.toList())); + // Iterate through the operations and accept operations as long as we are within the configured target partitions + // limit + List filteredList = operations.stream() + .collect(Collectors.groupingBy(CompactionOperation::getPartitionPath)).entrySet().stream() + .sorted(Map.Entry.comparingByKey(comparator)).limit(writeConfig.getTargetPartitionsPerDayBasedCompaction()) + .flatMap(e -> e.getValue().stream()) + .collect(Collectors.toList()); + return filteredList; } } \ No newline at end of file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index 9965cfcba29fa..cb53fdd8dcecb 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -33,6 +33,7 @@ import java.util.Optional; import java.util.Random; import java.util.stream.Collectors; +import org.junit.Assert; import org.junit.Test; public class TestHoodieCompactionStrategy { @@ -109,17 +110,33 @@ public void testPartitionAwareCompactionSimple() { sizesMap.put(110 * MB, Lists.newArrayList()); sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); + sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); + sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); + sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); + DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy(); HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) - .build(); - List operations = createCompactionOperations(writeConfig, sizesMap); + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) + .withTargetPartitionsPerDayBasedCompaction(1) + .build()).build(); + + List operations = Lists.newArrayList(sizesMap.size()); + int partitionPathIndex = 0; + for (Map.Entry> entry : sizesMap.entrySet()) { + operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(entry.getKey())), + partitionPaths[(partitionPathIndex % (partitionPaths.length - 1))], + entry.getValue().stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), writeConfig)); + partitionPathIndex++; + } List returned = strategy.orderAndFilter(writeConfig, operations); assertTrue("DayBasedCompactionStrategy should have resulted in fewer compactions", returned.size() < operations.size()); + Assert.assertEquals("DayBasedCompactionStrategy should have resulted in fewer compactions", + returned.size(), 2); - int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1), returned.get(0)); + int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1).getPartitionPath(), returned + .get(0).getPartitionPath()); // Either the partition paths are sorted in descending order or they are equal assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", comparision >= 0); } From 6694ae3a9a49398cf9e7d1040068370cde3d8ac5 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Mon, 9 Jul 2018 16:58:05 -0700 Subject: [PATCH 064/374] Fixing bug introducted in rollback for MOR table type with inserts into log files --- .../hoodie/table/HoodieMergeOnReadTable.java | 21 ++-- .../hoodie/table/TestMergeOnReadTable.java | 95 ++++++++++++++----- .../com/uber/hoodie/common/util/FSUtils.java | 10 ++ 3 files changed, 90 insertions(+), 36 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 8ae0684b4bdc6..109397363ae81 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -54,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -216,11 +217,14 @@ public List rollback(JavaSparkContext jsc, List comm final Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); - // In case all data was inserts and the commit failed, there is no partition stats - if (commitMetadata.getPartitionToWriteStats().size() == 0) { - super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); - } + // In case all data was inserts and the commit failed, delete the file belonging to that commit + super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); + final Set deletedFiles = filesToDeletedStatus.entrySet().stream() + .map(entry -> { + Path filePath = entry.getKey().getPath(); + return FSUtils.getFileIdFromFilePath(filePath); + }).collect(Collectors.toSet()); // append rollback blocks for updates if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { // This needs to be done since GlobalIndex at the moment does not store the latest commit time @@ -231,16 +235,9 @@ public List rollback(JavaSparkContext jsc, List comm .filter(wStat -> { if (wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT - && wStat.getPrevCommit() != null) { + && wStat.getPrevCommit() != null && !deletedFiles.contains(wStat.getFileId())) { return true; } - // we do not know fileIds for inserts (first inserts are either log files or parquet files), - // delete all files for the corresponding failed commit, if present (same as COW) - try { - super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); - } catch (IOException io) { - throw new UncheckedIOException(io); - } return false; }) .forEach(wStat -> { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 7b033d121f14a..863ff1998bbe3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -51,6 +51,8 @@ import com.uber.hoodie.index.bloom.HoodieBloomIndex; import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -389,10 +391,10 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { @Test public void testRollbackWithDeltaAndCompactionCommit() throws Exception { - HoodieWriteConfig cfg = getConfig(true); + HoodieWriteConfig cfg = getConfig(false); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - // Test delta commit rollback (with all log files) + // Test delta commit rollback /** * Write 1 (only inserts) */ @@ -403,7 +405,9 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - List statuses = client.upsert(writeRecords, newCommitTime).collect(); + JavaRDD writeStatusJavaRDD = client.upsert(writeRecords, newCommitTime); + client.commit(newCommitTime, writeStatusJavaRDD); + List statuses = writeStatusJavaRDD.collect(); assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); @@ -428,56 +432,99 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { dataFilesToRead.findAny().isPresent()); /** - * Write 2 (updates) + * Write 2 (inserts + updates - testing failed delta commit) */ - newCommitTime = "002"; - client.startCommitWithTime(newCommitTime); + final String commitTime1 = "002"; + // WriteClient with custom config (disable small file handling) + client = new HoodieWriteClient(jsc, HoodieWriteConfig.newBuilder().withPath(basePath) + .withSchema(TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(1 * 1024).withInlineCompaction(false) + .withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build()) + .forTable("test-trip-table").build()); + client.startCommitWithTime(commitTime1); + + List copyOfRecords = new ArrayList<>(records); + copyOfRecords = dataGen.generateUpdates(commitTime1, copyOfRecords); + copyOfRecords.addAll(dataGen.generateInserts(commitTime1, 200)); - records = dataGen.generateUpdates(newCommitTime, records); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); + assertEquals(recordsRead.size(), 200); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + statuses = client.upsert(jsc.parallelize(copyOfRecords, 1), commitTime1).collect(); // Verify there are no errors assertNoWriteErrors(statuses); - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); - assertTrue(deltaCommit.isPresent()); - assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp()); - commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); - assertFalse(commit.isPresent()); + // Test failed delta commit rollback + client.rollback(commitTime1); + allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + // After rollback, there should be no parquet file with the failed commit time + Assert.assertEquals(Arrays.asList(allFiles).stream().filter(file -> file.getPath().getName() + .contains(commitTime1)).collect(Collectors.toList()).size(), 0); + dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); + assertEquals(recordsRead.size(), 200); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); + /** + * Write 3 (inserts + updates - testing successful delta commit) + */ + final String commitTime2 = "002"; + client = new HoodieWriteClient(jsc, cfg); + client.startCommitWithTime(commitTime2); + + copyOfRecords = new ArrayList<>(records); + copyOfRecords = dataGen.generateUpdates(commitTime2, copyOfRecords); + copyOfRecords.addAll(dataGen.generateInserts(commitTime2, 200)); + + dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); - // Test delta commit rollback - client.rollback(newCommitTime); + writeStatusJavaRDD = client.upsert(writeRecords, commitTime2); + client.commit(commitTime2, writeStatusJavaRDD); + statuses = writeStatusJavaRDD.collect(); + // Verify there are no errors + assertNoWriteErrors(statuses); + + // Test successful delta commit rollback + client.rollback(commitTime2); + allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + // After rollback, there should be no parquet file with the failed commit time + Assert.assertEquals(Arrays.asList(allFiles).stream().filter(file -> file.getPath().getName() + .contains(commitTime2)).collect(Collectors.toList()).size(), 0); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); - + // check that the number of records read is still correct after rollback operation assertEquals(recordsRead.size(), 200); - //Test compaction commit rollback + // Test compaction commit rollback /** - * Write 2 (updates) + * Write 4 (updates) */ newCommitTime = "003"; client.startCommitWithTime(newCommitTime); - records = dataGen.generateUpdates(newCommitTime, 400); + records = dataGen.generateUpdates(newCommitTime, records); - statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + writeStatusJavaRDD = client.upsert(writeRecords, newCommitTime); + client.commit(newCommitTime, writeStatusJavaRDD); + statuses = writeStatusJavaRDD.collect(); + // Verify there are no errors assertNoWriteErrors(statuses); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); String compactionCommit = client.startCompaction(); - client.compact(compactionCommit); + JavaRDD writeStatus = client.compact(compactionCommit); + client.commitCompaction(compactionCommit, writeStatus); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index 437a8cfd20e57..f71f6b4cc84c6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -213,6 +213,16 @@ public static String getFileIdFromLogPath(Path path) { return matcher.group(1); } + /** + * Check if the file is a parquet file of a log file. Then get the fileId appropriately. + */ + public static String getFileIdFromFilePath(Path filePath) { + if (FSUtils.isLogFile(filePath)) { + return FSUtils.getFileIdFromLogPath(filePath); + } + return FSUtils.getFileId(filePath.getName()); + } + /** * Get the first part of the file name in the log file. That will be the fileId. Log file do not * have commitTime in the file name. From e5e567f19ca567bd492f88294e131a98dbe8b940 Mon Sep 17 00:00:00 2001 From: Omkar Joshi Date: Wed, 18 Jul 2018 11:11:32 -0700 Subject: [PATCH 065/374] adding setters so that subclasses can set it --- .../src/main/java/com/uber/hoodie/WriteStatus.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java index 14a01775386d7..ac54d890cd216 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java @@ -142,10 +142,18 @@ public long getTotalRecords() { return totalRecords; } + public void setTotalRecords(long totalRecords) { + this.totalRecords = totalRecords; + } + public long getTotalErrorRecords() { return totalErrorRecords; } + public void setTotalErrorRecords(long totalErrorRecords) { + this.totalErrorRecords = totalErrorRecords; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder("WriteStatus {"); From a0a780b89214987c4ed94a16d09154d1191f55da Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Thu, 12 Jul 2018 17:45:10 -0700 Subject: [PATCH 066/374] Fixing missing hoodie record location in HoodieRecord when record is read from disk after being spilled --- .../com/uber/hoodie/TestHoodieReadClient.java | 4 +++ .../hoodie/common/model/HoodieRecord.java | 2 +- .../converter/HoodieRecordConverter.java | 34 +++++++++++++++---- .../common/util/SpillableMapTestUtils.java | 10 ++++-- .../collection/TestExternalSpillableMap.java | 7 ++-- 5 files changed, 44 insertions(+), 13 deletions(-) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java index 45cc4d381edc3..9f60cc37078cb 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java @@ -165,9 +165,12 @@ private void testTagLocation( JavaRDD result = insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, insertFn, isPrepped, true, numRecords); + // Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly + // since they have been modified in the DAG JavaRDD recordRDD = jsc.parallelize( result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) + .map(record -> new HoodieRecord(record.getKey(), null)) .collect(Collectors.toList())); // Should have 100 records in table (check using Index), all in locations marked at commit HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); @@ -186,6 +189,7 @@ private void testTagLocation( recordRDD = jsc.parallelize( result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) + .map(record -> new HoodieRecord(record.getKey(), null)) .collect(Collectors.toList())); // Index should be able to locate all updates in correct locations. readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java index 16d75491a8d49..cef6e60a2632b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -102,7 +102,7 @@ public HoodieRecord setNewLocation(HoodieRecordLocation location) { } public Optional getNewLocation() { - return Optional.of(this.newLocation); + return Optional.ofNullable(this.newLocation); } public boolean isCurrentLocationKnown() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java index ee6c90d9c64f2..6aac7b365a475 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -18,6 +18,7 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.ReflectionUtils; @@ -28,6 +29,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.commons.lang3.SerializationUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -58,9 +60,15 @@ public byte[] getBytes(HoodieRecord hoodieRecord) { val = HoodieAvroUtils .avroToBytes((GenericRecord) hoodieRecord.getData().getInsertValue(schema).get()); } - Pair, byte[]> data = - Pair.of(Pair.of(hoodieRecord.getKey().getRecordKey(), - hoodieRecord.getKey().getPartitionPath()), val); + byte [] currentLocation = hoodieRecord.getCurrentLocation() != null ? SerializationUtils.serialize(hoodieRecord + .getCurrentLocation()) : new byte[0]; + byte [] newLocation = hoodieRecord.getNewLocation().isPresent() ? SerializationUtils.serialize( + (HoodieRecordLocation) hoodieRecord.getNewLocation().get()) : new byte[0]; + + // Triple, Pair, data> + Triple, Pair, byte[]> data = + Triple.of(Pair.of(hoodieRecord.getKey().getRecordKey(), + hoodieRecord.getKey().getPartitionPath()), Pair.of(currentLocation, newLocation), val); return SerializationUtils.serialize(data); } catch (IOException io) { throw new HoodieNotSerializableException("Cannot serialize value to bytes", io); @@ -70,17 +78,29 @@ public byte[] getBytes(HoodieRecord hoodieRecord) { @Override public HoodieRecord getData(byte[] bytes) { try { - Pair, byte[]> data = SerializationUtils.deserialize(bytes); + Triple, Pair, byte[]> data = SerializationUtils.deserialize(bytes); Optional payload = Optional.empty(); - if (data.getValue().length > 0) { + HoodieRecordLocation currentLocation = null; + HoodieRecordLocation newLocation = null; + if (data.getRight().length > 0) { // This can happen if the record is deleted, the payload is optional with 0 bytes - payload = Optional.of(HoodieAvroUtils.bytesToAvro(data.getValue(), schema)); + payload = Optional.of(HoodieAvroUtils.bytesToAvro(data.getRight(), schema)); + } + // Get the currentLocation for the HoodieRecord + if (data.getMiddle().getLeft().length > 0) { + currentLocation = SerializationUtils.deserialize(data.getMiddle().getLeft()); + } + // Get the newLocation for the HoodieRecord + if (data.getMiddle().getRight().length > 0) { + newLocation = SerializationUtils.deserialize(data.getMiddle().getRight()); } HoodieRecord hoodieRecord = new HoodieRecord<>( - new HoodieKey(data.getKey().getKey(), data.getKey().getValue()), + new HoodieKey(data.getLeft().getKey(), data.getLeft().getValue()), ReflectionUtils .loadPayload(payloadClazz, new Object[]{payload}, Optional.class)); + hoodieRecord.setCurrentLocation(currentLocation); + hoodieRecord.setNewLocation(newLocation); return hoodieRecord; } catch (IOException io) { throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java index 42549afcf56fa..3ffb23dff3fed 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java @@ -19,6 +19,7 @@ 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.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import java.util.ArrayList; import java.util.List; @@ -29,6 +30,9 @@ public class SpillableMapTestUtils { + public static final String DUMMY_COMMIT_TIME = "DUMMY_COMMIT_TIME"; + public static final String DUMMY_FILE_ID = "DUMMY_FILE_ID"; + public static List upsertRecords(List iRecords, Map> records) { List recordKeys = new ArrayList<>(); @@ -38,8 +42,10 @@ public static List upsertRecords(List iRecords, String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); recordKeys.add(key); - records.put(key, new HoodieRecord<>(new HoodieKey(key, partitionPath), - new HoodieAvroPayload(Optional.of((GenericRecord) r)))); + HoodieRecord record = new HoodieRecord<>(new HoodieKey(key, partitionPath), + new HoodieAvroPayload(Optional.of((GenericRecord) r))); + record.setCurrentLocation(new HoodieRecordLocation("DUMMY_COMMIT_TIME", "DUMMY_FILE_ID")); + records.put(key, record); }); return recordKeys; } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index 7a5239f9d6993..c85e869ddb67b 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -149,11 +149,12 @@ public void testAllMapOperations() throws IOException, URISyntaxException { new HoodieAvroPayload(Optional.of((GenericRecord) onDiskRecord))); // assert size assert records.size() == 100; - // get should return the same HoodieKey and same value + // get should return the same HoodieKey, same location and same value assert inMemoryHoodieRecord.getKey().equals(records.get(ikey).getKey()); assert onDiskHoodieRecord.getKey().equals(records.get(dkey).getKey()); - //assert inMemoryHoodieRecord.equals(records.get(ikey)); - //assert onDiskHoodieRecord.equals(records.get(dkey)); + // compare the member variables of HoodieRecord not set by the constructor + assert records.get(ikey).getCurrentLocation().getFileId().equals(SpillableMapTestUtils.DUMMY_FILE_ID); + assert records.get(ikey).getCurrentLocation().getCommitTime().equals(SpillableMapTestUtils.DUMMY_COMMIT_TIME); // test contains assertTrue(records.containsKey(ikey)); From be68b903a45676d8158369e8185576189294cf1e Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 23 May 2018 16:54:53 -0700 Subject: [PATCH 067/374] FileSystemView and Timeline level changes to support Async Compaction --- .../com/uber/hoodie/HoodieWriteClient.java | 10 +- .../uber/hoodie/io/HoodieAppendHandle.java | 2 +- .../compact/HoodieRealtimeTableCompactor.java | 2 +- .../hoodie/table/HoodieMergeOnReadTable.java | 5 +- .../com/uber/hoodie/table/HoodieTable.java | 5 +- .../hoodie/table/TestMergeOnReadTable.java | 6 +- .../uber/hoodie/common/model/FileSlice.java | 14 +- .../hoodie/common/model/HoodieFileGroup.java | 87 ++-- .../hoodie/common/model/HoodieLogFile.java | 12 +- .../common/table/HoodieTableMetaClient.java | 17 + .../hoodie/common/table/HoodieTimeline.java | 70 ++- .../common/table/TableFileSystemView.java | 16 + .../log/AbstractHoodieLogRecordScanner.java | 1 + .../table/timeline/HoodieActiveTimeline.java | 65 ++- .../table/timeline/HoodieDefaultTimeline.java | 23 + .../common/table/timeline/HoodieInstant.java | 60 ++- .../table/view/HoodieTableFileSystemView.java | 183 +++++++- .../string/HoodieActiveTimelineTest.java | 4 +- .../view/HoodieTableFileSystemViewTest.java | 433 +++++++++++++++++- .../realtime/HoodieRealtimeInputFormat.java | 9 +- 20 files changed, 892 insertions(+), 132 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index ef805a94f011e..061fa908e95fd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -398,7 +398,7 @@ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, Hood }); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - Optional instant = activeTimeline.filterInflights().lastInstant(); + Optional instant = activeTimeline.filterInflightsExcludingCompaction().lastInstant(); activeTimeline.saveToInflight(instant.get(), Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } catch (IOException io) { @@ -692,7 +692,7 @@ public boolean rollbackToSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline(); + HoodieTimeline commitTimeline = table.getMetaClient().getCommitsAndCompactionTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); @@ -709,8 +709,8 @@ public boolean rollbackToSavepoint(String savepointTime) { rollback(commitsToRollback); // Make sure the rollback was successful - Optional lastInstant = activeTimeline.reload().getCommitsTimeline() - .filterCompletedInstants().lastInstant(); + Optional lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline() + .filterCompletedAndCompactionInstants().lastInstant(); Preconditions.checkArgument(lastInstant.isPresent()); Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), savepointTime + "is not the last commit after rolling back " + commitsToRollback @@ -1051,7 +1051,7 @@ JavaRDD> deduplicateRecords(JavaRDD> records, private void rollbackInflightCommits() { HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflights(); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsExcludingCompaction(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); Collections.reverse(commits); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index d53c19bc2db0a..fa7d857d6f20e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -113,7 +113,7 @@ private void init(HoodieRecord record) { .filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst(); String baseInstantTime = commitTime; if (fileSlice.isPresent()) { - baseInstantTime = fileSlice.get().getBaseCommitTime(); + baseInstantTime = fileSlice.get().getBaseInstantTime(); } else { // This means there is no base data file, start appending to a new log file fileSlice = Optional.of(new FileSlice(baseInstantTime, this.fileId)); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index fc03b6cad8737..5a60f0d8a996f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -182,7 +182,7 @@ private List getCompactionWorkload(JavaSparkContext jsc, .getLatestFileSlices(partitionPath).map( s -> { List logFiles = s.getLogFiles().sorted(HoodieLogFile - .getLogVersionComparator().reversed()).collect(Collectors.toList()); + .getBaseInstantAndLogVersionComparator().reversed()).collect(Collectors.toList()); totalLogFiles.add((long) logFiles.size()); totalFileSlices.add(1L); return new CompactionOperation(s.getDataFile(), partitionPath, logFiles, config); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 109397363ae81..74e83e92c0c1b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -230,7 +230,7 @@ public List rollback(JavaSparkContext jsc, List comm // This needs to be done since GlobalIndex at the moment does not store the latest commit time Map fileIdToLatestCommitTimeMap = hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null; + .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime)) : null; commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { if (wStat != null @@ -341,7 +341,8 @@ protected List getSmallFiles(String partitionPath) { // TODO - check if index.isglobal then small files are log files too Optional smallFileSlice = getRTFileSystemView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( + // Use the merged file-slice for small file selection + .getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getDataFile().get().getFileSize() < config .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 383743a22da8b..76231331869c4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -119,7 +119,8 @@ public TableFileSystemView.ReadOptimizedView getROFileSystemView() { * Get the real time view of the file system for this table */ public TableFileSystemView.RealtimeView getRTFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline()); + return new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants()); } /** @@ -140,7 +141,7 @@ public HoodieTimeline getCompletedCommitTimeline() { * Get only the inflights (no-completed) commit timeline */ public HoodieTimeline getInflightCommitTimeline() { - return metaClient.getCommitsTimeline().filterInflights(); + return metaClient.getCommitsTimeline().filterInflightsExcludingCompaction(); } /** diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 863ff1998bbe3..980b3d07746ca 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -595,8 +595,9 @@ public void testUpsertPartitioner() throws Exception { roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); + List dataFilesList = dataFilesToRead.collect(Collectors.toList()); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", - dataFilesToRead.findAny().isPresent()); + dataFilesList.size() > 0); /** * Write 2 (only updates + inserts, written to .log file + correction of existing parquet @@ -624,7 +625,8 @@ public void testUpsertPartitioner() throws Exception { roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); - Map parquetFileIdToNewSize = dataFilesToRead.collect( + List newDataFilesList = dataFilesToRead.collect(Collectors.toList()); + Map parquetFileIdToNewSize = newDataFilesList.stream().collect( Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); assertTrue(parquetFileIdToNewSize.entrySet().stream() diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java index b0f4c41822b27..0720fc99b134b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java @@ -37,7 +37,7 @@ public class FileSlice implements Serializable { /** * Point in the timeline, at which the slice was created */ - private String baseCommitTime; + private String baseInstantTime; /** * data file, with the compacted data, for this slice @@ -50,11 +50,11 @@ public class FileSlice implements Serializable { */ private final TreeSet logFiles; - public FileSlice(String baseCommitTime, String fileId) { + public FileSlice(String baseInstantTime, String fileId) { this.fileId = fileId; - this.baseCommitTime = baseCommitTime; + this.baseInstantTime = baseInstantTime; this.dataFile = null; - this.logFiles = new TreeSet<>(HoodieLogFile.getLogVersionComparator()); + this.logFiles = new TreeSet<>(HoodieLogFile.getBaseInstantAndLogVersionComparator()); } public void setDataFile(HoodieDataFile dataFile) { @@ -69,8 +69,8 @@ public Stream getLogFiles() { return logFiles.stream(); } - public String getBaseCommitTime() { - return baseCommitTime; + public String getBaseInstantTime() { + return baseInstantTime; } public String getFileId() { @@ -84,7 +84,7 @@ public Optional getDataFile() { @Override public String toString() { final StringBuilder sb = new StringBuilder("FileSlice {"); - sb.append("baseCommitTime=").append(baseCommitTime); + sb.append("baseCommitTime=").append(baseInstantTime); sb.append(", dataFile='").append(dataFile).append('\''); sb.append(", logFiles='").append(logFiles).append('\''); sb.append('}'); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java index f3111a1098264..1f30cfad4611f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java @@ -72,6 +72,16 @@ public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) this.lastInstant = timeline.lastInstant(); } + /** + * Potentially add a new file-slice by adding base-instant time + * A file-slice without any data-file and log-files can exist (if a compaction just got requested) + */ + public void addNewFileSliceAtInstant(String baseInstantTime) { + if (!fileSlices.containsKey(baseInstantTime)) { + fileSlices.put(baseInstantTime, new FileSlice(baseInstantTime, id)); + } + } + /** * Add a new datafile into the file group */ @@ -106,13 +116,27 @@ public String getPartitionPath() { */ private boolean isFileSliceCommitted(FileSlice slice) { String maxCommitTime = lastInstant.get().getTimestamp(); - return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) - && HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(), + return timeline.containsOrBeforeTimelineStarts(slice.getBaseInstantTime()) + && HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(), maxCommitTime, HoodieTimeline.LESSER_OR_EQUAL); } + /** + * Get all the the file slices including in-flight ones as seen in underlying file-system + */ + public Stream getAllFileSlicesIncludingInflight() { + return fileSlices.entrySet().stream().map(sliceEntry -> sliceEntry.getValue()); + } + + /** + * Get latest file slices including in-flight ones + */ + public Optional getLatestFileSlicesIncludingInflight() { + return getAllFileSlicesIncludingInflight().findFirst(); + } + /** * Provides a stream of committed file slices, sorted reverse base commit time. */ @@ -141,15 +165,29 @@ public Optional getLatestFileSlice() { public Optional getLatestFileSliceBeforeOrOn(String maxCommitTime) { return getAllFileSlices() .filter(slice -> - HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(), + HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(), maxCommitTime, HoodieTimeline.LESSER_OR_EQUAL)) .findFirst(); } + /** + * Obtain the latest file slice, upto a commitTime i.e < maxInstantTime + * @param maxInstantTime Max Instant Time + * @return + */ + public Optional getLatestFileSliceBefore(String maxInstantTime) { + return getAllFileSlices() + .filter(slice -> + HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(), + maxInstantTime, + HoodieTimeline.LESSER)) + .findFirst(); + } + public Optional getLatestFileSliceInRange(List commitRange) { return getAllFileSlices() - .filter(slice -> commitRange.contains(slice.getBaseCommitTime())) + .filter(slice -> commitRange.contains(slice.getBaseInstantTime())) .findFirst(); } @@ -162,47 +200,6 @@ public Stream getAllDataFiles() { .map(slice -> slice.getDataFile().get()); } - /** - * Get the latest committed data file - */ - public Optional getLatestDataFile() { - return getAllDataFiles().findFirst(); - } - - /** - * Get the latest data file, that is <= max commit time - */ - public Optional getLatestDataFileBeforeOrOn(String maxCommitTime) { - return getAllDataFiles() - .filter(dataFile -> - HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), - maxCommitTime, - HoodieTimeline.LESSER_OR_EQUAL)) - .findFirst(); - } - - /** - * Get the latest data file, that is contained within the provided commit range. - */ - public Optional getLatestDataFileInRange(List commitRange) { - return getAllDataFiles() - .filter(dataFile -> commitRange.contains(dataFile.getCommitTime())) - .findFirst(); - } - - /** - * Obtain the latest log file (based on latest committed data file), currently being appended to - * - * @return logfile if present, empty if no log file has been opened already. - */ - public Optional getLatestLogFile() { - Optional latestSlice = getLatestFileSlice(); - if (latestSlice.isPresent() && latestSlice.get().getLogFiles().count() > 0) { - return latestSlice.get().getLogFiles().findFirst(); - } - return Optional.empty(); - } - @Override public String toString() { final StringBuilder sb = new StringBuilder("HoodieFileGroup {"); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java index 7dfaf0bdf9f48..9f159d6cae51f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java @@ -94,10 +94,16 @@ public HoodieLogFile rollOver(FileSystem fs) throws IOException { FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion))); } - public static Comparator getLogVersionComparator() { + public static Comparator getBaseInstantAndLogVersionComparator() { return (o1, o2) -> { - // reverse the order - return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion()); + String baseInstantTime1 = o1.getBaseCommitTime(); + String baseInstantTime2 = o2.getBaseCommitTime(); + if (baseInstantTime1.equals(baseInstantTime2)) { + // reverse the order by log-version when base-commit is same + return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion()); + } + // reverse the order by base-commits + return new Integer(baseInstantTime2.compareTo(baseInstantTime1)); }; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 3a89f96d07a63..1a0edfc76de36 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -274,6 +274,23 @@ public HoodieTimeline getCommitsTimeline() { } } + /** + * Get the commit + pending-compaction timeline visible for this table. + * A RT filesystem view is constructed with this timeline so that file-slice after pending compaction-requested + * instant-time is also considered valid. A RT file-system view for reading must then merge the file-slices before + * and after pending compaction instant so that all delta-commits are read. + */ + public HoodieTimeline getCommitsAndCompactionTimeline() { + switch (this.getTableType()) { + case COPY_ON_WRITE: + return getActiveTimeline().getCommitTimeline(); + case MERGE_ON_READ: + return getActiveTimeline().getCommitsAndCompactionTimeline(); + default: + throw new HoodieException("Unsupported table type :" + this.getTableType()); + } + } + /** * Get the compacted commit timeline visible for this table */ diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java index 867fa5a10e264..790ba770e0488 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java @@ -18,10 +18,12 @@ import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import java.io.Serializable; import java.util.Optional; import java.util.function.BiPredicate; import java.util.stream.Stream; +import org.apache.commons.lang3.StringUtils; /** * HoodieTimeline is a view of meta-data instants in the hoodie dataset. Instants are specific @@ -42,6 +44,10 @@ public interface HoodieTimeline extends Serializable { String ROLLBACK_ACTION = "rollback"; String SAVEPOINT_ACTION = "savepoint"; String INFLIGHT_EXTENSION = ".inflight"; + // With Async Compaction, compaction instant can be in 3 states : + // (compaction-requested), (compaction-inflight), (completed) + String COMPACTION_ACTION = "compaction"; + String REQUESTED_EXTENSION = ".requested"; String COMMIT_EXTENSION = "." + COMMIT_ACTION; String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION; @@ -54,6 +60,12 @@ public interface HoodieTimeline extends Serializable { String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION; String INFLIGHT_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + INFLIGHT_EXTENSION; String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION; + String REQUESTED_COMPACTION_SUFFIX = + StringUtils.join(COMPACTION_ACTION, REQUESTED_EXTENSION); + String REQUESTED_COMPACTION_EXTENSION = + StringUtils.join(".", REQUESTED_COMPACTION_SUFFIX); + String INFLIGHT_COMPACTION_EXTENSION = + StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION); /** * Filter this timeline to just include the in-flights @@ -62,6 +74,13 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterInflights(); + /** + * Filter this timeline to just include the in-flights excluding compaction instants + * + * @return New instance of HoodieTimeline with just in-flights excluding compaction inflights + */ + HoodieTimeline filterInflightsExcludingCompaction(); + /** * Filter this timeline to just include the completed instants * @@ -69,6 +88,20 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterCompletedInstants(); + /** + * Filter this timeline to just include the completed + compaction (inflight + requested) instants + * A RT filesystem view is constructed with this timeline so that file-slice after pending compaction-requested + * instant-time is also considered valid. A RT file-system view for reading must then merge the file-slices before + * and after pending compaction instant so that all delta-commits are read. + * @return New instance of HoodieTimeline with just completed instants + */ + HoodieTimeline filterCompletedAndCompactionInstants(); + + /** + * Filter this timeline to just include inflight and requested compaction instants + * @return + */ + HoodieTimeline filterPendingCompactionTimeline(); /** * Create a new Timeline with instants after startTs and before or on endTs @@ -157,45 +190,60 @@ static HoodieInstant getCompletedInstant(final HoodieInstant instant) { return new HoodieInstant(false, instant.getAction(), instant.getTimestamp()); } + static HoodieInstant getCompactionRequestedInstant(final String timestamp) { + return new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, timestamp); + } + + static HoodieInstant getCompactionInflightInstant(final String timestamp) { + return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp); + } static HoodieInstant getInflightInstant(final HoodieInstant instant) { return new HoodieInstant(true, instant.getAction(), instant.getTimestamp()); } static String makeCommitFileName(String commitTime) { - return commitTime + HoodieTimeline.COMMIT_EXTENSION; + return StringUtils.join(commitTime, HoodieTimeline.COMMIT_EXTENSION); } static String makeInflightCommitFileName(String commitTime) { - return commitTime + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION; + return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); } static String makeCleanerFileName(String instant) { - return instant + HoodieTimeline.CLEAN_EXTENSION; + return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION); } static String makeInflightCleanerFileName(String instant) { - return instant + HoodieTimeline.INFLIGHT_CLEAN_EXTENSION; + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION); } static String makeRollbackFileName(String instant) { - return instant + HoodieTimeline.ROLLBACK_EXTENSION; + return StringUtils.join(instant, HoodieTimeline.ROLLBACK_EXTENSION); } static String makeInflightRollbackFileName(String instant) { - return instant + HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION; + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); } static String makeInflightSavePointFileName(String commitTime) { - return commitTime + HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION; + return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION); } static String makeSavePointFileName(String commitTime) { - return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION; + return StringUtils.join(commitTime, HoodieTimeline.SAVEPOINT_EXTENSION); } static String makeInflightDeltaFileName(String commitTime) { - return commitTime + HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION; + return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION); + } + + static String makeInflightCompactionFileName(String commitTime) { + return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); + } + + static String makeRequestedCompactionFileName(String commitTime) { + return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); } static String makeDeltaFileName(String commitTime) { @@ -211,8 +259,6 @@ static String makeFileNameAsComplete(String fileName) { } static String makeFileNameAsInflight(String fileName) { - return fileName + HoodieTimeline.INFLIGHT_EXTENSION; + return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION); } - - } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java index d9ffae790bd09..637f6e874ef0d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java @@ -73,6 +73,12 @@ interface RealtimeView { */ Stream getLatestFileSlices(String partitionPath); + /** + * Stream all the latest uncompacted file slices in the given partition + */ + Stream getLatestUnCompactedFileSlices(String partitionPath); + + /** * Stream all the latest file slices in the given partition with precondition that * commitTime(file) before maxCommitTime @@ -80,6 +86,16 @@ interface RealtimeView { Stream getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime); + /** + * Stream all "merged" file-slices before on an instant time + * If a file-group has a pending compaction request, the file-slice before and after compaction request instant + * is merged and returned. + * @param partitionPath Partition Path + * @param maxInstantTime Max Instant Time + * @return + */ + public Stream getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime); + /** * Stream all the latest file slices, in the given range */ diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index a1309c73f1061..427adda65dd09 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -64,6 +64,7 @@ public abstract class AbstractHoodieLogRecordScanner { // Reader schema for the records private final Schema readerSchema; // Latest valid instant time + // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark. private final String latestInstantTime; private final HoodieTableMetaClient hoodieTableMetaClient; // Merge strategy to use when combining records from log diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index a8d35411e1558..20dea2805d6c1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -95,7 +95,7 @@ public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { this(metaClient, new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, - CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); + CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION}); } /** @@ -118,19 +118,31 @@ private void readObject(java.io.ObjectInputStream in) /** * Get all instants (commits, delta commits) that produce new data, in the active timeline * + * */ public HoodieTimeline getCommitsTimeline() { return getTimelineOfActions( Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION)); } + /** + * Get all instants (commits, delta commits, in-flight/request compaction) that produce new data, in the active + * timeline * + * With Async compaction a requested/inflight compaction-instant is a valid baseInstant for a file-slice as there + * could be delta-commits with that baseInstant. + */ + public HoodieTimeline getCommitsAndCompactionTimeline() { + return getTimelineOfActions( + Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION)); + } + /** * Get all instants (commits, delta commits, clean, savepoint, rollback) that result in actions, * in the active timeline * */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions( - Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, + Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION)); } @@ -200,7 +212,7 @@ public void saveAsComplete(HoodieInstant instant, Optional data) { log.info("Marking instant complete " + instant); Preconditions.checkArgument(instant.isInflight(), "Could not mark an already completed instant as complete again " + instant); - moveInflightToComplete(instant, HoodieTimeline.getCompletedInstant(instant), data); + transitionState(instant, HoodieTimeline.getCompletedInstant(instant), data); log.info("Completed " + instant); } @@ -211,7 +223,18 @@ public void revertToInflight(HoodieInstant instant) { } public void deleteInflight(HoodieInstant instant) { - log.info("Deleting in-flight " + instant); + Preconditions.checkArgument(instant.isInflight()); + deleteInstantFile(instant); + } + + public void deleteCompactionRequested(HoodieInstant instant) { + Preconditions.checkArgument(instant.isRequested()); + Preconditions.checkArgument(instant.getAction() == HoodieTimeline.COMPACTION_ACTION); + deleteInstantFile(instant); + } + + private void deleteInstantFile(HoodieInstant instant) { + log.info("Deleting instant " + instant); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName()); try { boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false); @@ -232,24 +255,43 @@ public Optional getInstantDetails(HoodieInstant instant) { return readDataFromPath(detailPath); } - protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed, + public void revertFromInflightToRequested(HoodieInstant inflightInstant, HoodieInstant requestedInstant, + Optional data) { + Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + transitionState(inflightInstant, requestedInstant, data); + } + + public void transitionFromRequestedToInflight(HoodieInstant requestedInstant, HoodieInstant inflightInstant, + Optional data) { + Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + transitionState(requestedInstant, inflightInstant, data); + } + + protected void moveInflightToComplete(HoodieInstant inflightInstant, HoodieInstant commitInstant, Optional data) { - Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); + transitionState(inflightInstant, commitInstant, data); + } + + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, + Optional data) { + Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp())); + Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName()); try { // open a new file and write the commit metadata in - Path inflightCommitFile = new Path(metaClient.getMetaPath(), inflight.getFileName()); - createFileInMetaPath(inflight.getFileName(), data); + Path inflightCommitFile = new Path(metaClient.getMetaPath(), fromInstant.getFileName()); + createFileInMetaPath(fromInstant.getFileName(), data); boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath); if (!success) { throw new HoodieIOException( "Could not rename " + inflightCommitFile + " to " + commitFilePath); } } catch (IOException e) { - throw new HoodieIOException("Could not complete " + inflight, e); + throw new HoodieIOException("Could not complete " + fromInstant, e); } } protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { + Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp())); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName()); try { if (!metaClient.getFs().exists(inFlightCommitFilePath)) { @@ -269,6 +311,11 @@ public void saveToInflight(HoodieInstant instant, Optional content) { createFileInMetaPath(instant.getFileName(), content); } + public void saveToRequested(HoodieInstant instant, Optional content) { + Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + createFileInMetaPath(instant.getFileName(), content); + } + protected void createFileInMetaPath(String filename, Optional content) { Path fullPath = new Path(metaClient.getMetaPath(), filename); try { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java index c855203a3bad5..87c664c60fb93 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java @@ -53,15 +53,38 @@ public HoodieDefaultTimeline(Stream instants, public HoodieDefaultTimeline() { } + @Override public HoodieTimeline filterInflights() { return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight), details); } + @Override + public HoodieTimeline filterInflightsExcludingCompaction() { + return new HoodieDefaultTimeline(instants.stream().filter(instant -> { + return instant.isInflight() && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + }), details); + } + + @Override public HoodieTimeline filterCompletedInstants() { return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details); } + @Override + public HoodieTimeline filterCompletedAndCompactionInstants() { + return new HoodieDefaultTimeline(instants.stream().filter(s -> { + return !s.isInflight() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION); + }), details); + } + + @Override + public HoodieTimeline filterPendingCompactionTimeline() { + return new HoodieDefaultTimeline( + instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), + details); + } + @Override public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) { return new HoodieDefaultTimeline(instants.stream().filter( diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java index 59be65555d75d..f87d614da4522 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java @@ -30,7 +30,19 @@ */ public class HoodieInstant implements Serializable { - private boolean isInflight = false; + /** + * Instant State + */ + public enum State { + // Requested State (valid state for Compaction) + REQUESTED, + // Inflight instant + INFLIGHT, + // Committed instant + COMPLETED + } + + private State state = State.COMPLETED; private String action; private String timestamp; @@ -49,21 +61,35 @@ public HoodieInstant(FileStatus fileStatus) { // This is to support backwards compatibility on how in-flight commit files were written // General rule is inflight extension is ..inflight, but for commit it is .inflight action = "commit"; - isInflight = true; + state = State.INFLIGHT; } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) { - isInflight = true; + state = State.INFLIGHT; action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, ""); + } else if (action.equals(HoodieTimeline.REQUESTED_COMPACTION_SUFFIX)) { + state = State.REQUESTED; + action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, ""); } } public HoodieInstant(boolean isInflight, String action, String timestamp) { - this.isInflight = isInflight; + //TODO: vb - Preserving for avoiding cascading changes. This constructor will be updated in subsequent PR + this.state = isInflight ? State.INFLIGHT : State.COMPLETED; + this.action = action; + this.timestamp = timestamp; + } + + public HoodieInstant(State state, String action, String timestamp) { + this.state = state; this.action = action; this.timestamp = timestamp; } public boolean isInflight() { - return isInflight; + return state == State.INFLIGHT; + } + + public boolean isRequested() { + return state == State.REQUESTED; } public String getAction() { @@ -79,20 +105,28 @@ public String getTimestamp() { */ public String getFileName() { if (HoodieTimeline.COMMIT_ACTION.equals(action)) { - return isInflight ? HoodieTimeline.makeInflightCommitFileName(timestamp) + return isInflight() ? HoodieTimeline.makeInflightCommitFileName(timestamp) : HoodieTimeline.makeCommitFileName(timestamp); } else if (HoodieTimeline.CLEAN_ACTION.equals(action)) { - return isInflight ? HoodieTimeline.makeInflightCleanerFileName(timestamp) + return isInflight() ? HoodieTimeline.makeInflightCleanerFileName(timestamp) : HoodieTimeline.makeCleanerFileName(timestamp); } else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) { - return isInflight ? HoodieTimeline.makeInflightRollbackFileName(timestamp) + return isInflight() ? HoodieTimeline.makeInflightRollbackFileName(timestamp) : HoodieTimeline.makeRollbackFileName(timestamp); } else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) { - return isInflight ? HoodieTimeline.makeInflightSavePointFileName(timestamp) + return isInflight() ? HoodieTimeline.makeInflightSavePointFileName(timestamp) : HoodieTimeline.makeSavePointFileName(timestamp); } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { - return isInflight ? HoodieTimeline.makeInflightDeltaFileName(timestamp) + return isInflight() ? HoodieTimeline.makeInflightDeltaFileName(timestamp) : HoodieTimeline.makeDeltaFileName(timestamp); + } else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) { + if (isInflight()) { + return HoodieTimeline.makeInflightCompactionFileName(timestamp); + } else if (isRequested()) { + return HoodieTimeline.makeRequestedCompactionFileName(timestamp); + } else { + return HoodieTimeline.makeCommitFileName(timestamp); + } } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } @@ -106,18 +140,18 @@ public boolean equals(Object o) { return false; } HoodieInstant that = (HoodieInstant) o; - return isInflight == that.isInflight + return state == that.state && Objects.equals(action, that.action) && Objects.equals(timestamp, that.timestamp); } @Override public int hashCode() { - return Objects.hash(isInflight, action, timestamp); + return Objects.hash(state, action, timestamp); } @Override public String toString() { - return "[" + ((isInflight) ? "==>" : "") + timestamp + "__" + action + "]"; + return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]"; } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index 8978bf91af2d7..fe6e326a09c0d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -64,6 +64,11 @@ public class HoodieTableFileSystemView implements TableFileSystemView, // mapping from file id to the file group. protected HashMap fileGroupMap; + /** + * File Id to pending compaction instant time + */ + private final Map fileIdToPendingCompactionInstantTime; + /** * Create a file system view, as of the given timeline */ @@ -73,6 +78,8 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, this.visibleActiveTimeline = visibleActiveTimeline; this.fileGroupMap = new HashMap<>(); this.partitionToFileGroupsMap = new HashMap<>(); + //TODO: vb Will be implemented in next PR + this.fileIdToPendingCompactionInstantTime = new HashMap<>(); } @@ -128,14 +135,19 @@ private List addFilesToView(FileStatus[] statuses) { List fileGroups = new ArrayList<>(); fileIdSet.forEach(pair -> { - HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), - visibleActiveTimeline); + String fileId = pair.getValue(); + HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, visibleActiveTimeline); if (dataFiles.containsKey(pair)) { dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); } if (logFiles.containsKey(pair)) { logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); } + if (fileIdToPendingCompactionInstantTime.containsKey(fileId)) { + // If there is no delta-commit after compaction request, this step would ensure a new file-slice appears + // so that any new ingestion uses the correct base-instant + group.addNewFileSliceAtInstant(fileIdToPendingCompactionInstantTime.get(fileId)); + } fileGroups.add(group); }); @@ -165,19 +177,37 @@ private Stream convertFileStatusesToLogFiles(FileStatus[] statuse return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); } + /** + * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore + * those data-files + * + * @param dataFile Data File + */ + private boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) { + String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(dataFile.getFileId()); + if ((null != compactionInstantTime) && dataFile.getCommitTime().equals(compactionInstantTime)) { + return true; + } + return false; + } + @Override public Stream getLatestDataFiles(final String partitionPath) { return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestDataFile()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(fileGroup -> { + return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst(); + }) + .filter(Optional::isPresent) .map(Optional::get); } @Override public Stream getLatestDataFiles() { return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestDataFile()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(fileGroup -> { + return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst(); + }) + .filter(Optional::isPresent) .map(Optional::get); } @@ -185,16 +215,29 @@ public Stream getLatestDataFiles() { public Stream getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) { return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(fileGroup -> { + return fileGroup.getAllDataFiles() + .filter(dataFile -> + HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), + maxCommitTime, + HoodieTimeline.LESSER_OR_EQUAL)) + .filter(df -> !isDataFileDueToPendingCompaction(df)) + .findFirst(); + }) + .filter(Optional::isPresent) .map(Optional::get); } @Override public Stream getLatestDataFilesInRange(List commitsToReturn) { return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(fileGroup -> { + return fileGroup.getAllDataFiles() + .filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime()) + && !isDataFileDueToPendingCompaction(dataFile)) + .findFirst(); + }) + .filter(Optional::isPresent) .map(Optional::get); } @@ -202,23 +245,125 @@ public Stream getLatestDataFilesInRange(List commitsToRe public Stream getAllDataFiles(String partitionPath) { return getAllFileGroups(partitionPath) .map(fileGroup -> fileGroup.getAllDataFiles()) - .flatMap(dataFileList -> dataFileList); + .flatMap(dataFileList -> dataFileList) + .filter(df -> !isDataFileDueToPendingCompaction(df)); } @Override public Stream getLatestFileSlices(String partitionPath) { return getAllFileGroups(partitionPath) .map(fileGroup -> fileGroup.getLatestFileSlice()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) + .filter(Optional::isPresent) + .map(Optional::get) + .map(this::filterDataFileAfterPendingCompaction); + } + + @Override + public Stream getLatestUnCompactedFileSlices(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> { + FileSlice fileSlice = fileGroup.getLatestFileSlice().get(); + // if the file-group is under compaction, pick the latest before compaction instant time. + if (isFileSliceAfterPendingCompaction(fileSlice)) { + String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(fileSlice.getFileId()); + return fileGroup.getLatestFileSliceBefore(compactionInstantTime); + } + return Optional.of(fileSlice); + }) .map(Optional::get); } + /** + * Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches + * compaction Instant + * @param fileSlice File Slice + * @return + */ + private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) { + String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(fileSlice.getFileId()); + if ((null != compactionInstantTime) && fileSlice.getBaseInstantTime().equals(compactionInstantTime)) { + return true; + } + return false; + } + + /** + * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, + * Ignore those data-files + * @param fileSlice File Slice + * @return + */ + private FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) { + if (isFileSliceAfterPendingCompaction(fileSlice)) { + // Data file is filtered out of the file-slice as the corresponding compaction + // instant not completed yet. + FileSlice transformed = new FileSlice(fileSlice.getBaseInstantTime(), fileSlice.getFileId()); + fileSlice.getLogFiles().forEach(lf -> transformed.addLogFile(lf)); + return transformed; + } + return fileSlice; + } + @Override public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) { return getAllFileGroups(partitionPath) .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) + .filter(Optional::isPresent) + .map(Optional::get) + .map(this::filterDataFileAfterPendingCompaction); + } + + /** + * Helper to merge last 2 file-slices. These 2 file-slices do not have compaction done yet. + * + * @param lastSlice Latest File slice for a file-group + * @param penultimateSlice Penultimate file slice for a file-group in commit timeline order + */ + private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) { + FileSlice merged = new FileSlice(penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId()); + if (penultimateSlice.getDataFile().isPresent()) { + merged.setDataFile(penultimateSlice.getDataFile().get()); + } + // Add Log files from penultimate and last slices + penultimateSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf)); + lastSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf)); + return merged; + } + + /** + * If the file-slice is because of pending compaction instant, this method merges the file-slice with the one before + * the compaction instant time + * @param fileGroup File Group for which the file slice belongs to + * @param fileSlice File Slice which needs to be merged + * @return + */ + private FileSlice getMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) { + // if the file-group is under construction, pick the latest before compaction instant time. + if (fileIdToPendingCompactionInstantTime.containsKey(fileSlice.getFileId())) { + String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(fileSlice.getFileId()); + if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) { + Optional prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime); + if (prevFileSlice.isPresent()) { + return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get()); + } + } + } + return fileSlice; + } + + @Override + public Stream getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> { + Optional fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); + // if the file-group is under construction, pick the latest before compaction instant time. + if (fileSlice.isPresent()) { + fileSlice = Optional.of(getMergedFileSlice(fileGroup, fileSlice.get())); + } + return fileSlice; + }) + .filter(Optional::isPresent) .map(Optional::get); } @@ -226,7 +371,6 @@ public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, public Stream getLatestFileSliceInRange(List commitsToReturn) { return fileGroupMap.values().stream() .map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) .map(Optional::get); } @@ -260,4 +404,15 @@ public Stream getAllFileGroups(String partitionPathStr) { "Failed to list data files in partition " + partitionPathStr, e); } } + + /** + * Used by tests to add pending compaction entries TODO: This method is temporary and should go away in subsequent + * Async Compaction PR + * + * @param fileId File Id + * @param compactionInstantTime Compaction Instant Time + */ + protected void addPendingCompactionFileId(String fileId, String compactionInstantTime) { + fileIdToPendingCompactionInstantTime.put(fileId, compactionInstantTime); + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index df642490b4975..37a0152e4a4b3 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -80,7 +80,7 @@ public void testLoadingInstantsFromFiles() throws IOException { Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete), timeline.getCommitTimeline().filterCompletedInstants().getInstants()); HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5), - timeline.getCommitTimeline().filterInflights().getInstants()); + timeline.getCommitTimeline().filterInflightsExcludingCompaction().getInstants()); } @Test @@ -106,7 +106,7 @@ public void testTimelineOperations() throws Exception { timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2).getInstants() .map(HoodieInstant::getTimestamp)); assertFalse(timeline.empty()); - assertFalse(timeline.getCommitTimeline().filterInflights().empty()); + assertFalse(timeline.getCommitTimeline().filterInflightsExcludingCompaction().empty()); assertEquals("", 12, timeline.countInstants()); HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants(); assertEquals("", 10, activeCommitTimeline.countInstants()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 6862412716963..757a3caae274c 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -32,15 +32,18 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.util.FSUtils; import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.junit.Before; @@ -53,7 +56,7 @@ public class HoodieTableFileSystemViewTest { private HoodieTableMetaClient metaClient; private String basePath; - private TableFileSystemView fsView; + private HoodieTableFileSystemView fsView; private TableFileSystemView.ReadOptimizedView roView; private TableFileSystemView.RealtimeView rtView; @@ -74,15 +77,419 @@ private void refreshFsView(FileStatus[] statuses) { metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); if (statuses != null) { fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), statuses); + metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(), + statuses); } else { fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); + metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants()); } roView = (TableFileSystemView.ReadOptimizedView) fsView; rtView = (TableFileSystemView.RealtimeView) fsView; } + /** + * Test case for view generation on a file group where + * the only file-slice does not have data-file. This is the case where upserts directly go to log-files + */ + @Test + public void testViewForFileSlicesWithNoBaseFile() throws Exception { + String partitionPath = "2016/05/01"; + new File(basePath + "/" + partitionPath).mkdirs(); + String fileId = UUID.randomUUID().toString(); + + String instantTime1 = "1"; + String deltaInstantTime1 = "2"; + String deltaInstantTime2 = "3"; + String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0); + String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 1); + new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); + new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + + commitTimeline.saveAsComplete(instant1, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant2, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant3, Optional.empty()); + + refreshFsView(null); + + List dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + assertTrue("No data file expected", dataFiles.isEmpty()); + List fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals(1, fileSliceList.size()); + FileSlice fileSlice = fileSliceList.get(0); + assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId()); + assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent()); + assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime()); + List logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size()); + assertEquals("Log File Order check", fileName2, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName1, logFiles.get(1).getFileName()); + + // Check Merged File Slices API + fileSliceList = rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime2) + .collect(Collectors.toList()); + assertEquals(1, fileSliceList.size()); + fileSlice = fileSliceList.get(0); + assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId()); + assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent()); + assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime()); + logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size()); + assertEquals("Log File Order check", fileName2, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName1, logFiles.get(1).getFileName()); + + // Check UnCompacted File Slices API + fileSliceList = rtView.getLatestUnCompactedFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals(1, fileSliceList.size()); + fileSlice = fileSliceList.get(0); + assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId()); + assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent()); + assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime()); + logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size()); + assertEquals("Log File Order check", fileName2, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName1, logFiles.get(1).getFileName()); + } + + @Test + public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction() throws Exception { + testViewForFileSlicesWithAsyncCompaction(true, false); + } + + @Test + public void testViewForFileSlicesWithBaseFileAndRequestedCompaction() throws Exception { + testViewForFileSlicesWithAsyncCompaction(false, false); + } + + @Test + public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction() throws Exception { + testViewForFileSlicesWithAsyncCompaction(true, true); + } + + @Test + public void testViewForFileSlicesWithBaseFileAndInflightCompaction() throws Exception { + testViewForFileSlicesWithAsyncCompaction(false, true); + } + + /** + * Returns all file-slices including uncommitted ones. + * @param partitionPath + * @return + */ + private Stream getAllRawFileSlices(String partitionPath) { + return fsView.getAllFileGroups(partitionPath) + .map(group -> group.getAllFileSlicesIncludingInflight()) + .flatMap(sliceList -> sliceList); + } + + /** + * Returns latest raw file-slices including uncommitted ones. + * @param partitionPath + * @return + */ + public Stream getLatestRawFileSlices(String partitionPath) { + return fsView.getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestFileSlicesIncludingInflight()) + .filter(fileSliceOpt -> fileSliceOpt.isPresent()) + .map(Optional::get); + } + + /** + * Helper method to test Views in the presence of concurrent compaction + * @param skipCreatingDataFile if set, first File Slice will not have data-file set. This would + * simulate inserts going directly to log files + * @param isCompactionInFlight if set, compaction was inflight (running) when view was tested first time, + * otherwise compaction was in requested state + * @throws Exception + */ + private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile, + boolean isCompactionInFlight) throws Exception { + String partitionPath = "2016/05/01"; + new File(basePath + "/" + partitionPath).mkdirs(); + String fileId = UUID.randomUUID().toString(); + + // if skipCreatingDataFile, then instantTime1 below acts like delta-commit, otherwise it is base-commit + String instantTime1 = "1"; + String deltaInstantTime1 = "2"; + String deltaInstantTime2 = "3"; + + String dataFileName = null; + if (!skipCreatingDataFile) { + dataFileName = FSUtils.makeDataFileName(instantTime1, 1, fileId); + new File(basePath + "/" + partitionPath + "/" + dataFileName).createNewFile(); + } + String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0); + String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 1); + new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); + new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + + commitTimeline.saveAsComplete(instant1, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant2, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant3, Optional.empty()); + + // Fake delta-ingestion after compaction-requested + String compactionRequestedTime = "4"; + String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId); + HoodieInstant compactionInstant = null; + if (isCompactionInFlight) { + // Create a Data-file but this should be skipped by view + new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile(); + compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); + commitTimeline.saveToInflight(compactionInstant, Optional.empty()); + } else { + compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); + commitTimeline.saveToRequested(compactionInstant, Optional.empty()); + } + String deltaInstantTime4 = "5"; + String deltaInstantTime5 = "6"; + List allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2, + compactionRequestedTime, deltaInstantTime4, deltaInstantTime5); + String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 0); + String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 1); + new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile(); + HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4); + HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5); + commitTimeline.saveAsComplete(deltaInstant4, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant5, Optional.empty()); + refreshFsView(null); + fsView.addPendingCompactionFileId(fileId, compactionRequestedTime); + + List dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList()); + if (skipCreatingDataFile) { + assertTrue("No data file expected", dataFiles.isEmpty()); + } else { + assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size()); + assertEquals("Expect only valid data-file", dataFileName, dataFiles.get(0).getFileName()); + } + + /** Merge API Tests **/ + List fileSliceList = rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5) + .collect(Collectors.toList()); + assertEquals("Expect file-slice to be merged", 1, fileSliceList.size()); + FileSlice fileSlice = fileSliceList.get(0); + assertEquals(fileId, fileSlice.getFileId()); + if (!skipCreatingDataFile) { + assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName()); + } else { + assertFalse("No data-file expected as it was not created", fileSlice.getDataFile().isPresent()); + } + assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1, + fileSlice.getBaseInstantTime()); + List logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Log files must include those after compaction request", 4, logFiles.size()); + assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); + assertEquals("Log File Order check", fileName2, logFiles.get(2).getFileName()); + assertEquals("Log File Order check", fileName1, logFiles.get(3).getFileName()); + + fileSliceList = rtView.getLatestFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5) + .collect(Collectors.toList()); + assertEquals("Expect only one file-id", 1, fileSliceList.size()); + fileSlice = fileSliceList.get(0); + assertEquals(fileId, fileSlice.getFileId()); + assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent()); + assertEquals("Compaction requested instant must be base instant", compactionRequestedTime, + fileSlice.getBaseInstantTime()); + logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Log files must include only those after compaction request", 2, logFiles.size()); + assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); + + /** Data Files API tests */ + dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + if (skipCreatingDataFile) { + assertEquals("Expect no data file to be returned", 0, dataFiles.size()); + } else { + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); + }); + } + dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList()); + if (skipCreatingDataFile) { + assertEquals("Expect no data file to be returned", 0, dataFiles.size()); + } else { + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); + }); + } + dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); + if (skipCreatingDataFile) { + assertEquals("Expect no data file to be returned", 0, dataFiles.size()); + } else { + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); + }); + } + dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList()); + if (skipCreatingDataFile) { + assertEquals("Expect no data file to be returned", 0, dataFiles.size()); + } else { + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); + }); + } + + /** Inflight/Orphan File-groups needs to be in the view **/ + + // There is a data-file with this inflight file-id + final String inflightFileId1 = UUID.randomUUID().toString(); + // There is a log-file with this inflight file-id + final String inflightFileId2 = UUID.randomUUID().toString(); + // There is an orphan data file with this file-id + final String orphanFileId1 = UUID.randomUUID().toString(); + // There is an orphan log data file with this file-id + final String orphanFileId2 = UUID.randomUUID().toString(); + final String invalidInstantId = "INVALIDTIME"; + String inflightDeltaInstantTime = "7"; + String orphanDataFileName = FSUtils.makeDataFileName(invalidInstantId, 1, orphanFileId1); + new File(basePath + "/" + partitionPath + "/" + orphanDataFileName).createNewFile(); + String orphanLogFileName = + FSUtils.makeLogFileName(orphanFileId2, HoodieLogFile.DELTA_EXTENSION, invalidInstantId, 0); + new File(basePath + "/" + partitionPath + "/" + orphanLogFileName).createNewFile(); + String inflightDataFileName = FSUtils.makeDataFileName(inflightDeltaInstantTime, 1, inflightFileId1); + new File(basePath + "/" + partitionPath + "/" + inflightDataFileName).createNewFile(); + String inflightLogFileName = + FSUtils.makeLogFileName(inflightFileId2, HoodieLogFile.DELTA_EXTENSION, inflightDeltaInstantTime, 0); + new File(basePath + "/" + partitionPath + "/" + inflightLogFileName).createNewFile(); + // Mark instant as inflight + commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, + inflightDeltaInstantTime), Optional.empty()); + refreshFsView(null); + fsView.addPendingCompactionFileId(fileId, compactionRequestedTime); + + List allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList()); + dataFiles = allRawFileSlices.stream().flatMap(slice -> { + if (slice.getDataFile().isPresent()) { + return Stream.of(slice.getDataFile().get()); + } + return Stream.empty(); + }).collect(Collectors.toList()); + assertEquals("Inflight/Orphan data-file is also expected", 2 + + (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size()); + Set fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet()); + assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName)); + assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName)); + if (!skipCreatingDataFile) { + assertTrue("Expect old committed data-file", fileNames.contains(dataFileName)); + } + + if (isCompactionInFlight) { + assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName)); + } + + fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals("Expect both inflight and orphan file-slice to be included", + 5, fileSliceList.size()); + Map fileSliceMap = + fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r)); + FileSlice orphanFileSliceWithDataFile = fileSliceMap.get(orphanFileId1); + FileSlice orphanFileSliceWithLogFile = fileSliceMap.get(orphanFileId2); + FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1); + FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2); + + assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId, + orphanFileSliceWithDataFile.getBaseInstantTime()); + assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName, + orphanFileSliceWithDataFile.getDataFile().get().getFileName()); + assertEquals("Orphan File Slice with data-file check data-file", 0, + orphanFileSliceWithDataFile.getLogFiles().count()); + assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime, + inflightFileSliceWithDataFile.getBaseInstantTime()); + assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName, + inflightFileSliceWithDataFile.getDataFile().get().getFileName()); + assertEquals("Inflight File Slice with data-file check data-file", 0, + inflightFileSliceWithDataFile.getLogFiles().count()); + assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId, + orphanFileSliceWithLogFile.getBaseInstantTime()); + assertFalse("Orphan File Slice with log-file check data-file", + orphanFileSliceWithLogFile.getDataFile().isPresent()); + logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList()); + assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size()); + assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName, + logFiles.get(0).getFileName()); + assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime, + inflightFileSliceWithLogFile.getBaseInstantTime()); + assertFalse("Inflight File Slice with log-file check data-file", + inflightFileSliceWithLogFile.getDataFile().isPresent()); + logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList()); + assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size()); + assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName, + logFiles.get(0).getFileName()); + + // Now simulate Compaction completing - Check the view + if (!isCompactionInFlight) { + // For inflight compaction, we already create a data-file to test concurrent inflight case. + // If we skipped creating data file corresponding to compaction commit, create it now + new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile(); + } + if (isCompactionInFlight) { + commitTimeline.deleteInflight(compactionInstant); + } else { + commitTimeline.deleteCompactionRequested(compactionInstant); + } + compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); + commitTimeline.saveAsComplete(compactionInstant, Optional.empty()); + refreshFsView(null); + // populate the cache + roView.getAllDataFiles(partitionPath); + + fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); + dataFiles = fileSliceList.stream().map(FileSlice::getDataFile) + .filter(Optional::isPresent).map(Optional::get).collect(Collectors.toList()); + System.out.println("fileSliceList : " + fileSliceList); + assertEquals("Expect only one data-files in latest view as there is only one file-group", 1, dataFiles.size()); + assertEquals("Data Filename must match", compactDataFileName, dataFiles.get(0).getFileName()); + assertEquals("Only one latest file-slice in the partition", 1, fileSliceList.size()); + fileSlice = fileSliceList.get(0); + assertEquals("Check file-Id is set correctly", fileId, fileSlice.getFileId()); + assertEquals("Check data-filename is set correctly", + compactDataFileName, fileSlice.getDataFile().get().getFileName()); + assertEquals("Ensure base-instant is now compaction request instant", + compactionRequestedTime, fileSlice.getBaseInstantTime()); + logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Only log-files after compaction request shows up", 2, logFiles.size()); + assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); + + /** Data Files API tests */ + dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), + compactionRequestedTime); + }); + dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList()); + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), + compactionRequestedTime); + }); + dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), + compactionRequestedTime); + }); + dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList()); + assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); + dataFiles.stream().forEach(df -> { + assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), + compactionRequestedTime); + }); + } + @Test public void testGetLatestDataFilesForFileId() throws IOException { String partitionPath = "2016/05/01"; @@ -328,15 +735,15 @@ public void streamLatestVersionInRange() throws IOException { assertEquals(3, slices.size()); for (FileSlice slice : slices) { if (slice.getFileId().equals(fileId1)) { - assertEquals(slice.getBaseCommitTime(), commitTime3); + assertEquals(slice.getBaseInstantTime(), commitTime3); assertTrue(slice.getDataFile().isPresent()); assertEquals(slice.getLogFiles().count(), 0); } else if (slice.getFileId().equals(fileId2)) { - assertEquals(slice.getBaseCommitTime(), commitTime4); + assertEquals(slice.getBaseInstantTime(), commitTime4); assertFalse(slice.getDataFile().isPresent()); assertEquals(slice.getLogFiles().count(), 1); } else if (slice.getFileId().equals(fileId3)) { - assertEquals(slice.getBaseCommitTime(), commitTime4); + assertEquals(slice.getBaseInstantTime(), commitTime4); assertTrue(slice.getDataFile().isPresent()); assertEquals(slice.getLogFiles().count(), 0); } @@ -433,17 +840,17 @@ public void streamLatestVersions() throws IOException { List slices = fileGroup.getAllFileSlices().collect(Collectors.toList()); if (fileGroup.getId().equals(fileId1)) { assertEquals(2, slices.size()); - assertEquals(commitTime4, slices.get(0).getBaseCommitTime()); - assertEquals(commitTime1, slices.get(1).getBaseCommitTime()); + assertEquals(commitTime4, slices.get(0).getBaseInstantTime()); + assertEquals(commitTime1, slices.get(1).getBaseInstantTime()); } else if (fileGroup.getId().equals(fileId2)) { assertEquals(3, slices.size()); - assertEquals(commitTime3, slices.get(0).getBaseCommitTime()); - assertEquals(commitTime2, slices.get(1).getBaseCommitTime()); - assertEquals(commitTime1, slices.get(2).getBaseCommitTime()); + assertEquals(commitTime3, slices.get(0).getBaseInstantTime()); + assertEquals(commitTime2, slices.get(1).getBaseInstantTime()); + assertEquals(commitTime1, slices.get(2).getBaseInstantTime()); } else if (fileGroup.getId().equals(fileId3)) { assertEquals(2, slices.size()); - assertEquals(commitTime4, slices.get(0).getBaseCommitTime()); - assertEquals(commitTime3, slices.get(1).getBaseCommitTime()); + assertEquals(commitTime4, slices.get(0).getBaseInstantTime()); + assertEquals(commitTime3, slices.get(1).getBaseInstantTime()); } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index a6f12b8f9d996..e423bd01804ed 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -24,6 +24,7 @@ import com.uber.hoodie.common.model.HoodieRecord; 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.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; @@ -108,7 +109,13 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { .getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); try { - Stream latestFileSlices = fsView.getLatestFileSlices(relPartitionPath); + // Both commit and delta-commits are included - pick the latest completed one + Optional latestCompletedInstant = + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + + Stream latestFileSlices = latestCompletedInstant.map(instant -> + fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp())) + .orElse(Stream.empty()); // subgroup splits again by file id & match with log files. Map> groupedInputSplits = partitionsToParquetSplits From 68630ee5a5586b5d9933f92a4fca860a3dab57b2 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 23 May 2018 20:49:24 -0700 Subject: [PATCH 068/374] (1) Define CompactionWorkload in avro to allow storing them in instant files. (2) Split APIs in HoodieRealtimeCompactor to separate generating compaction workload from running compaction --- .../hoodie/config/HoodieCompactionConfig.java | 1 + .../hoodie/io/compact/HoodieCompactor.java | 38 ++++--- .../compact/HoodieRealtimeTableCompactor.java | 65 ++++++++---- .../strategy/BoundedIOCompactionStrategy.java | 13 +-- .../compact/strategy/CompactionStrategy.java | 54 +++++++--- .../strategy/DayBasedCompactionStrategy.java | 11 +- .../LogFileSizeBasedCompactionStrategy.java | 27 ++--- .../strategy/UnBoundedCompactionStrategy.java | 7 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 12 +++ .../hoodie/table/HoodieMergeOnReadTable.java | 26 ++++- .../com/uber/hoodie/table/HoodieTable.java | 21 ++++ .../TestHoodieCompactionStrategy.java | 69 +++++++----- .../main/avro/HoodieCompactionOperation.avsc | 66 ++++++++++++ .../common/model}/CompactionOperation.java | 50 +++++---- .../uber/hoodie/common/util/AvroUtils.java | 11 +- .../hoodie/common/util/CompactionUtils.java | 100 ++++++++++++++++++ 16 files changed, 443 insertions(+), 128 deletions(-) create mode 100644 hoodie-common/src/main/avro/HoodieCompactionOperation.avsc rename {hoodie-client/src/main/java/com/uber/hoodie/io/compact => hoodie-common/src/main/java/com/uber/hoodie/common/model}/CompactionOperation.java (65%) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index d1584aac02649..97c5e5c68b3fe 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -108,6 +108,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { + ".partitions"; // 500GB of target IO per compaction (both read and write) public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10); + public static final String DEFAULT_COMPACTOR_ID = "default"; private HoodieCompactionConfig(Properties props) { super(props); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index 7475176c44efa..ad1c18e2fc3e0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -17,13 +17,11 @@ package com.uber.hoodie.io.compact; import com.uber.hoodie.WriteStatus; -import com.uber.hoodie.common.table.HoodieTimeline; -import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; -import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; import java.io.Serializable; -import java.util.Date; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -34,17 +32,31 @@ public interface HoodieCompactor extends Serializable { /** * Compact the delta files with the data files + * + * @deprecated : Will be removed in next PR */ + @Deprecated JavaRDD compact(JavaSparkContext jsc, final HoodieWriteConfig config, HoodieTable hoodieTable, String compactionCommitTime) throws Exception; + /** + * Generate a new compaction plan for scheduling + * + * @param jsc Spark Context + * @param hoodieTable Hoodie Table + * @param config Hoodie Write Configuration + * @param compactionCommitTime scheduled compaction commit time + * @return Compaction Plan + * @throws IOException when encountering errors + */ + HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, + HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) + throws IOException; - // Helper methods - default String startCompactionCommit(HoodieTable hoodieTable) { - String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); - HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline(); - activeTimeline - .createInflight(new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime)); - return commitTime; - } -} + /** + * Execute compaction operations and report back status + */ + JavaRDD compact(JavaSparkContext jsc, + HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config, + String compactionCommitTime) throws IOException; +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 5a60f0d8a996f..b314e8eb34914 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -22,6 +22,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.model.CompactionOperation; +import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; @@ -29,6 +33,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; +import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -36,9 +41,11 @@ import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.avro.Schema; @@ -70,26 +77,25 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable hoodieTable, String compactionCommitTime) throws IOException { - totalLogFiles = new LongAccumulator(); - totalFileSlices = new LongAccumulator(); - jsc.sc().register(totalLogFiles); - jsc.sc().register(totalFileSlices); - - List operations = getCompactionWorkload(jsc, hoodieTable, config, + HoodieCompactionPlan compactionPlan = generateCompactionPlan(jsc, hoodieTable, config, compactionCommitTime); - if (operations == null) { + List operations = compactionPlan.getOperations(); + if ((operations == null) || (operations.isEmpty())) { return jsc.emptyRDD(); } - return executeCompaction(jsc, operations, hoodieTable, config, compactionCommitTime); + return compact(jsc, compactionPlan, hoodieTable, config, compactionCommitTime); } - private JavaRDD executeCompaction(JavaSparkContext jsc, - List operations, HoodieTable hoodieTable, HoodieWriteConfig config, + @Override + public JavaRDD compact(JavaSparkContext jsc, + HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) throws IOException { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); // Compacting is very similar to applying updates to existing file HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); - log.info("After filtering, Compacting " + operations + " files"); + List operations = compactionPlan.getOperations().stream() + .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); + log.info("Compactor " + compactionPlan.getCompactorId() + " running, Compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) .map(s -> compact(table, metaClient, config, s, compactionCommitTime)) .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); @@ -144,8 +150,8 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); s.getStat().setPartitionPath(operation.getPartitionPath()); - s.getStat().setTotalLogSizeCompacted((long) operation.getMetrics().get( - CompactionStrategy.TOTAL_LOG_FILE_SIZE)); + s.getStat().setTotalLogSizeCompacted(operation.getMetrics().get( + CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue()); s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks()); s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks()); s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks()); @@ -156,10 +162,16 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, }).collect(toList()); } - private List getCompactionWorkload(JavaSparkContext jsc, + @Override + public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) throws IOException { + totalLogFiles = new LongAccumulator(); + totalFileSlices = new LongAccumulator(); + jsc.sc().register(totalLogFiles); + jsc.sc().register(totalFileSlices); + Preconditions .checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, "HoodieRealtimeTableCompactor can only compact table of type " @@ -176,7 +188,7 @@ private List getCompactionWorkload(JavaSparkContext jsc, TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - List operations = + List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) .flatMap((FlatMapFunction) partitionPath -> fileSystemView .getLatestFileSlices(partitionPath).map( @@ -185,10 +197,16 @@ private List getCompactionWorkload(JavaSparkContext jsc, .getBaseInstantAndLogVersionComparator().reversed()).collect(Collectors.toList()); totalLogFiles.add((long) logFiles.size()); totalFileSlices.add(1L); - return new CompactionOperation(s.getDataFile(), partitionPath, logFiles, config); + // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO + // for spark Map operations and collecting them finally in Avro generated classes for storing + // into meta files. + Optional dataFile = s.getDataFile(); + return new CompactionOperation(dataFile, partitionPath, logFiles, + config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); }) .filter(c -> !c.getDeltaFilePaths().isEmpty()) - .collect(toList()).iterator()).collect(); + .collect(toList()).iterator()).collect().stream().map(CompactionUtils::buildHoodieCompactionOperation) + .collect(toList()); log.info("Total of " + operations.size() + " compactions are retrieved"); log.info("Total number of latest files slices " + totalFileSlices.value()); log.info("Total number of log files " + totalLogFiles.value()); @@ -196,12 +214,13 @@ private List getCompactionWorkload(JavaSparkContext jsc, // Filter the compactions with the passed in filter. This lets us choose most effective // compactions only - operations = config.getCompactionStrategy().orderAndFilter(config, operations); - if (operations.isEmpty()) { + // TODO: In subsequent PRs, pending Compaction plans will be wired in. Strategy can look at pending compaction + // plans to schedule next compaction plan + HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations, + new ArrayList<>()); + if (compactionPlan.getOperations().isEmpty()) { log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); - return null; } - return operations; + return compactionPlan; } - -} \ No newline at end of file +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java index 4f4cdf12801c8..409d786dacd8e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java @@ -17,8 +17,9 @@ package com.uber.hoodie.io.compact.strategy; import com.google.common.collect.Lists; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.compact.CompactionOperation; import java.util.List; /** @@ -30,15 +31,15 @@ public class BoundedIOCompactionStrategy extends CompactionStrategy { @Override - public List orderAndFilter(HoodieWriteConfig writeConfig, - List operations) { + public List orderAndFilter(HoodieWriteConfig writeConfig, + List operations, List pendingCompactionPlans) { // Iterate through the operations in order and accept operations as long as we are within the // IO limit // Preserves the original ordering of compactions - List finalOperations = Lists.newArrayList(); + List finalOperations = Lists.newArrayList(); long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB(); - for (CompactionOperation op : operations) { - long opIo = (Long) op.getMetrics().get(TOTAL_IO_MB); + for (HoodieCompactionOperation op : operations) { + long opIo = op.getMetrics().get(TOTAL_IO_MB).longValue(); targetIORemaining -= opIo; finalOperations.add(op); if (targetIORemaining <= 0) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index 2a7e659545b45..2b5070dbaf8f2 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -17,11 +17,13 @@ package com.uber.hoodie.io.compact.strategy; import com.google.common.collect.Maps; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.compact.CompactionOperation; import java.io.Serializable; import java.util.List; import java.util.Map; @@ -35,7 +37,6 @@ * passed in every time * * @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor - * @see CompactionOperation */ public abstract class CompactionStrategy implements Serializable { @@ -46,7 +47,7 @@ public abstract class CompactionStrategy implements Serializable { public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES"; /** - * Callback hook when a CompactionOperation is created. Individual strategies can capture the + * Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the * metrics they need to decide on the priority. * * @param dataFile - Base file to compact @@ -54,9 +55,9 @@ public abstract class CompactionStrategy implements Serializable { * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieWriteConfig writeConfig, Optional dataFile, String - partitionPath, List logFiles) { - Map metrics = Maps.newHashMap(); + public Map captureMetrics(HoodieWriteConfig writeConfig, Optional dataFile, + String partitionPath, List logFiles) { + Map metrics = Maps.newHashMap(); Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); // Total size of all the log files Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent) @@ -70,25 +71,44 @@ public Map captureMetrics(HoodieWriteConfig writeConfig, Optiona // Total IO will the the IO for read + write Long totalIO = totalIORead + totalIOWrite; // Save these metrics and we will use during the filter - metrics.put(TOTAL_IO_READ_MB, totalIORead); - metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite); - metrics.put(TOTAL_IO_MB, totalIO); - metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize); - metrics.put(TOTAL_LOG_FILES, logFiles.size()); + metrics.put(TOTAL_IO_READ_MB, totalIORead.doubleValue()); + metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue()); + metrics.put(TOTAL_IO_MB, totalIO.doubleValue()); + metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); + metrics.put(TOTAL_LOG_FILES, Double.valueOf(logFiles.size())); return metrics; + } + /** + * Generate Compaction plan. Allows clients to order and filter the list of compactions to be set. The default + * implementation takes care of setting compactor Id from configuration allowing subclasses to only worry about + * ordering and filtering compaction operations + * + * @param writeConfig Hoodie Write Config + * @param operations Compaction Operations to be ordered and filtered + * @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan + * @return Compaction plan to be scheduled. + */ + public HoodieCompactionPlan generateCompactionPlan(HoodieWriteConfig writeConfig, + List operations, List pendingCompactionPlans) { + // Strategy implementation can overload this method to set specific compactor-id + return HoodieCompactionPlan.newBuilder().setCompactorId(HoodieCompactionConfig.DEFAULT_COMPACTOR_ID) + .setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans)) + .build(); } /** - * Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to - * order and filter out compactions + * Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to order and filter out + * compactions * - * @param writeConfig - HoodieWriteConfig - config for this compaction is passed in - * @param operations - list of compactions collected + * @param writeConfig config for this compaction is passed in + * @param operations list of compactions collected + * @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan * @return list of compactions to perform in this run */ - public List orderAndFilter(HoodieWriteConfig writeConfig, - List operations) { + protected List orderAndFilter(HoodieWriteConfig writeConfig, + List operations, + List pendingCompactionPlans) { return operations; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java index 714dad54e6031..3360feb783c56 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java @@ -18,9 +18,10 @@ package com.uber.hoodie.io.compact.strategy; import com.google.common.annotations.VisibleForTesting; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.io.compact.CompactionOperation; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Comparator; @@ -58,12 +59,12 @@ public Comparator getComparator() { } @Override - public List orderAndFilter(HoodieWriteConfig writeConfig, - List operations) { + public List orderAndFilter(HoodieWriteConfig writeConfig, + List operations, List pendingCompactionPlans) { // Iterate through the operations and accept operations as long as we are within the configured target partitions // limit - List filteredList = operations.stream() - .collect(Collectors.groupingBy(CompactionOperation::getPartitionPath)).entrySet().stream() + List filteredList = operations.stream() + .collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream() .sorted(Map.Entry.comparingByKey(comparator)).limit(writeConfig.getTargetPartitionsPerDayBasedCompaction()) .flatMap(e -> e.getValue().stream()) .collect(Collectors.toList()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java index 39f66b2fe528c..8dd36695ab847 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -16,10 +16,11 @@ package com.uber.hoodie.io.compact.strategy; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.compact.CompactionOperation; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -34,37 +35,37 @@ * @see CompactionStrategy */ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements - Comparator { + Comparator { private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; @Override - public Map captureMetrics(HoodieWriteConfig config, Optional dataFile, String - partitionPath, - List logFiles) { + public Map captureMetrics(HoodieWriteConfig config, Optional dataFile, + String partitionPath, List logFiles) { + Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); - Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); // Total size of all the log files Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) .orElse(0L); // save the metrics needed during the order - metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize); + metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); return metrics; } @Override - public List orderAndFilter(HoodieWriteConfig writeConfig, - List operations) { + public List orderAndFilter(HoodieWriteConfig writeConfig, + List operations, List pendingCompactionPlans) { // Order the operations based on the reverse size of the logs and limit them by the IO return super - .orderAndFilter(writeConfig, operations.stream().sorted(this).collect(Collectors.toList())); + .orderAndFilter(writeConfig, + operations.stream().sorted(this).collect(Collectors.toList()), pendingCompactionPlans); } @Override - public int compare(CompactionOperation op1, CompactionOperation op2) { - Long totalLogSize1 = (Long) op1.getMetrics().get(TOTAL_LOG_FILE_SIZE); - Long totalLogSize2 = (Long) op2.getMetrics().get(TOTAL_LOG_FILE_SIZE); + public int compare(HoodieCompactionOperation op1, HoodieCompactionOperation op2) { + Long totalLogSize1 = op1.getMetrics().get(TOTAL_LOG_FILE_SIZE).longValue(); + Long totalLogSize2 = op2.getMetrics().get(TOTAL_LOG_FILE_SIZE).longValue(); // Reverse the comparison order - so that larger log file size is compacted first return totalLogSize2.compareTo(totalLogSize1); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java index 3f8297f280aa2..f8edc2ce12c74 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/UnBoundedCompactionStrategy.java @@ -16,8 +16,9 @@ package com.uber.hoodie.io.compact.strategy; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.compact.CompactionOperation; import java.util.List; /** @@ -30,8 +31,8 @@ public class UnBoundedCompactionStrategy extends CompactionStrategy { @Override - public List orderAndFilter(HoodieWriteConfig config, - List operations) { + public List orderAndFilter(HoodieWriteConfig config, + List operations, List pendingCompactionWorkloads) { return operations; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 25ba78421f45d..b05ba602f0e50 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -19,6 +19,7 @@ import com.google.common.collect.Maps; import com.google.common.hash.Hashing; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.HoodieCommitMetadata; @@ -161,11 +162,22 @@ public boolean isWorkloadProfileNeeded() { return true; } + @Override + public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String commitTime) { + throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); + } + @Override public JavaRDD compact(JavaSparkContext jsc, String commitTime) { throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); } + @Override + public JavaRDD compact(JavaSparkContext jsc, String compactionInstantTime, + HoodieCompactionPlan compactionPlan) { + throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); + } + public Iterator> handleUpdate(String commitTime, String fileLoc, Iterator> recordItr) throws IOException { // these are updates diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 74e83e92c0c1b..ef2ed7f0e4b1a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -19,6 +19,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieCommitMetadata; @@ -126,7 +127,7 @@ public Iterator> handleInsert(String commitTime, } @Override - public JavaRDD compact(JavaSparkContext jsc, String compactionCommitTime) { + public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) { logger.info("Checking if compaction needs to be run on " + config.getBasePath()); Optional lastCompaction = getActiveTimeline().getCommitTimeline() .filterCompletedInstants().lastInstant(); @@ -141,10 +142,20 @@ public JavaRDD compact(JavaSparkContext jsc, String compactionCommi logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction + " delta commits was found since last compaction " + deltaCommitsSinceTs + ". Waiting for " + config.getInlineCompactDeltaCommitMax()); - return jsc.emptyRDD(); + return new HoodieCompactionPlan(); } logger.info("Compacting merge on read table " + config.getBasePath()); + HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); + try { + return compactor.generateCompactionPlan(jsc, this, config, instantTime); + } catch (IOException e) { + throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); + } + } + + @Override + public JavaRDD compact(JavaSparkContext jsc, String compactionCommitTime) { HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); try { return compactor.compact(jsc, config, this, compactionCommitTime); @@ -153,6 +164,17 @@ public JavaRDD compact(JavaSparkContext jsc, String compactionCommi } } + @Override + public JavaRDD compact(JavaSparkContext jsc, String compactionInstantTime, + HoodieCompactionPlan compactionPlan) { + HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); + try { + return compactor.compact(jsc, compactionPlan, this, config, compactionInstantTime); + } catch (IOException e) { + throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e); + } + } + @Override public List rollback(JavaSparkContext jsc, List commits) throws IOException { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 76231331869c4..3e83ca1a3fa36 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -17,6 +17,7 @@ package com.uber.hoodie.table; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.avro.model.HoodieSavepointMetadata; import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieRollbackStat; @@ -211,12 +212,32 @@ public abstract Iterator> handleUpsertPartition(String commitT public abstract Iterator> handleInsertPartition(String commitTime, Integer partition, Iterator> recordIterator, Partitioner partitioner); + /** + * Schedule compaction for the instant time + * @param jsc Spark Context + * @param instantTime Instant Time for scheduling compaction + * @return + */ + public abstract HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime); + /** * Run Compaction on the table. Compaction arranges the data so that it is optimized for data * access + * @deprecated Will be replaced with newer APIs */ + @Deprecated public abstract JavaRDD compact(JavaSparkContext jsc, String commitTime); + /** + * Run Compaction on the table. Compaction arranges the data so that it is optimized for data access + * + * @param jsc Spark Context + * @param compactionInstantTime Instant Time + * @param compactionPlan Compaction Plan + */ + public abstract JavaRDD compact(JavaSparkContext jsc, String compactionInstantTime, + HoodieCompactionPlan compactionPlan); + /** * Clean partition paths according to cleaning policy and returns the number of files cleaned. */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index cb53fdd8dcecb..44a3e6537f549 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -20,19 +20,24 @@ import static org.junit.Assert.assertTrue; import com.beust.jcommander.internal.Lists; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.compact.CompactionOperation; import com.uber.hoodie.io.compact.strategy.BoundedIOCompactionStrategy; import com.uber.hoodie.io.compact.strategy.DayBasedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; import org.junit.Assert; import org.junit.Test; @@ -51,8 +56,8 @@ public void testUnBounded() { UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy(); HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build(); - List operations = createCompactionOperations(writeConfig, sizesMap); - List returned = strategy.orderAndFilter(writeConfig, operations); + List operations = createCompactionOperations(writeConfig, sizesMap); + List returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>()); assertEquals("UnBounded should not re-order or filter", operations, returned); } @@ -67,14 +72,14 @@ public void testBoundedIOSimple() { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) .build(); - List operations = createCompactionOperations(writeConfig, sizesMap); - List returned = strategy.orderAndFilter(writeConfig, operations); + List operations = createCompactionOperations(writeConfig, sizesMap); + List returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>()); assertTrue("BoundedIOCompaction should have resulted in fewer compactions", returned.size() < operations.size()); assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", 2, returned.size()); // Total size of all the log files Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) - .map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L); + .map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L); assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 610, (long) returnedSize); } @@ -90,15 +95,15 @@ public void testLogFileSizeCompactionSimple() { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) .build(); - List operations = createCompactionOperations(writeConfig, sizesMap); - List returned = strategy.orderAndFilter(writeConfig, operations); + List operations = createCompactionOperations(writeConfig, sizesMap); + List returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>()); assertTrue("LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions", returned.size() < operations.size()); assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", 1, returned.size()); // Total size of all the log files Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) - .map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L); + .map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L); assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 1204, (long) returnedSize); } @@ -110,25 +115,20 @@ public void testPartitionAwareCompactionSimple() { sizesMap.put(110 * MB, Lists.newArrayList()); sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); - sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); - sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); - sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); + + Map keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, partitionPaths[2]) + .put(110 * MB, partitionPaths[2]) + .put(100 * MB, partitionPaths[1]) + .put(90 * MB, partitionPaths[0]) + .build(); DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy(); HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) .withTargetPartitionsPerDayBasedCompaction(1) .build()).build(); - - List operations = Lists.newArrayList(sizesMap.size()); - int partitionPathIndex = 0; - for (Map.Entry> entry : sizesMap.entrySet()) { - operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(entry.getKey())), - partitionPaths[(partitionPathIndex % (partitionPaths.length - 1))], - entry.getValue().stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), writeConfig)); - partitionPathIndex++; - } - List returned = strategy.orderAndFilter(writeConfig, operations); + List operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap); + List returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>()); assertTrue("DayBasedCompactionStrategy should have resulted in fewer compactions", returned.size() < operations.size()); @@ -141,13 +141,28 @@ public void testPartitionAwareCompactionSimple() { assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", comparision >= 0); } - private List createCompactionOperations(HoodieWriteConfig config, + private List createCompactionOperations(HoodieWriteConfig config, Map> sizesMap) { - List operations = Lists.newArrayList(sizesMap.size()); + Map keyToParitionMap = sizesMap.entrySet().stream().map(e -> + Pair.of(e.getKey(), partitionPaths[new Random().nextInt(partitionPaths.length - 1)])) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + return createCompactionOperations(config, sizesMap, keyToParitionMap); + } + + private List createCompactionOperations(HoodieWriteConfig config, + Map> sizesMap, Map keyToPartitionMap) { + List operations = Lists.newArrayList(sizesMap.size()); + sizesMap.forEach((k, v) -> { - operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(k)), - partitionPaths[new Random().nextInt(partitionPaths.length - 1)], - v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config)); + HoodieDataFile df = TestHoodieDataFile.newDataFile(k); + String partitionPath = keyToPartitionMap.get(k); + List logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()); + operations.add(new HoodieCompactionOperation(df.getCommitTime(), + logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), + df.getPath(), + df.getFileId(), + partitionPath, + config.getCompactionStrategy().captureMetrics(config, Optional.of(df), partitionPath, logFiles))); }); return operations; } diff --git a/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc b/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc new file mode 100644 index 0000000000000..8400fa53aac47 --- /dev/null +++ b/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc @@ -0,0 +1,66 @@ +{ + "namespace":"com.uber.hoodie.avro.model", + "type":"record", + "name":"HoodieCompactionPlan", + "fields":[ + { + "name":"compactorId", + "type":["null","string"] + }, + { + "name":"operations", + "type":["null", { + "type":"array", + "items":{ + "name":"HoodieCompactionOperation", + "type":"record", + "fields":[ + { + "name":"baseInstantTime", + "type":["null","string"] + }, + { + "name":"deltaFilePaths", + "type":["null", { + "type":"array", + "items":"string" + }], + "default": null + }, + { + "name":"dataFilePath", + "type":["null","string"], + "default": null + }, + { + "name":"fileId", + "type":["null","string"] + }, + { + "name":"partitionPath", + "type":["null","string"], + "default": null + }, + { + "name":"metrics", + "type":["null", { + "type":"map", + "values":"double" + }], + "default": null + } + ] + } + }], + "default": null + }, + { + "name":"extraMetadata", + "type":["null", { + "type":"map", + "values":"string" + }], + "default": null + } + ] +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java similarity index 65% rename from hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java index 3d5ffc40cb38f..9f23c4a12aff8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java @@ -14,14 +14,13 @@ * limitations under the License. */ -package com.uber.hoodie.io.compact; +package com.uber.hoodie.common.model; -import com.uber.hoodie.common.model.HoodieDataFile; -import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.common.util.FSUtils; -import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -31,17 +30,16 @@ * Encapsulates all the needed information about a compaction and make a decision whether this * compaction is effective or not * - * @see CompactionStrategy */ public class CompactionOperation implements Serializable { + private String baseInstantTime; private Optional dataFileCommitTime; - private Optional dataFileSize; private List deltaFilePaths; private Optional dataFilePath; private String fileId; private String partitionPath; - private Map metrics; + private Map metrics; //Only for serialization/de-serialization @Deprecated @@ -49,32 +47,32 @@ public CompactionOperation() { } public CompactionOperation(Optional dataFile, String partitionPath, - List logFiles, HoodieWriteConfig writeConfig) { + List logFiles, Map metrics) { if (dataFile.isPresent()) { + this.baseInstantTime = dataFile.get().getCommitTime(); this.dataFilePath = Optional.of(dataFile.get().getPath()); this.fileId = dataFile.get().getFileId(); this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime()); - this.dataFileSize = Optional.of(dataFile.get().getFileSize()); } else { assert logFiles.size() > 0; this.dataFilePath = Optional.empty(); + this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath()); this.fileId = FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()); this.dataFileCommitTime = Optional.empty(); - this.dataFileSize = Optional.empty(); } + this.partitionPath = partitionPath; this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) .collect(Collectors.toList()); - this.metrics = writeConfig.getCompactionStrategy() - .captureMetrics(writeConfig, dataFile, partitionPath, logFiles); + this.metrics = metrics; } - public Optional getDataFileCommitTime() { - return dataFileCommitTime; + public String getBaseInstantTime() { + return baseInstantTime; } - public Optional getDataFileSize() { - return dataFileSize; + public Optional getDataFileCommitTime() { + return dataFileCommitTime; } public List getDeltaFilePaths() { @@ -93,7 +91,23 @@ public String getPartitionPath() { return partitionPath; } - public Map getMetrics() { + public Map getMetrics() { return metrics; } + + /** + * Convert Avro generated Compaction operation to POJO for Spark RDD operation + * @param operation Hoodie Compaction Operation + * @return + */ + public static CompactionOperation convertFromAvroRecordInstance(HoodieCompactionOperation operation) { + CompactionOperation op = new CompactionOperation(); + op.baseInstantTime = operation.getBaseInstantTime(); + op.dataFilePath = Optional.ofNullable(operation.getDataFilePath()); + op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths()); + op.fileId = operation.getFileId(); + op.metrics = new HashMap<>(operation.getMetrics()); + op.partitionPath = operation.getPartitionPath(); + return op; + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java index 633960d0a3a2a..a28857ff3becf 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import com.uber.hoodie.avro.model.HoodieCleanMetadata; import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.avro.model.HoodieRollbackMetadata; import com.uber.hoodie.avro.model.HoodieRollbackPartitionMetadata; import com.uber.hoodie.avro.model.HoodieSavepointMetadata; @@ -142,6 +143,10 @@ public static HoodieSavepointMetadata convertSavepointMetadata(String user, Stri partitionMetadataBuilder.build()); } + public static Optional serializeCompactionWorkload(HoodieCompactionPlan compactionWorkload) + throws IOException { + return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class); + } public static Optional serializeCleanMetadata(HoodieCleanMetadata metadata) throws IOException { @@ -169,6 +174,11 @@ public static Optional serializeAvroMetad return Optional.of(baos.toByteArray()); } + public static HoodieCompactionPlan deserializeHoodieCompactionPlan(byte[] bytes) + throws IOException { + return deserializeAvroMetadata(bytes, HoodieCompactionPlan.class); + } + public static HoodieCleanMetadata deserializeHoodieCleanMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieCleanMetadata.class); @@ -188,5 +198,4 @@ public static T deserializeAvroMetadata(byte[] by .checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz); return fileReader.next(); } - } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java new file mode 100644 index 0000000000000..45091da85b3c9 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -0,0 +1,100 @@ +/* + * 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.common.util; + +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.model.CompactionOperation; +import com.uber.hoodie.common.model.FileSlice; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; +import javafx.util.Pair; + +/** + * Helper class to generate compaction workload from FileGroup/FileSlice abstraction + */ +public class CompactionUtils { + + /** + * Generate compaction operation from file-slice + * + * @param partitionPath Partition path + * @param fileSlice File Slice + * @param metricsCaptureFunction Metrics Capture function + * @return Compaction Operation + */ + public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice, + Optional, Map>> metricsCaptureFunction) { + HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder(); + builder.setPartitionPath(partitionPath); + builder.setFileId(fileSlice.getFileId()); + builder.setBaseInstantTime(fileSlice.getBaseInstantTime()); + builder.setDeltaFilePaths(fileSlice.getLogFiles().map(lf -> lf.getPath().toString()).collect(Collectors.toList())); + if (fileSlice.getDataFile().isPresent()) { + builder.setDataFilePath(fileSlice.getDataFile().get().getPath()); + } + + if (metricsCaptureFunction.isPresent()) { + builder.setMetrics(metricsCaptureFunction.get().apply(new Pair(partitionPath, fileSlice))); + } + return builder.build(); + } + + /** + * Generate compaction workload from file-slices + * + * @param compactorId Compactor Id to set + * @param partitionFileSlicePairs list of partition file-slice pairs + * @param extraMetadata Extra Metadata + * @param metricsCaptureFunction Metrics Capture function + */ + public static HoodieCompactionPlan buildFromFileSlices(String compactorId, + List> partitionFileSlicePairs, + Optional> extraMetadata, + Optional, Map>> metricsCaptureFunction) { + HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder(); + builder.setCompactorId(compactorId); + extraMetadata.ifPresent(m -> builder.setExtraMetadata(m)); + builder.setOperations(partitionFileSlicePairs.stream().map(pfPair -> + buildFromFileSlice(pfPair.getKey(), pfPair.getValue(), metricsCaptureFunction)).collect(Collectors.toList())); + return builder.build(); + } + + /** + * Build Avro generated Compaction operation payload from compaction operation POJO for serialization + */ + public static HoodieCompactionOperation buildHoodieCompactionOperation(CompactionOperation op) { + return HoodieCompactionOperation.newBuilder().setFileId(op.getFileId()) + .setBaseInstantTime(op.getBaseInstantTime()) + .setPartitionPath(op.getPartitionPath()) + .setDataFilePath(op.getDataFilePath().isPresent() ? op.getDataFilePath().get() : null) + .setDeltaFilePaths(op.getDeltaFilePaths()) + .setMetrics(op.getMetrics()).build(); + } + + /** + * Build Compaction operation payload from Avro version for using in Spark executors + * + * @param hc HoodieCompactionOperation + */ + public static CompactionOperation buildCompactionOperation(HoodieCompactionOperation hc) { + return CompactionOperation.convertFromAvroRecordInstance(hc); + } +} From 542eb72dbd5c4e728eca89bf6403a744a0430d64 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 24 May 2018 11:19:40 -0700 Subject: [PATCH 069/374] Track fileIds with pending compaction in FileSystemView to provide correct API semantics --- .../hoodie/config/HoodieCompactionConfig.java | 1 - .../compact/HoodieRealtimeTableCompactor.java | 6 +- .../compact/strategy/CompactionStrategy.java | 3 +- .../main/avro/HoodieCompactionOperation.avsc | 6 +- .../table/view/HoodieTableFileSystemView.java | 29 +- .../uber/hoodie/common/util/AvroUtils.java | 2 +- .../hoodie/common/util/CompactionUtils.java | 76 +++- .../view/HoodieTableFileSystemViewTest.java | 20 +- .../common/util/TestCompactionUtils.java | 345 ++++++++++++++++++ 9 files changed, 448 insertions(+), 40 deletions(-) create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index 97c5e5c68b3fe..d1584aac02649 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -108,7 +108,6 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { + ".partitions"; // 500GB of target IO per compaction (both read and write) public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10); - public static final String DEFAULT_COMPACTOR_ID = "default"; private HoodieCompactionConfig(Properties props) { super(props); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index b314e8eb34914..90a24b6d98017 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -41,7 +41,6 @@ import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -49,6 +48,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.avro.Schema; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -95,7 +95,7 @@ public JavaRDD compact(JavaSparkContext jsc, HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); List operations = compactionPlan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); - log.info("Compactor " + compactionPlan.getCompactorId() + " running, Compacting " + operations + " files"); + log.info("Compactor compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) .map(s -> compact(table, metaClient, config, s, compactionCommitTime)) .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); @@ -217,7 +217,7 @@ public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, // TODO: In subsequent PRs, pending Compaction plans will be wired in. Strategy can look at pending compaction // plans to schedule next compaction plan HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations, - new ArrayList<>()); + CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); if (compactionPlan.getOperations().isEmpty()) { log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index 2b5070dbaf8f2..fa5db25278b96 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -22,7 +22,6 @@ import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.util.FSUtils; -import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import java.io.Serializable; import java.util.List; @@ -92,7 +91,7 @@ public Map captureMetrics(HoodieWriteConfig writeConfig, Optiona public HoodieCompactionPlan generateCompactionPlan(HoodieWriteConfig writeConfig, List operations, List pendingCompactionPlans) { // Strategy implementation can overload this method to set specific compactor-id - return HoodieCompactionPlan.newBuilder().setCompactorId(HoodieCompactionConfig.DEFAULT_COMPACTOR_ID) + return HoodieCompactionPlan.newBuilder() .setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans)) .build(); } diff --git a/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc b/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc index 8400fa53aac47..450b898985e3b 100644 --- a/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc +++ b/hoodie-common/src/main/avro/HoodieCompactionOperation.avsc @@ -3,11 +3,7 @@ "type":"record", "name":"HoodieCompactionPlan", "fields":[ - { - "name":"compactorId", - "type":["null","string"] - }, - { + { "name":"operations", "type":["null", { "type":"array", diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index fe6e326a09c0d..e1ef48a35df30 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -16,6 +16,7 @@ package com.uber.hoodie.common.table.view; +import com.google.common.collect.ImmutableMap; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieFileGroup; @@ -23,6 +24,8 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; @@ -41,6 +44,8 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** * Common abstract implementation for multiple TableFileSystemView Implementations. 2 possible @@ -55,6 +60,8 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.ReadOptimizedView, TableFileSystemView.RealtimeView, Serializable { + private static Logger log = LogManager.getLogger(HoodieTableFileSystemView.class); + protected HoodieTableMetaClient metaClient; // This is the commits that will be visible for all views extending this view protected HoodieTimeline visibleActiveTimeline; @@ -78,10 +85,15 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, this.visibleActiveTimeline = visibleActiveTimeline; this.fileGroupMap = new HashMap<>(); this.partitionToFileGroupsMap = new HashMap<>(); - //TODO: vb Will be implemented in next PR - this.fileIdToPendingCompactionInstantTime = new HashMap<>(); - } + // Build fileId to Pending Compaction Instants + List pendingCompactionInstants = + metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); + this.fileIdToPendingCompactionInstantTime = ImmutableMap.copyOf( + CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream().map(entry -> { + return Pair.of(entry.getKey(), entry.getValue().getKey()); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue))); + } /** * Create a file system view, as of the given timeline, with the provided file statuses. @@ -404,15 +416,4 @@ public Stream getAllFileGroups(String partitionPathStr) { "Failed to list data files in partition " + partitionPathStr, e); } } - - /** - * Used by tests to add pending compaction entries TODO: This method is temporary and should go away in subsequent - * Async Compaction PR - * - * @param fileId File Id - * @param compactionInstantTime Compaction Instant Time - */ - protected void addPendingCompactionFileId(String fileId, String compactionInstantTime) { - fileIdToPendingCompactionInstantTime.put(fileId, compactionInstantTime); - } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java index a28857ff3becf..1cde27f318d12 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java @@ -174,7 +174,7 @@ public static Optional serializeAvroMetad return Optional.of(baos.toByteArray()); } - public static HoodieCompactionPlan deserializeHoodieCompactionPlan(byte[] bytes) + public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieCompactionPlan.class); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java index 45091da85b3c9..cadc9d7bd977c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -20,15 +20,21 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; -import javafx.util.Pair; +import java.util.stream.Stream; +import org.apache.commons.lang3.tuple.Pair; /** - * Helper class to generate compaction workload from FileGroup/FileSlice abstraction + * Helper class to generate compaction plan from FileGroup/FileSlice abstraction */ public class CompactionUtils { @@ -41,7 +47,7 @@ public class CompactionUtils { * @return Compaction Operation */ public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice, - Optional, Map>> metricsCaptureFunction) { + Optional, Map>> metricsCaptureFunction) { HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder(); builder.setPartitionPath(partitionPath); builder.setFileId(fileSlice.getFileId()); @@ -52,25 +58,23 @@ public static HoodieCompactionOperation buildFromFileSlice(String partitionPath } if (metricsCaptureFunction.isPresent()) { - builder.setMetrics(metricsCaptureFunction.get().apply(new Pair(partitionPath, fileSlice))); + builder.setMetrics(metricsCaptureFunction.get().apply(Pair.of(partitionPath, fileSlice))); } return builder.build(); } /** - * Generate compaction workload from file-slices + * Generate compaction plan from file-slices * - * @param compactorId Compactor Id to set * @param partitionFileSlicePairs list of partition file-slice pairs * @param extraMetadata Extra Metadata * @param metricsCaptureFunction Metrics Capture function */ - public static HoodieCompactionPlan buildFromFileSlices(String compactorId, + public static HoodieCompactionPlan buildFromFileSlices( List> partitionFileSlicePairs, Optional> extraMetadata, - Optional, Map>> metricsCaptureFunction) { + Optional, Map>> metricsCaptureFunction) { HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder(); - builder.setCompactorId(compactorId); extraMetadata.ifPresent(m -> builder.setExtraMetadata(m)); builder.setOperations(partitionFileSlicePairs.stream().map(pfPair -> buildFromFileSlice(pfPair.getKey(), pfPair.getValue(), metricsCaptureFunction)).collect(Collectors.toList())); @@ -97,4 +101,58 @@ public static HoodieCompactionOperation buildHoodieCompactionOperation(Compactio public static CompactionOperation buildCompactionOperation(HoodieCompactionOperation hc) { return CompactionOperation.convertFromAvroRecordInstance(hc); } + + /** + * Get all pending compaction plans along with their instants + * + * @param metaClient Hoodie Meta Client + */ + public static List> getAllPendingCompactionPlans( + HoodieTableMetaClient metaClient) { + List pendingCompactionInstants = + metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); + return pendingCompactionInstants.stream().map(instant -> { + try { + HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( + metaClient.getActiveTimeline().getInstantDetails(instant).get()); + return Pair.of(instant, compactionPlan); + } catch (IOException e) { + throw new HoodieException(e); + } + }).collect(Collectors.toList()); + } + + /** + * Get all file-ids with pending Compaction operations and their target compaction instant time + * + * @param metaClient Hoodie Table Meta Client + */ + public static Map> getAllPendingCompactionOperations( + HoodieTableMetaClient metaClient) { + List> pendingCompactionPlanWithInstants = + getAllPendingCompactionPlans(metaClient); + + Map> fileIdToPendingCompactionWithInstantMap = new HashMap<>(); + pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> { + HoodieInstant instant = instantPlanPair.getKey(); + HoodieCompactionPlan compactionPlan = instantPlanPair.getValue(); + List ops = compactionPlan.getOperations(); + if (null != ops) { + return ops.stream().map(op -> { + return Pair.of(op.getFileId(), Pair.of(instant.getTimestamp(), op)); + }); + } else { + return Stream.empty(); + } + }).forEach(pair -> { + // Defensive check to ensure a single-fileId does not have more than one pending compaction + if (fileIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) { + String msg = "Hoodie File Id (" + pair.getKey() + ") has more thant 1 pending compactions. Instants: " + + pair.getValue() + ", " + fileIdToPendingCompactionWithInstantMap.get(pair.getKey()); + throw new IllegalStateException(msg); + } + fileIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue()); + }); + return fileIdToPendingCompactionWithInstantMap; + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 757a3caae274c..713b6b80275e9 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieFileGroup; @@ -33,9 +34,12 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -44,6 +48,7 @@ import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.junit.Before; @@ -234,19 +239,26 @@ private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFi commitTimeline.saveAsComplete(deltaInstant2, Optional.empty()); commitTimeline.saveAsComplete(deltaInstant3, Optional.empty()); - // Fake delta-ingestion after compaction-requested + refreshFsView(null); + List fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); String compactionRequestedTime = "4"; String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId); + List> partitionFileSlicesPairs = new ArrayList<>(); + partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0))); + HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, + Optional.empty(), Optional.empty()); HoodieInstant compactionInstant = null; if (isCompactionInFlight) { // Create a Data-file but this should be skipped by view new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile(); compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); - commitTimeline.saveToInflight(compactionInstant, Optional.empty()); + commitTimeline.saveToInflight(compactionInstant, AvroUtils.serializeCompactionWorkload(compactionPlan)); } else { compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); - commitTimeline.saveToRequested(compactionInstant, Optional.empty()); + commitTimeline.saveToRequested(compactionInstant, AvroUtils.serializeCompactionWorkload(compactionPlan)); } + + // Fake delta-ingestion after compaction-requested String deltaInstantTime4 = "5"; String deltaInstantTime5 = "6"; List allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2, @@ -260,7 +272,6 @@ private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFi commitTimeline.saveAsComplete(deltaInstant4, Optional.empty()); commitTimeline.saveAsComplete(deltaInstant5, Optional.empty()); refreshFsView(null); - fsView.addPendingCompactionFileId(fileId, compactionRequestedTime); List dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList()); if (skipCreatingDataFile) { @@ -367,7 +378,6 @@ private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFi commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, inflightDeltaInstantTime), Optional.empty()); refreshFsView(null); - fsView.addPendingCompactionFileId(fileId, compactionRequestedTime); List allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList()); dataFiles = allRawFileSlices.stream().flatMap(slice -> { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java new file mode 100644 index 0000000000000..9faaca3e5d25b --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java @@ -0,0 +1,345 @@ +/* + * 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; + +import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS; +import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestCompactionUtils { + + private static final Map metrics = + new ImmutableMap.Builder() + .put("key1", 1.0) + .put("key2", 3.0).build(); + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + private HoodieTableMetaClient metaClient; + private String basePath; + private Function, Map> metricsCaptureFn = (partitionFileSlice) -> metrics; + + @Before + public void init() throws IOException { + metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath()); + basePath = metaClient.getBasePath(); + } + + @Test + public void testBuildFromFileSlice() { + // Empty File-Slice with no data and log files + FileSlice emptyFileSlice = new FileSlice("000", "empty1"); + HoodieCompactionOperation op = CompactionUtils.buildFromFileSlice( + DEFAULT_PARTITION_PATHS[0], emptyFileSlice, Optional.of(metricsCaptureFn)); + testFileSliceCompactionOpEquality(emptyFileSlice, op, DEFAULT_PARTITION_PATHS[0]); + + // File Slice with data-file but no log files + FileSlice noLogFileSlice = new FileSlice("000", "noLog1"); + noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); + op = CompactionUtils.buildFromFileSlice( + DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Optional.of(metricsCaptureFn)); + testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]); + + //File Slice with no data-file but log files present + FileSlice noDataFileSlice = new FileSlice("000", "noData1"); + noDataFileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); + noDataFileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 2)))); + op = CompactionUtils.buildFromFileSlice( + DEFAULT_PARTITION_PATHS[0], noDataFileSlice, Optional.of(metricsCaptureFn)); + testFileSliceCompactionOpEquality(noDataFileSlice, op, DEFAULT_PARTITION_PATHS[0]); + + //File Slice with data-file and log files present + FileSlice fileSlice = new FileSlice("000", "noData1"); + fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); + fileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); + fileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 2)))); + op = CompactionUtils.buildFromFileSlice( + DEFAULT_PARTITION_PATHS[0], fileSlice, Optional.of(metricsCaptureFn)); + testFileSliceCompactionOpEquality(fileSlice, op, DEFAULT_PARTITION_PATHS[0]); + } + + /** + * Generate input for compaction plan tests + */ + private Pair>, HoodieCompactionPlan> buildCompactionPlan() { + FileSlice emptyFileSlice = new FileSlice("000", "empty1"); + FileSlice fileSlice = new FileSlice("000", "noData1"); + fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); + fileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); + fileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 2)))); + FileSlice noLogFileSlice = new FileSlice("000", "noLog1"); + noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); + FileSlice noDataFileSlice = new FileSlice("000", "noData1"); + noDataFileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); + noDataFileSlice.addLogFile(new HoodieLogFile(new Path( + FSUtils.makeLogFileName("noData1", ".log", "000", 2)))); + List fileSliceList = Arrays.asList(emptyFileSlice, noDataFileSlice, fileSlice, noLogFileSlice); + List> input = fileSliceList.stream().map(f -> Pair.of(DEFAULT_PARTITION_PATHS[0], f)) + .collect(Collectors.toList()); + return Pair.of(input, CompactionUtils.buildFromFileSlices(input, Optional.empty(), Optional.of(metricsCaptureFn))); + } + + @Test + public void testBuildFromFileSlices() { + Pair>, HoodieCompactionPlan> inputAndPlan = buildCompactionPlan(); + testFileSlicesCompactionPlanEquality(inputAndPlan.getKey(), inputAndPlan.getValue()); + } + + @Test + public void testCompactionTransformation() { + // check HoodieCompactionOperation <=> CompactionOperation transformation function + Pair>, HoodieCompactionPlan> inputAndPlan = buildCompactionPlan(); + HoodieCompactionPlan plan = inputAndPlan.getRight(); + List originalOps = plan.getOperations(); + List regeneratedOps = + originalOps.stream().map(op -> { + // Convert to CompactionOperation + return CompactionUtils.buildCompactionOperation(op); + }).map(op2 -> { + // Convert back to HoodieCompactionOperation and check for equality + return CompactionUtils.buildHoodieCompactionOperation(op2); + }).collect(Collectors.toList()); + Assert.assertTrue("Transformation did get tested", originalOps.size() > 0); + Assert.assertEquals("All fields set correctly in transformations", originalOps, regeneratedOps); + } + + @Test(expected = IllegalStateException.class) + public void testGetAllPendingCompactionOperationsWithDupFileId() throws IOException { + // Case where there is duplicate fileIds in compaction requests + HoodieCompactionPlan plan1 = createCompactionPlan("000", 10); + HoodieCompactionPlan plan2 = createCompactionPlan("001", 10); + scheduleCompaction("000", plan1); + scheduleCompaction("001", plan2); + // schedule same plan again so that there will be duplicates + scheduleCompaction("003", plan1); + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + Map> res = + CompactionUtils.getAllPendingCompactionOperations(metaClient); + } + + @Test + public void testGetAllPendingCompactionOperations() throws IOException { + // Case where there are 4 compaction requests where 1 is empty. + testGetAllPendingCompactionOperations(false, 10, 10, 10, 0); + } + + @Test + public void testGetAllPendingInflightCompactionOperations() throws IOException { + // Case where there are 4 compaction requests where 1 is empty. All of them are marked inflight + testGetAllPendingCompactionOperations(true, 10, 10, 10, 0); + } + + @Test + public void testGetAllPendingCompactionOperationsForEmptyCompactions() throws IOException { + // Case where there are 4 compaction requests and all are empty. + testGetAllPendingCompactionOperations(false, 0, 0, 0, 0); + } + + private void testGetAllPendingCompactionOperations(boolean inflight, int numEntriesInPlan1, int numEntriesInPlan2, + int numEntriesInPlan3, int numEntriesInPlan4) throws IOException { + HoodieCompactionPlan plan1 = createCompactionPlan("000", numEntriesInPlan1); + HoodieCompactionPlan plan2 = createCompactionPlan("001", numEntriesInPlan2); + HoodieCompactionPlan plan3 = createCompactionPlan("002", numEntriesInPlan3); + HoodieCompactionPlan plan4 = createCompactionPlan("003", numEntriesInPlan4); + + if (inflight) { + scheduleInflightCompaction("000", plan1); + scheduleInflightCompaction("001", plan2); + scheduleInflightCompaction("002", plan3); + scheduleInflightCompaction("003", plan4); + } else { + scheduleCompaction("000", plan1); + scheduleCompaction("001", plan2); + scheduleCompaction("002", plan3); + scheduleCompaction("003", plan4); + } + + List expectedNumEntries = + Arrays.asList(numEntriesInPlan1, numEntriesInPlan2, numEntriesInPlan3, numEntriesInPlan4); + List plans = new ImmutableList.Builder() + .add(plan1, plan2, plan3, plan4).build(); + IntStream.range(0, 4).boxed().forEach(idx -> { + if (expectedNumEntries.get(idx) > 0) { + Assert.assertEquals("check if plan " + idx + " has exp entries", + expectedNumEntries.get(idx).longValue(), plans.get(idx).getOperations().size()); + } else { + Assert.assertNull("Plan " + idx + " has null ops", plans.get(idx).getOperations()); + } + }); + + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + Map> pendingCompactionMap = + CompactionUtils.getAllPendingCompactionOperations(metaClient); + + Map> expPendingCompactionMap = + generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4)); + + // Ensure all the + Assert.assertEquals(expPendingCompactionMap, pendingCompactionMap); + } + + private Map> generateExpectedCompactionOperations( + List plans) { + return plans.stream() + .flatMap(plan -> { + if (plan.getOperations() != null) { + return plan.getOperations().stream().map(op -> Pair.of(op.getFileId(), + Pair.of(op.getBaseInstantTime(), op))); + } + return Stream.empty(); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } + + private void scheduleCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { + metaClient.getActiveTimeline().saveToRequested(new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), + AvroUtils.serializeCompactionWorkload(compactionPlan)); + } + + private void scheduleInflightCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { + metaClient.getActiveTimeline().saveToInflight(new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, instantTime), + AvroUtils.serializeCompactionWorkload(compactionPlan)); + } + + private HoodieCompactionPlan createCompactionPlan(String instantId, int numFileIds) { + List ops = IntStream.range(0, numFileIds).boxed().map(idx -> { + try { + String fileId = + HoodieTestUtils.createNewDataFile(basePath, DEFAULT_PARTITION_PATHS[0], instantId); + HoodieTestUtils.createNewLogFile(metaClient.getFs(), basePath, DEFAULT_PARTITION_PATHS[0], + instantId, fileId, Optional.of(1)); + HoodieTestUtils.createNewLogFile(metaClient.getFs(), basePath, DEFAULT_PARTITION_PATHS[0], + instantId, fileId, Optional.of(2)); + FileSlice slice = new FileSlice(instantId, fileId); + slice.setDataFile(new TestHoodieDataFile(HoodieTestUtils.createDataFile(basePath, DEFAULT_PARTITION_PATHS[0], + instantId, fileId))); + String logFilePath1 = HoodieTestUtils.getLogFilePath(basePath, DEFAULT_PARTITION_PATHS[0], instantId, fileId, + Optional.of(1)); + String logFilePath2 = HoodieTestUtils.getLogFilePath(basePath, DEFAULT_PARTITION_PATHS[0], instantId, fileId, + Optional.of(2)); + slice.addLogFile(new HoodieLogFile(new Path(logFilePath1))); + slice.addLogFile(new HoodieLogFile(new Path(logFilePath2))); + return CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Optional.empty()); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }).collect(Collectors.toList()); + return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>()); + } + + /** + * Validates if generated compaction plan matches with input file-slices + * + * @param input File Slices with partition-path + * @param plan Compaction Plan + */ + private void testFileSlicesCompactionPlanEquality(List> input, + HoodieCompactionPlan plan) { + Assert.assertEquals("All file-slices present", input.size(), plan.getOperations().size()); + IntStream.range(0, input.size()).boxed().forEach(idx -> + testFileSliceCompactionOpEquality(input.get(idx).getValue(), plan.getOperations().get(idx), + input.get(idx).getKey())); + } + + /** + * Validates if generated compaction operation matches with input file slice and partition path + * + * @param slice File Slice + * @param op HoodieCompactionOperation + * @param expPartitionPath Partition path + */ + private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompactionOperation op, + String expPartitionPath) { + Assert.assertEquals("Partition path is correct", expPartitionPath, op.getPartitionPath()); + Assert.assertEquals("Same base-instant", slice.getBaseInstantTime(), op.getBaseInstantTime()); + Assert.assertEquals("Same file-id", slice.getFileId(), op.getFileId()); + if (slice.getDataFile().isPresent()) { + Assert.assertEquals("Same data-file", slice.getDataFile().get().getPath(), op.getDataFilePath()); + } + List paths = slice.getLogFiles().map(l -> l.getPath().toString()).collect(Collectors.toList()); + IntStream.range(0, paths.size()).boxed().forEach(idx -> { + Assert.assertEquals("Log File Index " + idx, paths.get(idx), op.getDeltaFilePaths().get(idx)); + }); + Assert.assertEquals("Metrics set", metrics, op.getMetrics()); + } + + + private static class TestHoodieDataFile extends HoodieDataFile { + + private final String path; + + public TestHoodieDataFile(String path) { + super(null); + this.path = path; + } + + @Override + public String getPath() { + return path; + } + + @Override + public String getFileId() { + return UUID.randomUUID().toString(); + } + + @Override + public String getCommitTime() { + return "100"; + } + + @Override + public long getFileSize() { + return 0; + } + } +} From ac583c67dad1e47b1cc6914abfdba5e772ced2c4 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 31 May 2018 14:11:43 -0700 Subject: [PATCH 070/374] Ensure Compaction workload is stored in write-once meta-data files separate from timeline files. This avoids concurrency issues when compactor(s) and ingestor are running in parallel. In the Next PR -> Safety concern regarding Cleaner retaining all meta-data and file-slices for pending compactions will be addressed --- .../HoodieCompactionInstantWithPlan.java | 42 ++++++ .../common/HoodieTestDataGenerator.java | 18 +++ .../common/table/HoodieTableMetaClient.java | 45 +++++++ .../table/timeline/HoodieActiveTimeline.java | 121 ++++++++++++------ .../common/table/timeline/HoodieInstant.java | 4 + .../uber/hoodie/common/util/AvroUtils.java | 2 +- .../hoodie/common/util/CompactionUtils.java | 4 +- .../view/HoodieTableFileSystemViewTest.java | 6 +- .../common/util/TestCompactionUtils.java | 12 +- 9 files changed, 207 insertions(+), 47 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java new file mode 100644 index 0000000000000..ac0b5b331e61a --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java @@ -0,0 +1,42 @@ +/* + * 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.io.compact; + +import com.uber.hoodie.avro.model.HoodieCompactionPlan; + +/** + * Contains Hoodie Compaction instant along with workload + */ +public class HoodieCompactionInstantWithPlan { + + private final String compactionInstantTime; + private final HoodieCompactionPlan compactionPlan; + + public HoodieCompactionInstantWithPlan(String compactionInstantTime, + HoodieCompactionPlan compactionPlan) { + this.compactionInstantTime = compactionInstantTime; + this.compactionPlan = compactionPlan; + } + + public String getCompactionInstantTime() { + return compactionInstantTime; + } + + public HoodieCompactionPlan getCompactionPlan() { + return compactionPlan; + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index 83161aa2c3205..1d3f6a7ee1cfd 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -16,6 +16,7 @@ package com.uber.hoodie.common; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodiePartitionMetadata; @@ -23,6 +24,8 @@ import com.uber.hoodie.common.model.HoodieTestUtils; 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.AvroUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import java.io.IOException; @@ -135,6 +138,21 @@ public static void createCommitFile(String basePath, String commitTime, Configur } } + public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInstant instant, + Configuration configuration) throws IOException { + Path commitFile = new Path( + basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName()); + FileSystem fs = FSUtils.getFs(basePath, configuration); + FSDataOutputStream os = fs.create(commitFile, true); + HoodieCompactionPlan workload = new HoodieCompactionPlan(); + try { + // Write empty commit metadata + os.writeBytes(new String(AvroUtils.serializeCompactionPlan(workload).get(), StandardCharsets.UTF_8)); + } finally { + os.close(); + } + } + public static void createSavepointFile(String basePath, String commitTime) throws IOException { createSavepointFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 1a0edfc76de36..12649c5271dde 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -22,14 +22,20 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieException; import java.io.File; import java.io.IOException; import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; import java.util.Objects; import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -54,6 +60,7 @@ public class HoodieTableMetaClient implements Serializable { private static final transient Logger log = LogManager.getLogger(HoodieTableMetaClient.class); public static String METAFOLDER_NAME = ".hoodie"; public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; + public static String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux"; private String basePath; private transient FileSystem fs; @@ -135,6 +142,13 @@ public String getMetaPath() { return metaPath; } + /** + * @return Auxiliary Meta path + */ + public String getMetaAuxiliaryPath() { + return basePath + File.separator + AUXILIARYFOLDER_NAME; + } + /** * @return path where archived timeline is stored */ @@ -242,6 +256,13 @@ public static HoodieTableMetaClient initializePathAsHoodieDataset(Configuration if (!fs.exists(temporaryFolder)) { fs.mkdirs(temporaryFolder); } + + // Always create auxiliary folder which is needed to track compaction workloads (stats and any metadata in future) + final Path auxiliaryFolder = new Path(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); + if (!fs.exists(auxiliaryFolder)) { + fs.mkdirs(auxiliaryFolder); + } + HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); // We should not use fs.getConf as this might be different from the original configuration // used to create the fs in unit tests @@ -321,6 +342,30 @@ public String getCommitActionType() { } + /** + * Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects + * + * @param fs FileSystem + * @param metaPath Meta Path where hoodie instants are present + * @param includedExtensions Included hoodie extensions + * @return List of Hoodie Instants generated + * @throws IOException in case of failure + */ + public static List scanHoodieInstantsFromFileSystem( + FileSystem fs, Path metaPath, Set includedExtensions) throws IOException { + return Arrays.stream( + HoodieTableMetaClient + .scanFiles(fs, metaPath, path -> { + // Include only the meta files with extensions that needs to be included + String extension = FSUtils.getFileExtension(path.getName()); + return includedExtensions.contains(extension); + })).sorted(Comparator.comparing( + // Sort the meta-data by the instant time (first part of the file name) + fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName()))) + // create HoodieInstantMarkers from FileStatus, which extracts properties + .map(HoodieInstant::new).collect(Collectors.toList()); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index 20dea2805d6c1..f2c1f34328c81 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -20,18 +20,16 @@ import com.google.common.collect.Sets; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; -import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.Comparator; import java.util.Date; +import java.util.HashSet; import java.util.Optional; import java.util.Set; import java.util.function.Function; -import java.util.function.Predicate; -import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.time.FastDateFormat; @@ -54,6 +52,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat .getInstance("yyyyMMddHHmmss"); + public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( + new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, + INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, + CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION})); + private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class); private HoodieTableMetaClient metaClient; @@ -64,22 +67,12 @@ public static String createNewCommitTime() { return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); } - protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includedExtensions) { + protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set includedExtensions) { // Filter all the filter in the metapath and include only the extensions passed and // convert them into HoodieInstant try { - this.instants = - Arrays.stream( - HoodieTableMetaClient - .scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> { - // Include only the meta files with extensions that needs to be included - String extension = FSUtils.getFileExtension(path.getName()); - return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); - })).sorted(Comparator.comparing( - // Sort the meta-data by the instant time (first part of the file name) - fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName()))) - // create HoodieInstantMarkers from FileStatus, which extracts properties - .map(HoodieInstant::new).collect(Collectors.toList()); + this.instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(), + new Path(metaClient.getMetaPath()), includedExtensions); log.info("Loaded instants " + instants); } catch (IOException e) { throw new HoodieIOException("Failed to scan metadata", e); @@ -92,10 +85,7 @@ protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includ } public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { - this(metaClient, - new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, - INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, - CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION}); + this(metaClient, VALID_EXTENSIONS_IN_ACTIVE_TIMELINE); } /** @@ -218,7 +208,7 @@ public void saveAsComplete(HoodieInstant instant, Optional data) { public void revertToInflight(HoodieInstant instant) { log.info("Reverting instant to inflight " + instant); - moveCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant)); + revertCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant)); log.info("Reverted " + instant + " to inflight"); } @@ -255,23 +245,67 @@ public Optional getInstantDetails(HoodieInstant instant) { return readDataFromPath(detailPath); } - public void revertFromInflightToRequested(HoodieInstant inflightInstant, HoodieInstant requestedInstant, - Optional data) { + /** BEGIN - COMPACTION RELATED META-DATA MANAGEMENT **/ + + public Optional getInstantAuxiliaryDetails(HoodieInstant instant) { + Path detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName()); + return readDataFromPath(detailPath); + } + + /** + * Revert compaction State from inflight to requested + * + * @param inflightInstant Inflight Instant + * @return requested instant + */ + public HoodieInstant revertCompactionInflightToRequested(HoodieInstant inflightInstant) { Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); - transitionState(inflightInstant, requestedInstant, data); + Preconditions.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, inflightInstant.getTimestamp()); + transitionState(inflightInstant, requestedInstant, Optional.empty()); + return requestedInstant; } - public void transitionFromRequestedToInflight(HoodieInstant requestedInstant, HoodieInstant inflightInstant, - Optional data) { + /** + * Transition Compaction State from requested to inflight + * + * @param requestedInstant Requested instant + * @return inflight instant + */ + public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant) { Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); - transitionState(requestedInstant, inflightInstant, data); + Preconditions.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, requestedInstant.getTimestamp()); + transitionState(requestedInstant, inflightInstant, Optional.empty()); + return inflightInstant; } - protected void moveInflightToComplete(HoodieInstant inflightInstant, HoodieInstant commitInstant, - Optional data) { + /** + * Transition Compaction State from inflight to Committed + * + * @param inflightInstant Inflight instant + * @param data Extra Metadata + * @return commit instant + */ + public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflightInstant, Optional data) { + Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + Preconditions.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, COMMIT_ACTION, inflightInstant.getTimestamp()); transitionState(inflightInstant, commitInstant, data); + return commitInstant; } + private void createFileInAuxiliaryFolder(HoodieInstant instant, Optional data) { + Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName()); + createFileInPath(fullPath, data); + } + + /** + * END - COMPACTION RELATED META-DATA MANAGEMENT + **/ + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Optional data) { Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp())); @@ -290,7 +324,7 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, } } - protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { + private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp())); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName()); try { @@ -308,35 +342,44 @@ protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inf } public void saveToInflight(HoodieInstant instant, Optional content) { + Preconditions.checkArgument(instant.isInflight()); createFileInMetaPath(instant.getFileName(), content); } - public void saveToRequested(HoodieInstant instant, Optional content) { + public void saveToCompactionRequested(HoodieInstant instant, Optional content) { Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + // Write workload to auxiliary folder + createFileInAuxiliaryFolder(instant, content); createFileInMetaPath(instant.getFileName(), content); } - protected void createFileInMetaPath(String filename, Optional content) { + private void createFileInMetaPath(String filename, Optional content) { Path fullPath = new Path(metaClient.getMetaPath(), filename); + createFileInPath(fullPath, content); + } + + private void createFileInPath(Path fullPath, Optional content) { try { - if (!content.isPresent()) { + // If the path does not exist, create it first + if (!metaClient.getFs().exists(fullPath)) { if (metaClient.getFs().createNewFile(fullPath)) { log.info("Created a new file in meta path: " + fullPath); - return; + } else { + throw new HoodieIOException("Failed to create file " + fullPath); } - } else { + } + + if (content.isPresent()) { FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true); fsout.write(content.get()); fsout.close(); - return; } - throw new HoodieIOException("Failed to create file " + fullPath); } catch (IOException e) { throw new HoodieIOException("Failed to create file " + fullPath, e); } } - protected Optional readDataFromPath(Path detailPath) { + private Optional readDataFromPath(Path detailPath) { try (FSDataInputStream is = metaClient.getFs().open(detailPath)) { return Optional.of(IOUtils.toByteArray(is)); } catch (IOException e) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java index f87d614da4522..a21683f0a7b1d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java @@ -84,6 +84,10 @@ public HoodieInstant(State state, String action, String timestamp) { this.timestamp = timestamp; } + public boolean isCompleted() { + return state == State.COMPLETED; + } + public boolean isInflight() { return state == State.INFLIGHT; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java index 1cde27f318d12..93fee8cb423a1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java @@ -143,7 +143,7 @@ public static HoodieSavepointMetadata convertSavepointMetadata(String user, Stri partitionMetadataBuilder.build()); } - public static Optional serializeCompactionWorkload(HoodieCompactionPlan compactionWorkload) + public static Optional serializeCompactionPlan(HoodieCompactionPlan compactionWorkload) throws IOException { return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java index cadc9d7bd977c..2ee99eb8dc0e5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -21,6 +21,7 @@ import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.FileSlice; 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.exception.HoodieException; import java.io.IOException; @@ -114,7 +115,8 @@ public static List> getAllPendingCompa return pendingCompactionInstants.stream().map(instant -> { try { HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( - metaClient.getActiveTimeline().getInstantDetails(instant).get()); + metaClient.getActiveTimeline().getInstantAuxiliaryDetails( + HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); return Pair.of(instant, compactionPlan); } catch (IOException e) { throw new HoodieException(e); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 713b6b80275e9..3035712ea07b5 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -252,10 +252,12 @@ private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFi // Create a Data-file but this should be skipped by view new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile(); compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); - commitTimeline.saveToInflight(compactionInstant, AvroUtils.serializeCompactionWorkload(compactionPlan)); + HoodieInstant requested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + commitTimeline.saveToCompactionRequested(requested, AvroUtils.serializeCompactionPlan(compactionPlan)); + commitTimeline.transitionCompactionRequestedToInflight(requested); } else { compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); - commitTimeline.saveToRequested(compactionInstant, AvroUtils.serializeCompactionWorkload(compactionPlan)); + commitTimeline.saveToCompactionRequested(compactionInstant, AvroUtils.serializeCompactionPlan(compactionPlan)); } // Fake delta-ingestion after compaction-requested diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java index 9faaca3e5d25b..65edb2de5be3d 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java @@ -241,13 +241,17 @@ private Map> generateExpectedCom } private void scheduleCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { - metaClient.getActiveTimeline().saveToRequested(new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), - AvroUtils.serializeCompactionWorkload(compactionPlan)); + metaClient.getActiveTimeline().saveToCompactionRequested( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), + AvroUtils.serializeCompactionPlan(compactionPlan)); } private void scheduleInflightCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { - metaClient.getActiveTimeline().saveToInflight(new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, instantTime), - AvroUtils.serializeCompactionWorkload(compactionPlan)); + metaClient.getActiveTimeline().saveToCompactionRequested( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), + AvroUtils.serializeCompactionPlan(compactionPlan)); + metaClient.getActiveTimeline().transitionCompactionRequestedToInflight( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime)); } private HoodieCompactionPlan createCompactionPlan(String instantId, int numFileIds) { From a061b8376672bf0c9c8d8ac0d97f317352590acf Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 31 May 2018 14:16:19 -0700 Subject: [PATCH 071/374] Ensure Cleaner and Archiver do not delete file-slices and workload marked for compaction --- .../com/uber/hoodie/HoodieWriteClient.java | 88 +++++- .../com/uber/hoodie/io/HoodieCleanHelper.java | 70 +++-- .../hoodie/io/HoodieCommitArchiveLog.java | 77 ++++- .../java/com/uber/hoodie/TestCleaner.java | 280 ++++++++++++++++-- .../com/uber/hoodie/TestClientRollback.java | 2 +- .../common/HoodieTestDataGenerator.java | 9 + .../hoodie/io/TestHoodieCommitArchiveLog.java | 164 ++++++++++ .../common/model/CompactionOperation.java | 2 +- .../table/view/HoodieTableFileSystemView.java | 32 +- .../hoodie/common/model/HoodieTestUtils.java | 22 ++ 10 files changed, 668 insertions(+), 78 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 061fa908e95fd..8ff78619e89ee 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -65,6 +66,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; @@ -91,6 +93,7 @@ public class HoodieWriteClient implements Seriali private final transient FileSystem fs; private final transient JavaSparkContext jsc; private final HoodieWriteConfig config; + private final boolean rollbackInFlight; private final transient HoodieMetrics metrics; private final transient HoodieIndex index; private transient Timer.Context writeContext = null; @@ -122,10 +125,7 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, this.config = clientConfig; this.index = index; this.metrics = new HoodieMetrics(config, config.getTableName()); - - if (rollbackInFlight) { - rollbackInflightCommits(); - } + this.rollbackInFlight = rollbackInFlight; } public static SparkConf registerClasses(SparkConf conf) { @@ -681,6 +681,42 @@ public void deleteSavepoint(String savepointTime) { logger.info("Savepoint " + savepointTime + " deleted"); } + /** + * Delete a compaction request that is pending. + * + * NOTE - This is an Admin operation. + * With async compaction, this is expected to be called with async compaction and ingestion shutdown. + * Otherwise, async compactor could fail with errors + * + * @param compactionTime - delete the compaction time + * @return + */ + private void deletePendingCompaction(String compactionTime) { + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + + HoodieInstant compactionRequestedInstant = + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime); + HoodieInstant compactionInflightInstant = + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionTime); + boolean isCompactionInstantInRequestedState = table.getActiveTimeline().filterPendingCompactionTimeline() + .containsInstant(compactionRequestedInstant); + boolean isCompactionInstantInInflightState = table.getActiveTimeline().filterPendingCompactionTimeline() + .containsInstant(compactionInflightInstant); + + if (isCompactionInstantInRequestedState) { + activeTimeline.deleteCompactionRequested(compactionRequestedInstant); + } else if (isCompactionInstantInInflightState) { + activeTimeline.revertCompactionInflightToRequested(compactionInflightInstant); + activeTimeline.deleteCompactionRequested(compactionRequestedInstant); + } else { + logger.error("No compaction present " + compactionTime); + throw new IllegalArgumentException("No compaction present " + compactionTime); + } + logger.info("Compaction " + compactionTime + " deleted"); + } + /** * Rollback the state to the savepoint. WARNING: This rollsback recent commits and deleted data * files. Queries accessing the files will mostly fail. This should be done during a downtime. @@ -692,6 +728,11 @@ public boolean rollbackToSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + + // Rollback to savepoint is expected to be a manual operation and no concurrent ingestion or compaction is expected + // to be running. Rollback to savepoint also removes any pending compaction actions that are generated after + // savepoint time. Allowing pending compaction to be retained is not safe as those workload could be referencing + // file-slices that will be rolled-back as part of this operation HoodieTimeline commitTimeline = table.getMetaClient().getCommitsAndCompactionTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, @@ -740,8 +781,10 @@ private void rollback(List commits) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); + Set pendingCompactions = + table.getActiveTimeline().filterPendingCompactionTimeline().getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); // Check if any of the commits is a savepoint - do not allow rollback on those commits @@ -755,37 +798,45 @@ private void rollback(List commits) { } }); + List pendingCompactionToRollback = + commits.stream().filter(c -> pendingCompactions.contains(c)).collect(Collectors.toList()); + List commitsToRollback = + commits.stream().filter(c -> !pendingCompactions.contains(c)).collect(Collectors.toList()); + try { - if (commitTimeline.empty() && inflightTimeline.empty()) { + if (commitTimeline.empty() && inflightCommitTimeline.empty()) { // nothing to rollback - logger.info("No commits to rollback " + commits); + logger.info("No commits to rollback " + commitsToRollback); } // Make sure only the last n commits are being rolled back // If there is a commit in-between or after that is not rolled back, then abort - String lastCommit = commits.get(commits.size() - 1); + String lastCommit = commitsToRollback.get(commitsToRollback.size() - 1); if (!commitTimeline.empty() && !commitTimeline .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { throw new HoodieRollbackException( "Found commits after time :" + lastCommit + ", please rollback greater commits first"); } - List inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) + List inflights = inflightCommitTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) { throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit + ", please rollback greater commits first"); } - List stats = table.rollback(jsc, commits); + // Remove interleaving pending compactions before rolling back commits + pendingCompactionToRollback.stream().forEach(this::deletePendingCompaction); + + List stats = table.rollback(jsc, commitsToRollback); // cleanup index entries - commits.stream().forEach(s -> { + commitsToRollback.stream().forEach(s -> { if (!index.rollbackCommit(s)) { throw new HoodieRollbackException("Rollback index changes failed, for time :" + s); } }); - logger.info("Index rolled back for commits " + commits); + logger.info("Index rolled back for commits " + commitsToRollback); Optional durationInMs = Optional.empty(); if (context != null) { @@ -795,11 +846,11 @@ private void rollback(List commits) { metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); } HoodieRollbackMetadata rollbackMetadata = AvroUtils - .convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); + .convertRollbackMetadata(startRollbackTime, durationInMs, commitsToRollback, stats); table.getActiveTimeline().saveAsComplete( new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime), AvroUtils.serializeRollbackMetadata(rollbackMetadata)); - logger.info("Commits " + commits + " rollback is complete"); + logger.info("Commits " + commitsToRollback + " rollback is complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) { logger.info("Cleaning up older rollback meta files"); @@ -810,7 +861,7 @@ private void rollback(List commits) { } } catch (IOException e) { throw new HoodieRollbackException( - "Failed to rollback " + config.getBasePath() + " commits " + commits, e); + "Failed to rollback " + config.getBasePath() + " commits " + commitsToRollback, e); } } @@ -890,6 +941,10 @@ public String startCommit() { } public void startCommitWithTime(String commitTime) { + if (rollbackInFlight) { + // Only rollback inflight commit/delta-commits. Do not touch compaction commits + rollbackInflightCommits(); + } logger.info("Generate a new commit time " + commitTime); HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); @@ -1061,6 +1116,7 @@ private void rollbackInflightCommits() { } private HoodieTable getTableAndInitCtx() { + // Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index 2f7006fe4e57f..e7864d496645e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -16,6 +16,7 @@ package com.uber.hoodie.io; +import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieCleaningPolicy; import com.uber.hoodie.common.model.HoodieDataFile; @@ -25,14 +26,17 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -48,6 +52,7 @@ public class HoodieCleanHelper> { private final TableFileSystemView fileSystemView; private final HoodieTimeline commitTimeline; + private final Map fileIdToPendingCompactionOperations; private HoodieTable hoodieTable; private HoodieWriteConfig config; @@ -56,9 +61,12 @@ public HoodieCleanHelper(HoodieTable hoodieTable, HoodieWriteConfig config) { this.fileSystemView = hoodieTable.getCompletedFileSystemView(); this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); this.config = config; + this.fileIdToPendingCompactionOperations = + ((HoodieTableFileSystemView)hoodieTable.getRTFileSystemView()).getFileIdToPendingCompaction().entrySet() + .stream().map(entry -> Pair.of(entry.getKey(), entry.getValue().getValue())) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } - /** * Selects the older versions of files for cleaning, such that it bounds the number of versions of * each file. This policy is useful, if you are simply interested in querying the table, and you @@ -81,8 +89,8 @@ private List getFilesToCleanKeepingLatestVersions(String partitionPath) while (fileSliceIterator.hasNext() && keepVersions > 0) { // Skip this most recent version FileSlice nextSlice = fileSliceIterator.next(); - HoodieDataFile dataFile = nextSlice.getDataFile().get(); - if (savepointedFiles.contains(dataFile.getFileName())) { + Optional dataFile = nextSlice.getDataFile(); + if (dataFile.isPresent() && savepointedFiles.contains(dataFile.get().getFileName())) { // do not clean up a savepoint data file continue; } @@ -91,12 +99,16 @@ private List getFilesToCleanKeepingLatestVersions(String partitionPath) // Delete the remaining files while (fileSliceIterator.hasNext()) { FileSlice nextSlice = fileSliceIterator.next(); - HoodieDataFile dataFile = nextSlice.getDataFile().get(); - deletePaths.add(dataFile.getFileStatus().getPath().toString()); - if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { - // If merge on read, then clean the log files for the commits as well - deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString()) - .collect(Collectors.toList())); + if (!isFileSliceNeededForPendingCompaction(nextSlice)) { + if (nextSlice.getDataFile().isPresent()) { + HoodieDataFile dataFile = nextSlice.getDataFile().get(); + deletePaths.add(dataFile.getFileStatus().getPath().toString()); + } + if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { + // If merge on read, then clean the log files for the commits as well + deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString()) + .collect(Collectors.toList())); + } } } } @@ -133,17 +145,21 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) .collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { List fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList()); - HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get(); - String lastVersion = dataFile.getCommitTime(); + + if (fileSliceList.isEmpty()) { + continue; + } + + String lastVersion = fileSliceList.get(0).getBaseInstantTime(); String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain); // Ensure there are more than 1 version of the file (we only clean old files from updates) // i.e always spare the last commit. for (FileSlice aSlice : fileSliceList) { - HoodieDataFile aFile = aSlice.getDataFile().get(); - String fileCommitTime = aFile.getCommitTime(); - if (savepointedFiles.contains(aFile.getFileName())) { + Optional aFile = aSlice.getDataFile(); + String fileCommitTime = aSlice.getBaseInstantTime(); + if (aFile.isPresent() && savepointedFiles.contains(aFile.get().getFileName())) { // do not clean up a savepoint data file continue; } @@ -159,11 +175,14 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) } // Always keep the last commit - if (HoodieTimeline + if (!isFileSliceNeededForPendingCompaction(aSlice) + && HoodieTimeline .compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) { // this is a commit, that should be cleaned. - deletePaths.add(aFile.getFileStatus().getPath().toString()); + if (aFile.isPresent()) { + deletePaths.add(aFile.get().getFileStatus().getPath().toString()); + } if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { // If merge on read, then clean the log files for the commits as well deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString()) @@ -183,7 +202,7 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) private String getLatestVersionBeforeCommit(List fileSliceList, HoodieInstant commitTime) { for (FileSlice file : fileSliceList) { - String fileCommitTime = file.getDataFile().get().getCommitTime(); + String fileCommitTime = file.getBaseInstantTime(); if (HoodieTimeline .compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) { // fileList is sorted on the reverse, so the first commit we find <= commitTime is the @@ -226,4 +245,19 @@ public Optional getEarliestCommitToRetain() { } return earliestCommitToRetain; } -} + + /** + * Determine if file slice needed to be preserved for pending compaction + * @param fileSlice File Slice + * @return true if file slice needs to be preserved, false otherwise. + */ + private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) { + CompactionOperation op = fileIdToPendingCompactionOperations.get(fileSlice.getFileId()); + if (null != op) { + // If file slice's instant time is newer or same as that of operation, do not clean + return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), op.getBaseInstantTime(), + HoodieTimeline.GREATER_OR_EQUAL); + } + return false; + } +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index bf78467ffe384..bf5febd1a2689 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -16,6 +16,10 @@ package com.uber.hoodie.io; +import static com.uber.hoodie.common.table.HoodieTimeline.COMMIT_ACTION; +import static com.uber.hoodie.common.table.HoodieTimeline.DELTA_COMMIT_ACTION; +import static com.uber.hoodie.common.table.HoodieTimeline.LESSER_OR_EQUAL; + import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; @@ -32,6 +36,7 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.AvroUtils; @@ -42,6 +47,7 @@ import com.uber.hoodie.table.HoodieTable; import java.io.IOException; import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -100,7 +106,7 @@ private void close() { /** * Check if commits need to be archived. If yes, archive commits. */ - public boolean archiveIfRequired(final JavaSparkContext jsc) { + public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException { try { List instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList()); boolean success = true; @@ -144,23 +150,34 @@ private Stream getInstantsToArchive(JavaSparkContext jsc) { //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify // with logic above to avoid Stream.concats HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); + Optional oldestPendingCompactionInstant = + table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); + // We cannot have any holes in the commit timeline. We cannot archive any commits which are // made after the first savepoint present. Optional firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) { // Actually do the commits - instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> { - // if no savepoint present, then dont filter - return !(firstSavepoint.isPresent() && HoodieTimeline - .compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(), - HoodieTimeline.LESSER_OR_EQUAL)); - }).limit(commitTimeline.countInstants() - minCommitsToKeep)); + instants = Stream.concat(instants, commitTimeline.getInstants() + .filter(s -> { + // if no savepoint present, then dont filter + return !(firstSavepoint.isPresent() && HoodieTimeline + .compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(), + HoodieTimeline.LESSER_OR_EQUAL)); + }) + .filter(s -> { + // Ensure commits >= oldest pending compaction commit is retained + return oldestPendingCompactionInstant.map(instant -> { + return HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER); + }).orElse(true); + }) + .limit(commitTimeline.countInstants() - minCommitsToKeep)); } return instants; } - private boolean deleteArchivedInstants(List archivedInstants) { + private boolean deleteArchivedInstants(List archivedInstants) throws IOException { log.info("Deleting instants " + archivedInstants); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { @@ -174,6 +191,48 @@ private boolean deleteArchivedInstants(List archivedInstants) { throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); } } + + // Remove older meta-data from auxiliary path too + Optional latestCommitted = + archivedInstants.stream() + .filter(i -> { + return i.isCompleted() + && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); + }) + .sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()).findFirst(); + if (latestCommitted.isPresent()) { + success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); + } + return success; + } + + /** + * Remove older instants from auxiliary meta folder + * + * @param thresholdInstant Hoodie Instant + * @return success if all eligible file deleted successfully + * @throws IOException in case of error + */ + private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant) + throws IOException { + List instants = + HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(), + new Path(metaClient.getMetaAuxiliaryPath()), + HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE); + + List instantsToBeDeleted = + instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(), + thresholdInstant.getTimestamp(), LESSER_OR_EQUAL)).collect(Collectors.toList()); + + boolean success = true; + for (HoodieInstant deleteInstant : instantsToBeDeleted) { + log.info("Deleting instant " + deleteInstant + " in auxiliary meta path " + metaClient.getMetaAuxiliaryPath()); + Path metaFile = new Path(metaClient.getMetaAuxiliaryPath(), deleteInstant.getFileName()); + if (metaClient.getFs().exists(metaFile)) { + success &= metaClient.getFs().delete(metaFile, false); + log.info("Deleted instant file in auxiliary metapath : " + metaFile); + } + } return success; } @@ -212,7 +271,7 @@ private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, archivedMetaWrapper.setActionType(ActionType.clean.name()); break; } - case HoodieTimeline.COMMIT_ACTION: { + case COMMIT_ACTION: { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get()); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index a05cbc7b6eb6a..2408333be8fc5 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -19,12 +19,16 @@ import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; +import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; +import static com.uber.hoodie.common.table.HoodieTimeline.GREATER; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.common.collect.Iterables; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.HoodieCleanStat; +import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieCleaningPolicy; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieDataFile; @@ -38,12 +42,16 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -53,7 +61,10 @@ import java.util.Optional; import java.util.Set; import java.util.TreeSet; +import java.util.function.Predicate; import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -61,6 +72,7 @@ import org.apache.spark.scheduler.SparkListener; import org.apache.spark.scheduler.SparkListenerTaskEnd; import org.apache.spark.util.AccumulatorV2; +import org.junit.Assert; import org.junit.Test; import scala.Option; import scala.collection.Iterator; @@ -82,7 +94,7 @@ public class TestCleaner extends TestHoodieClientBase { * @param insertFn Insertion API for testing * @throws Exception in case of error */ - private void insertFirstBigBatchForClientCleanerTest( + private String insertFirstBigBatchForClientCleanerTest( HoodieWriteConfig cfg, HoodieWriteClient client, Function2, String, Integer> recordGenFunction, @@ -118,6 +130,7 @@ private void insertFirstBigBatchForClientCleanerTest( HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); + return newCommitTime; } /** @@ -185,21 +198,51 @@ private void testInsertAndCleanByVersions( insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn); + Map selectedFileIdForCompaction = new HashMap<>(); + Map compactionFileIdToLatestFileSlice = new HashMap<>(); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); + for (String partitionPath : dataGen.getPartitionPaths()) { + TableFileSystemView fsView = table.getFileSystemView(); + Optional added = fsView.getAllFileGroups(partitionPath).findFirst() + .map(fg -> { + selectedFileIdForCompaction.put(fg.getId(), partitionPath); + fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getId(), fs)); + return true; + }); + if (added.isPresent()) { + // Select only one file-group for compaction + break; + } + } + + // Create workload with selected file-slices + List> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream() + .map(e -> Pair.of(selectedFileIdForCompaction.get(e.getKey()), e.getValue())).collect(Collectors.toList()); + HoodieCompactionPlan compactionPlan = + CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Optional.empty(), Optional.empty()); + List instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1); + String compactionTime = instantTimes.get(0); + table.getActiveTimeline().saveToCompactionRequested( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionTime), + AvroUtils.serializeCompactionPlan(compactionPlan)); + + instantTimes = instantTimes.subList(1, instantTimes.size()); // Keep doing some writes and clean inline. Make sure we have expected number of files // remaining. - HoodieTestUtils.monotonicIncreasingCommitTimestamps(8, 1).stream().forEach(newCommitTime -> { + for (String newInstantTime : instantTimes) { try { - client.startCommitWithTime(newCommitTime); - List records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100); + client.startCommitWithTime(newInstantTime); + List records = recordUpsertGenWrappedFunction.apply(newInstantTime, 100); List statuses = - upsertFn.apply(client, jsc.parallelize(records, 1), newCommitTime).collect(); + upsertFn.apply(client, jsc.parallelize(records, 1), newInstantTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); - HoodieTimeline timeline = metadata.getCommitsTimeline(); + metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); + HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline(); TableFileSystemView fsView = table.getFileSystemView(); // Need to ensure the following @@ -221,25 +264,39 @@ private void testInsertAndCleanByVersions( List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { - // No file has no more than max versions - String fileId = fileGroup.getId(); - List dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList()); - - assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions", - dataFiles.size() <= maxVersions); - - // Each file, has the latest N versions (i.e cleaning gets rid of older versions) - List commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId)); - for (int i = 0; i < dataFiles.size(); i++) { - assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions, - Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i)); + if (selectedFileIdForCompaction.containsKey(fileGroup.getId())) { + // Ensure latest file-slice selected for compaction is retained + String oldestCommitRetained = + fileGroup.getAllDataFiles().map(HoodieDataFile::getCommitTime).sorted().findFirst().get(); + + Optional dataFileForCompactionPresent = + fileGroup.getAllDataFiles().filter(df -> { + return compactionFileIdToLatestFileSlice.get(fileGroup.getId()) + .getBaseInstantTime().equals(df.getCommitTime()); + }).findAny(); + Assert.assertTrue("Data File selected for compaction is retained", + dataFileForCompactionPresent.isPresent()); + } else { + // file has no more than max versions + String fileId = fileGroup.getId(); + List dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList()); + + assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions", + dataFiles.size() <= maxVersions); + + // Each file, has the latest N versions (i.e cleaning gets rid of older versions) + List commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId)); + for (int i = 0; i < dataFiles.size(); i++) { + assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions, + Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i)); + } } } } } catch (IOException ioe) { throw new RuntimeException(ioe); } - }); + } } /** @@ -679,6 +736,168 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3); } + + /** + * Test Keep Latest Commits when there are pending compactions + */ + @Test + public void testKeepLatestCommitsWithPendingCompactions() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); + // Deletions: + // . FileId Parquet Logs Total Retained Commits + // FileId7 5 10 15 009, 011 + // FileId6 5 10 15 009 + // FileId5 3 6 9 005 + // FileId4 2 4 6 003 + // FileId3 1 2 3 001 + // FileId2 0 0 0 000 + // FileId1 0 0 0 000 + testPendingCompactions(config, 48, 18); + } + + /** + * Test Keep Latest Versions when there are pending compactions + */ + @Test + public void testKeepLatestVersionsWithPendingCompactions() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build()).build(); + // Deletions: + // . FileId Parquet Logs Total Retained Commits + // FileId7 5 10 15 009, 011 + // FileId6 4 8 12 007, 009 + // FileId5 2 4 6 003 005 + // FileId4 1 2 3 001, 003 + // FileId3 0 0 0 000, 001 + // FileId2 0 0 0 000 + // FileId1 0 0 0 000 + testPendingCompactions(config, 36, 9); + } + + /** + * Common test method for validating pending compactions + * + * @param config Hoodie Write Config + * @param expNumFilesDeleted Number of files deleted + */ + public void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDeleted, + int expNumFilesUnderCompactionDeleted) throws IOException { + HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, + HoodieTableType.MERGE_ON_READ); + String[] instants = new String[]{"000", "001", "003", "005", "007", "009", "011", "013"}; + String[] compactionInstants = new String[]{"002", "004", "006", "008", "010"}; + Map expFileIdToPendingCompaction = new HashMap<>(); + Map fileIdToLatestInstantBeforeCompaction = new HashMap<>(); + Map> compactionInstantsToFileSlices = new HashMap<>(); + + for (String instant : instants) { + HoodieTestUtils.createCommitFiles(basePath, instant); + } + + // Generate 7 file-groups. First one has only one slice and no pending compaction. File Slices (2 - 5) has + // multiple versions with pending compaction. File Slices (6 - 7) have multiple file-slices but not under + // compactions + // FileIds 2-5 will be under compaction + int maxNumFileIds = 7; + String[] fileIds = new String[] + {"fileId1", "fileId2", "fileId3", "fileId4", "fileId5", "fileId6", "fileId7"}; + int maxNumFileIdsForCompaction = 4; + for (int i = 0; i < maxNumFileIds; i++) { + final String fileId = HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0], + fileIds[i]); + HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0], + fileId, Optional.empty()); + HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0], + fileId, Optional.of(2)); + fileIdToLatestInstantBeforeCompaction.put(fileId, instants[0]); + for (int j = 1; j <= i; j++) { + if (j == i && j <= maxNumFileIdsForCompaction) { + expFileIdToPendingCompaction.put(fileId, compactionInstants[j]); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); + FileSlice slice = table.getRTFileSystemView().getLatestFileSlices(DEFAULT_FIRST_PARTITION_PATH) + .filter(fs -> fs.getFileId().equals(fileId)).findFirst().get(); + List slices = new ArrayList<>(); + if (compactionInstantsToFileSlices.containsKey(compactionInstants[j])) { + slices = compactionInstantsToFileSlices.get(compactionInstants[j]); + } + slices.add(slice); + compactionInstantsToFileSlices.put(compactionInstants[j], slices); + // Add log-files to simulate delta-commits after pending compaction + HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j], + fileId, Optional.empty()); + HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j], + fileId, Optional.of(2)); + } else { + HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId); + HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId, + Optional.empty()); + HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId, + Optional.of(2)); + fileIdToLatestInstantBeforeCompaction.put(fileId, instants[j]); + } + } + } + + // Setup pending compaction plans + for (String instant : compactionInstants) { + List fileSliceList = compactionInstantsToFileSlices.get(instant); + if (null != fileSliceList) { + HoodieTestUtils.createCompactionRequest(metaClient, instant, + fileSliceList.stream().map(fs -> Pair.of(DEFAULT_FIRST_PARTITION_PATH, fs)).collect(Collectors.toList())); + } + } + + // Clean now + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); + List hoodieCleanStats = table.clean(jsc); + + // Test for safety + final HoodieTable hoodieTable = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); + + expFileIdToPendingCompaction.entrySet().stream().forEach(entry -> { + String fileId = entry.getKey(); + String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId); + Optional fileSliceForCompaction = + hoodieTable.getRTFileSystemView().getLatestFileSlicesBeforeOrOn(DEFAULT_FIRST_PARTITION_PATH, + baseInstantForCompaction).filter(fs -> fs.getFileId().equals(fileId)).findFirst(); + Assert.assertTrue("Base Instant for Compaction must be preserved", fileSliceForCompaction.isPresent()); + Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getDataFile().isPresent()); + Assert.assertEquals("FileSlice has log-files", 2, + fileSliceForCompaction.get().getLogFiles().count()); + }); + + // Test for progress (Did we clean some files ?) + long numFilesUnderCompactionDeleted = + hoodieCleanStats.stream().flatMap(cleanStat -> { + return convertPathToFileIdWithCommitTime(metaClient, cleanStat.getDeletePathPatterns()).map( + fileIdWithCommitTime -> { + if (expFileIdToPendingCompaction.containsKey(fileIdWithCommitTime.getKey())) { + Assert.assertTrue("Deleted instant time must be less than pending compaction", + HoodieTimeline.compareTimestamps( + fileIdToLatestInstantBeforeCompaction.get(fileIdWithCommitTime.getKey()), + fileIdWithCommitTime.getValue(), GREATER)); + return true; + } + return false; + }); + }).filter(x -> x).count(); + long numDeleted = hoodieCleanStats.stream() + .flatMap(cleanStat -> cleanStat.getDeletePathPatterns().stream()).count(); + // Tighter check for regression + Assert.assertEquals("Correct number of files deleted", expNumFilesDeleted, numDeleted); + Assert.assertEquals("Correct number of files under compaction deleted", + expNumFilesUnderCompactionDeleted, numFilesUnderCompactionDeleted); + } + /** * Utility method to create temporary data files * @@ -703,4 +922,23 @@ private List createTempFiles(String commitTime, int numFiles) throws IOE private int getTotalTempFiles() throws IOException { return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length; } + + private Stream> convertPathToFileIdWithCommitTime( + final HoodieTableMetaClient metaClient, List paths) { + Predicate roFilePredicate = path -> + path.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); + Predicate rtFilePredicate = path -> + path.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); + Stream> stream1 = paths.stream().filter(roFilePredicate) + .map(fullPath -> { + String fileName = Paths.get(fullPath).getFileName().toString(); + return Pair.of(FSUtils.getFileId(fileName), FSUtils.getCommitTime(fileName)); + }); + Stream> stream2 = paths.stream().filter(rtFilePredicate) + .map(path -> { + return Pair.of(FSUtils.getFileIdFromLogPath(new Path(path)), + FSUtils.getBaseCommitTimeFromLogPath(new Path(path))); + }); + return Stream.concat(stream1, stream2); + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java index 68e3e9f0b81ee..2c88602196218 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java @@ -303,7 +303,7 @@ public void testAutoRollbackInflightCommit() throws Exception { && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); // Turn auto rollback on - new HoodieWriteClient(jsc, config, true); + new HoodieWriteClient(jsc, config, true).startCommit(); assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index 1d3f6a7ee1cfd..a36faef1cadef 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -138,6 +138,15 @@ public static void createCommitFile(String basePath, String commitTime, Configur } } + public static void createCompactionRequestedFile(String basePath, String commitTime, Configuration configuration) + throws IOException { + Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + HoodieTimeline.makeRequestedCompactionFileName(commitTime)); + FileSystem fs = FSUtils.getFs(basePath, configuration); + FSDataOutputStream os = fs.create(commitFile, true); + os.close(); + } + public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInstant instant, Configuration configuration) throws IOException { Path commitFile = new Path( diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index 57d285688e920..6f03691803e8e 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -16,7 +16,9 @@ package com.uber.hoodie.io; +import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.common.collect.Sets; @@ -30,7 +32,9 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import java.io.File; @@ -133,11 +137,47 @@ public void testArchiveDatasetWithArchival() throws IOException { HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build()) .forTable("test-trip-table").build(); HoodieTestUtils.init(hadoopConf, basePath); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "104"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "105"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "105"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); @@ -172,6 +212,37 @@ public void testArchiveDatasetWithArchival() throws IOException { timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); + // Check compaction instants + List instants = + HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(), + new Path(metaClient.getMetaAuxiliaryPath()), + HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE); + assertEquals("Should delete all compaction instants < 104", 4, instants.size()); + assertFalse("Requested Compaction must be absent for 100", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"))); + assertFalse("Inflight Compaction must be absent for 100", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"))); + assertFalse("Requested Compaction must be absent for 101", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"))); + assertFalse("Inflight Compaction must be absent for 101", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"))); + assertFalse("Requested Compaction must be absent for 102", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"))); + assertFalse("Inflight Compaction must be absent for 102", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"))); + assertFalse("Requested Compaction must be absent for 103", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"))); + assertFalse("Inflight Compaction must be absent for 103", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"))); + assertTrue("Requested Compaction must be present for 104", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"))); + assertTrue("Inflight Compaction must be present for 104", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "104"))); + assertTrue("Requested Compaction must be present for 105", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "105"))); + assertTrue("Inflight Compaction must be present for 105", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "105"))); + //read the file HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(dfs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), @@ -210,9 +281,33 @@ public void testArchiveDatasetWithNoArchival() throws IOException { HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + // Requested Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"), dfs.getConf()); + // Inflight Compaction + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"), dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); @@ -221,6 +316,28 @@ public void testArchiveDatasetWithNoArchival() throws IOException { assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); + + List instants = + HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(), + new Path(metaClient.getMetaAuxiliaryPath()), + HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE); + assertEquals("Should not delete any aux compaction files when maxCommitsToKeep is 5", 8, instants.size()); + assertTrue("Requested Compaction must be present for 100", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"))); + assertTrue("Inflight Compaction must be present for 100", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"))); + assertTrue("Requested Compaction must be present for 101", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"))); + assertTrue("Inflight Compaction must be present for 101", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"))); + assertTrue("Requested Compaction must be present for 102", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"))); + assertTrue("Inflight Compaction must be present for 102", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"))); + assertTrue("Requested Compaction must be present for 103", instants.contains( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"))); + assertTrue("Inflight Compaction must be present for 103", instants.contains( + new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"))); } @Test @@ -281,6 +398,53 @@ public void testArchiveCommitSavepointNoHole() throws IOException { timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103"))); } + @Test + public void testArchiveCommitCompactionNoHole() throws IOException { + HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .forTable("test-trip-table").withCompactionConfig( + HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); + HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", dfs.getConf()); + HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"), dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "106", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "107", dfs.getConf()); + + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); + assertEquals("Loaded 6 commits and the count should match", 8, timeline.countInstants()); + boolean result = archiveLog.archiveIfRequired(jsc); + assertTrue(result); + timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline(); + assertFalse("Instants before oldest pending compaction can be removed", + timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100"))); + assertEquals( + "Since we have a pending compaction at 101, we should never archive any commit " + + "after 101 (we only " + "archive 100)", 7, timeline.countInstants()); + assertTrue("Requested Compaction must still be present", + timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"))); + assertTrue("Instants greater than oldest pending compaction must be present", + timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102"))); + assertTrue("Instants greater than oldest pending compaction must be present", + timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103"))); + assertTrue("Instants greater than oldest pending compaction must be present", + timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"))); + assertTrue("Instants greater than oldest pending compaction must be present", + timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105"))); + assertTrue("Instants greater than oldest pending compaction must be present", + timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106"))); + assertTrue("Instants greater than oldest pending compaction must be present", + timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107"))); + } + private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { HoodieTimeline timeline = metaClient.getActiveTimeline().reload() .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java index 9f23c4a12aff8..f7f886e2a7a28 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java @@ -106,7 +106,7 @@ public static CompactionOperation convertFromAvroRecordInstance(HoodieCompaction op.dataFilePath = Optional.ofNullable(operation.getDataFilePath()); op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths()); op.fileId = operation.getFileId(); - op.metrics = new HashMap<>(operation.getMetrics()); + op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics()); op.partitionPath = operation.getPartitionPath(); return op; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index e1ef48a35df30..8aa585728b7e9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -17,6 +17,7 @@ package com.uber.hoodie.common.table.view; import com.google.common.collect.ImmutableMap; +import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieFileGroup; @@ -74,7 +75,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView, /** * File Id to pending compaction instant time */ - private final Map fileIdToPendingCompactionInstantTime; + private final Map> fileIdToPendingCompaction; /** * Create a file system view, as of the given timeline @@ -89,9 +90,10 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, // Build fileId to Pending Compaction Instants List pendingCompactionInstants = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); - this.fileIdToPendingCompactionInstantTime = ImmutableMap.copyOf( + this.fileIdToPendingCompaction = ImmutableMap.copyOf( CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream().map(entry -> { - return Pair.of(entry.getKey(), entry.getValue().getKey()); + return Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(), + CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue()))); }).collect(Collectors.toMap(Pair::getKey, Pair::getValue))); } @@ -155,10 +157,10 @@ private List addFilesToView(FileStatus[] statuses) { if (logFiles.containsKey(pair)) { logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); } - if (fileIdToPendingCompactionInstantTime.containsKey(fileId)) { + if (fileIdToPendingCompaction.containsKey(fileId)) { // If there is no delta-commit after compaction request, this step would ensure a new file-slice appears // so that any new ingestion uses the correct base-instant - group.addNewFileSliceAtInstant(fileIdToPendingCompactionInstantTime.get(fileId)); + group.addNewFileSliceAtInstant(fileIdToPendingCompaction.get(fileId).getKey()); } fileGroups.add(group); }); @@ -196,8 +198,9 @@ private Stream convertFileStatusesToLogFiles(FileStatus[] statuse * @param dataFile Data File */ private boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) { - String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(dataFile.getFileId()); - if ((null != compactionInstantTime) && dataFile.getCommitTime().equals(compactionInstantTime)) { + Pair compactionWithInstantTime = fileIdToPendingCompaction.get(dataFile.getFileId()); + if ((null != compactionWithInstantTime) && (null != compactionWithInstantTime.getLeft()) + && dataFile.getCommitTime().equals(compactionWithInstantTime.getKey())) { return true; } return false; @@ -277,7 +280,7 @@ public Stream getLatestUnCompactedFileSlices(String partitionPath) { FileSlice fileSlice = fileGroup.getLatestFileSlice().get(); // if the file-group is under compaction, pick the latest before compaction instant time. if (isFileSliceAfterPendingCompaction(fileSlice)) { - String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(fileSlice.getFileId()); + String compactionInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()).getLeft(); return fileGroup.getLatestFileSliceBefore(compactionInstantTime); } return Optional.of(fileSlice); @@ -292,8 +295,9 @@ public Stream getLatestUnCompactedFileSlices(String partitionPath) { * @return */ private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) { - String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(fileSlice.getFileId()); - if ((null != compactionInstantTime) && fileSlice.getBaseInstantTime().equals(compactionInstantTime)) { + Pair compactionWithInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()); + if ((null != compactionWithInstantTime) + && fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey())) { return true; } return false; @@ -352,8 +356,8 @@ private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, F */ private FileSlice getMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) { // if the file-group is under construction, pick the latest before compaction instant time. - if (fileIdToPendingCompactionInstantTime.containsKey(fileSlice.getFileId())) { - String compactionInstantTime = fileIdToPendingCompactionInstantTime.get(fileSlice.getFileId()); + if (fileIdToPendingCompaction.containsKey(fileSlice.getFileId())) { + String compactionInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()).getKey(); if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) { Optional prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime); if (prevFileSlice.isPresent()) { @@ -416,4 +420,8 @@ public Stream getAllFileGroups(String partitionPathStr) { "Failed to list data files in partition " + partitionPathStr, e); } } + + public Map> getFileIdToPendingCompaction() { + return fileIdToPendingCompaction; + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 0fbb46cd28f20..475cc4f804f82 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.uber.hoodie.avro.model.HoodieCleanMetadata; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.table.HoodieTableConfig; @@ -36,7 +37,10 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import java.io.ByteArrayInputStream; @@ -61,6 +65,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -113,6 +118,14 @@ public static final void createCommitFiles(String basePath, String... commitTime } } + public static final void createDeltaCommitFiles(String basePath, String... commitTimes) throws IOException { + for (String commitTime : commitTimes) { + new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeDeltaFileName(commitTime)) + .createNewFile(); + } + } + public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName( @@ -177,6 +190,15 @@ public static final void createCompactionCommitFiles(FileSystem fs, String baseP } } + public static final void createCompactionRequest(HoodieTableMetaClient metaClient, String instant, + List> fileSliceList) throws IOException { + HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Optional.empty(), Optional.empty()); + HoodieInstant compactionInstant = + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant); + metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, + AvroUtils.serializeCompactionPlan(plan)); + } + public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) throws IOException { return basePath + "/" + partitionPath + "/" + FSUtils From 202c0800a634ceacdc3c77515255fe5c5047bea5 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 23 May 2018 23:09:25 -0700 Subject: [PATCH 072/374] Async Compaction Main API changes --- .../com/uber/hoodie/HoodieReadClient.java | 17 + .../com/uber/hoodie/HoodieWriteClient.java | 361 +++++++++---- .../HoodieCompactionInstantWithPlan.java | 42 -- .../hoodie/io/compact/HoodieCompactor.java | 15 +- .../compact/HoodieRealtimeTableCompactor.java | 43 +- .../uber/hoodie/metrics/HoodieMetrics.java | 10 + .../hoodie/table/HoodieCopyOnWriteTable.java | 5 - .../hoodie/table/HoodieMergeOnReadTable.java | 19 +- .../com/uber/hoodie/table/HoodieTable.java | 7 - .../com/uber/hoodie/TestAsyncCompaction.java | 508 ++++++++++++++++++ .../com/uber/hoodie/TestHoodieClientBase.java | 18 +- .../uber/hoodie/io/TestHoodieCompactor.java | 21 +- .../hoodie/table/TestMergeOnReadTable.java | 37 +- .../test/resources/log4j-surefire.properties | 8 +- .../test/resources/log4j-surefire.properties | 6 +- .../test/resources/log4j-surefire.properties | 4 +- .../test/resources/log4j-surefire.properties | 6 +- .../test/resources/log4j-surefire.properties | 8 +- 18 files changed, 873 insertions(+), 262 deletions(-) delete mode 100644 hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 8dac00573cea7..416d2905c3340 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -17,11 +17,13 @@ package com.uber.hoodie; import com.google.common.base.Optional; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; 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.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -32,6 +34,8 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -183,4 +187,17 @@ public JavaRDD> tagLocation(JavaRDD> hoodieRecor throws HoodieIndexException { return index.tagLocation(hoodieRecords, jsc, hoodieTable); } + + /** + * Return all pending compactions with instant time for clients to decide what to compact next. + * @return + */ + public List> getPendingCompactions() { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + hoodieTable.getMetaClient().getBasePath(), true); + return CompactionUtils.getAllPendingCompactionPlans(metaClient).stream() + .map(instantWorkloadPair -> + Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue())) + .collect(Collectors.toList()); + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 8ff78619e89ee..84bceac825c69 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -19,8 +19,10 @@ import com.codahale.metrics.Timer; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.uber.hoodie.avro.model.HoodieCleanMetadata; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.avro.model.HoodieRollbackMetadata; import com.uber.hoodie.avro.model.HoodieSavepointMetadata; import com.uber.hoodie.common.HoodieCleanStat; @@ -97,6 +99,7 @@ public class HoodieWriteClient implements Seriali private final transient HoodieMetrics metrics; private final transient HoodieIndex index; private transient Timer.Context writeContext = null; + private transient Timer.Context compactionTimer; /** * @param jsc @@ -478,13 +481,13 @@ public boolean commit(String commitTime, JavaRDD writeStatuses) { * Commit changes performed at the given commitTime marker */ public boolean commit(String commitTime, JavaRDD writeStatuses, - Optional> extraMetadata) { + Optional> extraMetadata) { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType()); } private boolean commit(String commitTime, JavaRDD writeStatuses, - Optional> extraMetadata, String actionType) { + Optional> extraMetadata, String actionType) { logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible @@ -525,7 +528,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, // Do a inline compaction if enabled if (config.isInlineCompaction()) { metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); - forceCompact(); + forceCompact(extraMetadata); } else { metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); } @@ -685,7 +688,7 @@ public void deleteSavepoint(String savepointTime) { * Delete a compaction request that is pending. * * NOTE - This is an Admin operation. - * With async compaction, this is expected to be called with async compaction and ingestion shutdown. + * With async compaction, this is expected to be called with async compaction and write shutdown. * Otherwise, async compactor could fail with errors * * @param compactionTime - delete the compaction time @@ -729,7 +732,7 @@ public boolean rollbackToSavepoint(String savepointTime) { new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - // Rollback to savepoint is expected to be a manual operation and no concurrent ingestion or compaction is expected + // Rollback to savepoint is expected to be a manual operation and no concurrent write or compaction is expected // to be running. Rollback to savepoint also removes any pending compaction actions that are generated after // savepoint time. Allowing pending compaction to be retained is not safe as those workload could be referencing // file-slices that will be rolled-back as part of this operation @@ -811,8 +814,12 @@ private void rollback(List commits) { // Make sure only the last n commits are being rolled back // If there is a commit in-between or after that is not rolled back, then abort - String lastCommit = commitsToRollback.get(commitsToRollback.size() - 1); - if (!commitTimeline.empty() && !commitTimeline + String lastCommit = null; + if (!commitsToRollback.isEmpty()) { + lastCommit = commitsToRollback.get(commitsToRollback.size() - 1); + } + + if ((lastCommit != null) && !commitTimeline.empty() && !commitTimeline .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { throw new HoodieRollbackException( "Found commits after time :" + lastCommit + ", please rollback greater commits first"); @@ -820,7 +827,7 @@ private void rollback(List commits) { List inflights = inflightCommitTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); - if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) { + if ((lastCommit != null) && !inflights.isEmpty() && (inflights.indexOf(lastCommit) != inflights.size() - 1)) { throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit + ", please rollback greater commits first"); } @@ -940,141 +947,109 @@ public String startCommit() { return commitTime; } - public void startCommitWithTime(String commitTime) { + public void startCommitWithTime(String instantTime) { if (rollbackInFlight) { // Only rollback inflight commit/delta-commits. Do not touch compaction commits rollbackInflightCommits(); } - logger.info("Generate a new commit time " + commitTime); - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + logger.info("Generate a new instant time " + instantTime); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath()); + // if there are pending compactions, their instantTime must not be greater than that of this instant time + metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending -> { + Preconditions.checkArgument( + HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), instantTime, HoodieTimeline.LESSER), + "Latest pending compaction instant time must be earlier " + + "than this instant time. Latest Compaction :" + latestPending + ", Ingesting at " + instantTime); + }); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = table.getMetaClient().getCommitActionType(); - activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); + activeTimeline.createInflight(new HoodieInstant(true, commitActionType, instantTime)); } - /** - * Provides a new commit time for a compaction (commit) operation - */ - public String startCompaction() { - String commitTime = HoodieActiveTimeline.createNewCommitTime(); - logger.info("Generate a new commit time " + commitTime); - startCompactionWithTime(commitTime); - return commitTime; - } /** - * Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, we need to - * explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction - */ - public void startCompactionWithTime(String commitTime) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = HoodieTimeline.COMMIT_ACTION; - activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); - } - - /** - * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed - * asynchronously. Please always use this serially before or after an insert/upsert action. + * Schedules a new compaction instant + * @param extraMetadata + * @return */ - public JavaRDD compact(String commitTime) throws IOException { - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction - writeContext = metrics.getCommitCtx(); - JavaRDD statuses = table.compact(jsc, commitTime); - // Trigger the insert and collect statuses - statuses = statuses.persist(config.getWriteStatusStorageLevel()); - String actionType = HoodieActiveTimeline.COMMIT_ACTION; - commitOnAutoCommit(commitTime, statuses, actionType); - return statuses; + public Optional scheduleCompaction(Optional> extraMetadata) throws IOException { + String instantTime = HoodieActiveTimeline.createNewCommitTime(); + logger.info("Generate a new instant time " + instantTime); + boolean notEmpty = scheduleCompactionAtInstant(instantTime, extraMetadata); + return notEmpty ? Optional.of(instantTime) : Optional.empty(); } /** - * Commit a compaction operation + * Schedules a new compaction instant with passed-in instant time + * @param instantTime Compaction Instant Time + * @param extraMetadata Extra Metadata to be stored */ - public void commitCompaction(String commitTime, JavaRDD writeStatuses, - Optional> extraMetadata) { - String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION; - commit(commitTime, writeStatuses, extraMetadata, commitCompactionActionType); + public boolean scheduleCompactionAtInstant(String instantTime, Optional> extraMetadata) + throws IOException { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true); + // if there are inflight writes, their instantTime must not be less than that of compaction instant time + metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().ifPresent(earliestInflight -> { + Preconditions.checkArgument( + HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), instantTime, HoodieTimeline.GREATER), + "Earliest write inflight instant time must be later " + + "than compaction time. Earliest :" + earliestInflight + ", Compaction scheduled at " + instantTime); + }); + // Committed and pending compaction instants should have strictly lower timestamps + List conflictingInstants = + metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().getInstants().filter(instant -> + HoodieTimeline.compareTimestamps(instant.getTimestamp(), instantTime, + HoodieTimeline.GREATER_OR_EQUAL)).collect(Collectors.toList()); + Preconditions.checkArgument(conflictingInstants.isEmpty(), + "Following instants have timestamps >= compactionInstant. Instants :" + + conflictingInstants); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + HoodieCompactionPlan workload = table.scheduleCompaction(jsc, instantTime); + if (workload != null && (workload.getOperations() != null) && (!workload.getOperations().isEmpty())) { + extraMetadata.ifPresent(workload::setExtraMetadata); + HoodieInstant compactionInstant = + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); + metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, + AvroUtils.serializeCompactionPlan(workload)); + return true; + } + return false; } /** - * Commit a compaction operation + * Performs Compaction for the workload stored in instant-time + * @param compactionInstantTime Compaction Instant Time + * @return + * @throws IOException */ - public void commitCompaction(String commitTime, JavaRDD writeStatuses) { - String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION; - commit(commitTime, writeStatuses, Optional.empty(), commitCompactionActionType); + public JavaRDD compact(String compactionInstantTime) throws IOException { + return compact(compactionInstantTime, config.shouldAutoCommit()); } /** - * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed - * asynchronously. Please always use this serially before or after an insert/upsert action. + * Commit a compaction operation. Allow passing additional meta-data to be stored in commit instant file. */ - private void forceCompact(String compactionCommitTime) throws IOException { - // Create a Hoodie table which encapsulated the commits and files visible + public void commitCompaction(String compactionInstantTime, JavaRDD writeStatuses, + Optional> extraMetadata) throws IOException { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); - // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and - // then use getTableAndInitCtx - Timer.Context writeContext = metrics.getCommitCtx(); - JavaRDD compactedStatuses = table.compact(jsc, compactionCommitTime); - if (!compactedStatuses.isEmpty()) { - HoodieCommitMetadata metadata = commitForceCompaction(compactedStatuses, metaClient, compactionCommitTime); - long durationInMs = metrics.getDurationInMs(writeContext.stop()); - try { - metrics - .updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(), - durationInMs, metadata, HoodieActiveTimeline.COMMIT_ACTION); - } catch (ParseException e) { - throw new HoodieCommitException( - "Commit time is not of valid format.Failed to commit " + config.getBasePath() - + " at time " + compactionCommitTime, e); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( + timeline.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get()); + // Merge extra meta-data passed by user with the one already in inflight compaction + Optional> mergedMetaData = extraMetadata.map(m -> { + Map merged = new HashMap<>(); + Map extraMetaDataFromInstantFile = compactionPlan.getExtraMetadata(); + if (extraMetaDataFromInstantFile != null) { + merged.putAll(extraMetaDataFromInstantFile); } - logger.info("Compacted successfully on commit " + compactionCommitTime); - } else { - logger.info("Compaction did not run for commit " + compactionCommitTime); - } - } - - /** - * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed - * asynchronously. Please always use this serially before or after an insert/upsert action. - */ - private String forceCompact() throws IOException { - String compactionCommitTime = startCompaction(); - forceCompact(compactionCommitTime); - return compactionCommitTime; - } - - private HoodieCommitMetadata commitForceCompaction(JavaRDD writeStatuses, - HoodieTableMetaClient metaClient, String compactionCommitTime) { - List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) - .collect(); - - HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); - for (HoodieWriteStat stat : updateStatusMap) { - metadata.addWriteStat(stat.getPartitionPath(), stat); - } - - logger.info("Compaction finished with result " + metadata); - - logger.info("Committing Compaction " + compactionCommitTime); - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - - try { - activeTimeline.saveAsComplete( - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, compactionCommitTime), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - } catch (IOException e) { - throw new HoodieCompactionException( - "Failed to commit " + metaClient.getBasePath() + " at time " + compactionCommitTime, e); - } - return metadata; + // Overwrite/Merge with the user-passed meta-data + merged.putAll(m); + return Optional.of(merged); + }).orElseGet(() -> Optional.ofNullable(compactionPlan.getExtraMetadata())); + commitCompaction(writeStatuses, table, compactionInstantTime, true, mergedMetaData); } /** @@ -1127,4 +1102,158 @@ private HoodieTable getTableAndInitCtx() { } return table; } + + /** + * Compaction specific private methods + */ + + /** + * Ensures compaction instant is in expected state and performs Compaction for the workload stored in instant-time + * @param compactionInstantTime Compaction Instant Time + * @return + * @throws IOException + */ + private JavaRDD compact(String compactionInstantTime, boolean autoCommit) throws IOException { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + HoodieTimeline pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { + //inflight compaction - Needs to rollback first deleting new parquet files before we run compaction. + rollbackInflightCompaction(inflightInstant, table); + // refresh table + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); + pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline(); + } + + HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + if (pendingCompactionTimeline.containsInstant(instant)) { + return runCompaction(instant, metaClient.getActiveTimeline(), autoCommit); + } else { + throw new IllegalStateException("No Compaction request available at " + compactionInstantTime + + " to run compaction"); + } + } + + /** + * Perform compaction operations as specified in the compaction commit file + * + * @param compactionInstant Compacton Instant time + * @param activeTimeline Active Timeline + * @param autoCommit Commit after compaction + * @return RDD of Write Status + */ + private JavaRDD runCompaction( + HoodieInstant compactionInstant, HoodieActiveTimeline activeTimeline, boolean autoCommit) throws IOException { + HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( + activeTimeline.getInstantAuxiliaryDetails(compactionInstant).get()); + // Mark instant as compaction inflight + activeTimeline.transitionCompactionRequestedToInflight(compactionInstant); + compactionTimer = metrics.getCompactionCtx(); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + JavaRDD statuses = table.compact(jsc, compactionInstant.getTimestamp(), compactionPlan); + // Force compaction action + statuses.persist(config.getWriteStatusStorageLevel()); + // pass extra-metada so that it gets stored in commit file automatically + commitCompaction(statuses, table, compactionInstant.getTimestamp(), autoCommit, + Optional.ofNullable(compactionPlan.getExtraMetadata())); + return statuses; + } + + /** + * Commit Compaction and track metrics + * + * @param compactedStatuses Compaction Write status + * @param table Hoodie Table + * @param compactionCommitTime Compaction Commit Time + * @param autoCommit Auto Commit + * @param extraMetadata Extra Metadata to store + */ + protected void commitCompaction(JavaRDD compactedStatuses, HoodieTable table, + String compactionCommitTime, boolean autoCommit, Optional> extraMetadata) { + if (autoCommit) { + HoodieCommitMetadata metadata = + doCompactionCommit(compactedStatuses, table.getMetaClient(), compactionCommitTime, extraMetadata); + if (compactionTimer != null) { + long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); + try { + metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException( + "Commit time is not of valid format.Failed to commit compaction " + config.getBasePath() + + " at time " + compactionCommitTime, e); + } + } + logger.info("Compacted successfully on commit " + compactionCommitTime); + } else { + logger.info("Compaction did not run for commit " + compactionCommitTime); + } + } + + /** + * Rollback partial compactions + * @param inflightInstant Inflight Compaction Instant + * @param table Hoodie Table + */ + private void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) throws IOException { + table.rollback(jsc, ImmutableList.copyOf(new String[] { inflightInstant.getTimestamp() })); + // Revert instant state file + table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant); + } + + private HoodieCommitMetadata doCompactionCommit(JavaRDD writeStatuses, + HoodieTableMetaClient metaClient, String compactionCommitTime, Optional> extraMetadata) { + List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) + .collect(); + + HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); + for (HoodieWriteStat stat : updateStatusMap) { + metadata.addWriteStat(stat.getPartitionPath(), stat); + } + + // Copy extraMetadata + extraMetadata.ifPresent(m -> { + m.entrySet().stream().forEach(e -> { + metadata.addMetadata(e.getKey(), e.getValue()); + }); + }); + + logger.info("Compaction finished with result " + metadata); + + logger.info("Committing Compaction " + compactionCommitTime); + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + + try { + activeTimeline.transitionCompactionInflightToComplete( + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime), + Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieCompactionException( + "Failed to commit " + metaClient.getBasePath() + " at time " + compactionCommitTime, e); + } + return metadata; + } + + /** + * Performs a compaction operation on a dataset. WARNING: Compaction operation cannot be executed + * asynchronously. Please always use this serially before or after an insert/upsert action. + */ + private Optional forceCompact(Optional> extraMetadata) throws IOException { + Optional compactionInstantTimeOpt = scheduleCompaction(extraMetadata); + compactionInstantTimeOpt.ifPresent(compactionInstantTime -> { + try { + compact(compactionInstantTime); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }); + return compactionInstantTimeOpt; + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java deleted file mode 100644 index ac0b5b331e61a..0000000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactionInstantWithPlan.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.io.compact; - -import com.uber.hoodie.avro.model.HoodieCompactionPlan; - -/** - * Contains Hoodie Compaction instant along with workload - */ -public class HoodieCompactionInstantWithPlan { - - private final String compactionInstantTime; - private final HoodieCompactionPlan compactionPlan; - - public HoodieCompactionInstantWithPlan(String compactionInstantTime, - HoodieCompactionPlan compactionPlan) { - this.compactionInstantTime = compactionInstantTime; - this.compactionPlan = compactionPlan; - } - - public String getCompactionInstantTime() { - return compactionInstantTime; - } - - public HoodieCompactionPlan getCompactionPlan() { - return compactionPlan; - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index ad1c18e2fc3e0..27f04c426c48e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -22,6 +22,7 @@ import com.uber.hoodie.table.HoodieTable; import java.io.IOException; import java.io.Serializable; +import java.util.Set; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -30,15 +31,6 @@ */ public interface HoodieCompactor extends Serializable { - /** - * Compact the delta files with the data files - * - * @deprecated : Will be removed in next PR - */ - @Deprecated - JavaRDD compact(JavaSparkContext jsc, final HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws Exception; - /** * Generate a new compaction plan for scheduling * @@ -50,7 +42,8 @@ JavaRDD compact(JavaSparkContext jsc, final HoodieWriteConfig confi * @throws IOException when encountering errors */ HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, - HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) + HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime, + Set fileIdsWithPendingCompactions) throws IOException; /** @@ -58,5 +51,5 @@ HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, */ JavaRDD compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config, - String compactionCommitTime) throws IOException; + String compactionInstantTime) throws IOException; } \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 90a24b6d98017..ef97abff85e1a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -45,6 +45,7 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.avro.Schema; @@ -73,31 +74,22 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { // Accumulator to keep track of total log file slices for a dataset private AccumulatorV2 totalFileSlices; - @Override - public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws IOException { - - HoodieCompactionPlan compactionPlan = generateCompactionPlan(jsc, hoodieTable, config, - compactionCommitTime); - List operations = compactionPlan.getOperations(); - if ((operations == null) || (operations.isEmpty())) { - return jsc.emptyRDD(); - } - return compact(jsc, compactionPlan, hoodieTable, config, compactionCommitTime); - } - @Override public JavaRDD compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config, - String compactionCommitTime) throws IOException { + String compactionInstantTime) throws IOException { + if (compactionPlan == null || (compactionPlan.getOperations() == null) + || (compactionPlan.getOperations().isEmpty())) { + return jsc.emptyRDD(); + } HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); // Compacting is very similar to applying updates to existing file HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); - List operations = compactionPlan.getOperations().stream() - .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); + List operations = compactionPlan.getOperations().stream().map( + CompactionOperation::convertFromAvroRecordInstance).collect(toList()); log.info("Compactor compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) - .map(s -> compact(table, metaClient, config, s, compactionCommitTime)) + .map(s -> compact(table, metaClient, config, s, compactionInstantTime)) .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); } @@ -164,8 +156,8 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, @Override public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, - HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) - throws IOException { + HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime, + Set fileIdsWithPendingCompactions) throws IOException { totalLogFiles = new LongAccumulator(); totalFileSlices = new LongAccumulator(); @@ -191,7 +183,9 @@ public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) .flatMap((FlatMapFunction) partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath).map( + .getLatestFileSlices(partitionPath) + .filter(slice -> !fileIdsWithPendingCompactions.contains(slice.getFileId())) + .map( s -> { List logFiles = s.getLogFiles().sorted(HoodieLogFile .getBaseInstantAndLogVersionComparator().reversed()).collect(Collectors.toList()); @@ -211,13 +205,16 @@ public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, log.info("Total number of latest files slices " + totalFileSlices.value()); log.info("Total number of log files " + totalLogFiles.value()); log.info("Total number of file slices " + totalFileSlices.value()); - // Filter the compactions with the passed in filter. This lets us choose most effective // compactions only - // TODO: In subsequent PRs, pending Compaction plans will be wired in. Strategy can look at pending compaction - // plans to schedule next compaction plan HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations, CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); + Preconditions.checkArgument(compactionPlan.getOperations().stream() + .filter(op -> fileIdsWithPendingCompactions.contains(op.getFileId())).count() == 0, + "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. " + + "Please fix your strategy implementation." + + "FileIdsWithPendingCompactions :" + fileIdsWithPendingCompactions + + ", Selected workload :" + compactionPlan); if (compactionPlan.getOperations().isEmpty()) { log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index 901bd236e8bde..ead65d70d27bd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -38,6 +38,7 @@ public class HoodieMetrics { public String commitTimerName = null; public String deltaCommitTimerName = null; public String finalizeTimerName = null; + public String compactionTimerName = null; private HoodieWriteConfig config = null; private String tableName = null; private Timer rollbackTimer = null; @@ -45,6 +46,7 @@ public class HoodieMetrics { private Timer commitTimer = null; private Timer deltaCommitTimer = null; private Timer finalizeTimer = null; + private Timer compactionTimer = null; public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.config = config; @@ -56,6 +58,7 @@ public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION); this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION); this.finalizeTimerName = getMetricsName("timer", "finalize"); + this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION); } } @@ -70,6 +73,13 @@ public Timer.Context getRollbackCtx() { return rollbackTimer == null ? null : rollbackTimer.time(); } + public Timer.Context getCompactionCtx() { + if (config.isMetricsOn() && compactionTimer == null) { + compactionTimer = createTimer(commitTimerName); + } + return compactionTimer == null ? null : compactionTimer.time(); + } + public Timer.Context getCleanCtx() { if (config.isMetricsOn() && cleanTimer == null) { cleanTimer = createTimer(cleanTimerName); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index b05ba602f0e50..de9d840007f4c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -167,11 +167,6 @@ public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String comm throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); } - @Override - public JavaRDD compact(JavaSparkContext jsc, String commitTime) { - throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); - } - @Override public JavaRDD compact(JavaSparkContext jsc, String compactionInstantTime, HoodieCompactionPlan compactionPlan) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index ef2ed7f0e4b1a..3ead5904c95eb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; @@ -51,6 +52,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -148,22 +150,14 @@ public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String inst logger.info("Compacting merge on read table " + config.getBasePath()); HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); try { - return compactor.generateCompactionPlan(jsc, this, config, instantTime); + return compactor.generateCompactionPlan(jsc, this, config, instantTime, + new HashSet<>(((HoodieTableFileSystemView)getRTFileSystemView()) + .getFileIdToPendingCompaction().keySet())); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); } } - @Override - public JavaRDD compact(JavaSparkContext jsc, String compactionCommitTime) { - HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); - try { - return compactor.compact(jsc, config, this, compactionCommitTime); - } catch (IOException e) { - throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e); - } - } - @Override public JavaRDD compact(JavaSparkContext jsc, String compactionInstantTime, HoodieCompactionPlan compactionPlan) { @@ -185,7 +179,7 @@ public List rollback(JavaSparkContext jsc, List comm } Map commitsAndCompactions = this.getActiveTimeline() .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION)).getInstants() + HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants() .filter(i -> commits.contains(i.getTimestamp())) .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); @@ -219,6 +213,7 @@ public List rollback(JavaSparkContext jsc, List comm switch (instant.getAction()) { case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.COMPACTION_ACTION: try { Map results = super .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 3e83ca1a3fa36..4ef33f9b1767e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -223,13 +223,6 @@ public abstract Iterator> handleInsertPartition(String commitT /** * Run Compaction on the table. Compaction arranges the data so that it is optimized for data * access - * @deprecated Will be replaced with newer APIs - */ - @Deprecated - public abstract JavaRDD compact(JavaSparkContext jsc, String commitTime); - - /** - * Run Compaction on the table. Compaction arranges the data so that it is optimized for data access * * @param jsc Spark Context * @param compactionInstantTime Instant Time diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java new file mode 100644 index 0000000000000..f1a1b8ea8e8a9 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -0,0 +1,508 @@ +/* + * 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; + +import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.HoodieClientTestUtils; +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.model.HoodieTestUtils; +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.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieStorageConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.FileStatus; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Test; + +/** + * Test Cases for Async Compaction and Ingestion interaction + */ +public class TestAsyncCompaction extends TestHoodieClientBase { + + private HoodieWriteConfig getConfig(Boolean autoCommit) { + return getConfigBuilder(autoCommit).build(); + } + + private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false) + .withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + } + + @Test + public void testRollbackInflightIngestionWithPendingCompaction() throws Exception { + // Rollback inflight ingestion when there is pending compaction + HoodieWriteConfig cfg = getConfig(false); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String compactionInstantTime = "005"; + String inflightInstantTime = "006"; + String nextInflightInstantTime = "007"; + + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + // Schedule compaction but do not run them + scheduleCompaction(compactionInstantTime, client, cfg); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true); + + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieInstant pendingCompactionInstant = + metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); + assertTrue("Pending Compaction instant has expected instant time", + pendingCompactionInstant.getTimestamp().equals(compactionInstantTime)); + HoodieInstant inflightInstant = + metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get(); + assertTrue("inflight instant has expected instant time", + inflightInstant.getTimestamp().equals(inflightInstantTime)); + + //This should rollback + client.startCommitWithTime(nextInflightInstantTime); + + //Validate + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + inflightInstant = + metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get(); + assertTrue("inflight instant has expected instant time", + inflightInstant.getTimestamp().equals(nextInflightInstantTime)); + assertTrue("Expect only one inflight instant", + metaClient.getActiveTimeline().filterInflightsExcludingCompaction().getInstants().count() == 1); + //Expect pending Compaction to be present + pendingCompactionInstant = + metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); + assertTrue("Pending Compaction instant has expected instant time", + pendingCompactionInstant.getTimestamp().equals(compactionInstantTime)); + } + + @Test + public void testInflightCompaction() throws Exception { + // There is inflight compaction. Subsequent compaction run must work correctly + HoodieWriteConfig cfg = getConfig(true); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String compactionInstantTime = "005"; + String thirdInstantTime = "006"; + String fourthInstantTime = "007"; + + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + // Schedule and mark compaction instant as inflight + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + scheduleCompaction(compactionInstantTime, client, cfg); + moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg); + + // Complete ingestions + runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime), + records, cfg, false, Arrays.asList(compactionInstantTime)); + + // execute inflight compaction + executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true); + } + + @Test + public void testScheduleIngestionBeforePendingCompaction() throws Exception { + // Case: Failure case. Latest pending compaction instant time must be earlier than this instant time + HoodieWriteConfig cfg = getConfig(false); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String failedInstantTime = "005"; + String compactionInstantTime = "006"; + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + // Schedule compaction but do not run them + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + scheduleCompaction(compactionInstantTime, client, cfg); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieInstant pendingCompactionInstant = + metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); + assertTrue("Pending Compaction instant has expected instant time", + pendingCompactionInstant.getTimestamp().equals(compactionInstantTime)); + + boolean gotException = false; + try { + runNextDeltaCommits(client, Arrays.asList(failedInstantTime), + records, cfg, false, Arrays.asList(compactionInstantTime)); + } catch (IllegalArgumentException iex) { + // Latest pending compaction instant time must be earlier than this instant time. Should fail here + gotException = true; + } + assertTrue("Latest pending compaction instant time must be earlier than this instant time", gotException); + } + + @Test + public void testScheduleCompactionAfterPendingIngestion() throws Exception { + // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time + + HoodieWriteConfig cfg = getConfig(false); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String inflightInstantTime = "005"; + String compactionInstantTime = "006"; + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true); + + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieInstant inflightInstant = + metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get(); + assertTrue("inflight instant has expected instant time", + inflightInstant.getTimestamp().equals(inflightInstantTime)); + + boolean gotException = false; + try { + // Schedule compaction but do not run them + scheduleCompaction(compactionInstantTime, client, cfg); + } catch (IllegalArgumentException iex) { + // Earliest ingestion inflight instant time must be later than compaction time. Should fail here + gotException = true; + } + assertTrue("Earliest ingestion inflight instant time must be later than compaction time", gotException); + } + + @Test + public void testScheduleCompactionWithOlderOrSameTimestamp() throws Exception { + // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time + + HoodieWriteConfig cfg = getConfig(false); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String compactionInstantTime = "002"; + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + boolean gotException = false; + try { + // Schedule compaction but do not run them + scheduleCompaction(compactionInstantTime, client, cfg); + } catch (IllegalArgumentException iex) { + gotException = true; + } + assertTrue("Compaction Instant to be scheduled cannot have older timestamp", gotException); + + // Schedule with timestamp same as that of committed instant + gotException = false; + String dupCompactionInstantTime = secondInstantTime; + try { + // Schedule compaction but do not run them + scheduleCompaction(dupCompactionInstantTime, client, cfg); + } catch (IllegalArgumentException iex) { + gotException = true; + } + assertTrue("Compaction Instant to be scheduled cannot have same timestamp as committed instant", + gotException); + + compactionInstantTime = "006"; + scheduleCompaction(compactionInstantTime, client, cfg); + gotException = false; + try { + // Schedule compaction with the same times as a pending compaction + scheduleCompaction(dupCompactionInstantTime, client, cfg); + } catch (IllegalArgumentException iex) { + gotException = true; + } + assertTrue("Compaction Instant to be scheduled cannot have same timestamp as a pending compaction", + gotException); + } + + @Test + public void testCompactionAfterTwoDeltaCommits() throws Exception { + // No Delta Commits after compaction request + HoodieWriteConfig cfg = getConfig(true); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String compactionInstantTime = "005"; + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false); + } + + @Test + public void testInterleavedCompaction() throws Exception { + //Case: Two delta commits before and after compaction schedule + HoodieWriteConfig cfg = getConfig(true); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String compactionInstantTime = "005"; + String thirdInstantTime = "006"; + String fourthInstantTime = "007"; + + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + scheduleCompaction(compactionInstantTime, client, cfg); + + runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime), + records, cfg, false, Arrays.asList(compactionInstantTime)); + executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true); + } + + /** + * HELPER METHODS FOR TESTING + **/ + + private void validateDeltaCommit(String latestDeltaCommit, + final Map> fileIdToCompactionOperation, + HoodieWriteConfig cfg) throws IOException { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + List fileSliceList = getCurrentLatestFileSlices(table, cfg); + fileSliceList.forEach(fileSlice -> { + Pair opPair = fileIdToCompactionOperation.get(fileSlice.getFileId()); + if (opPair != null) { + System.out.println("FileSlice :" + fileSlice); + assertTrue("Expect baseInstant to match compaction Instant", + fileSlice.getBaseInstantTime().equals(opPair.getKey())); + assertTrue("Expect atleast one log file to be present where the latest delta commit was written", + fileSlice.getLogFiles().count() > 0); + assertFalse("Expect no data-file to be present", fileSlice.getDataFile().isPresent()); + } else { + assertTrue("Expect baseInstant to be less than or equal to latestDeltaCommit", + fileSlice.getBaseInstantTime().compareTo(latestDeltaCommit) <= 0); + } + }); + } + + private List runNextDeltaCommits(HoodieWriteClient client, List deltaInstants, + List records, HoodieWriteConfig cfg, boolean insertFirst, + List expPendingCompactionInstants) throws Exception { + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + List> pendingCompactions = + CompactionUtils.getAllPendingCompactionPlans(metaClient); + List gotPendingCompactionInstants = + pendingCompactions.stream().map(pc -> pc.getKey().getTimestamp()).sorted().collect(Collectors.toList()); + assertEquals(expPendingCompactionInstants, gotPendingCompactionInstants); + + Map> fileIdToCompactionOperation = + CompactionUtils.getAllPendingCompactionOperations(metaClient); + + if (insertFirst) { + // Use first instant for inserting records + String firstInstant = deltaInstants.get(0); + deltaInstants = deltaInstants.subList(1, deltaInstants.size()); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.startCommitWithTime(firstInstant); + JavaRDD statuses = client.upsert(writeRecords, firstInstant); + List statusList = statuses.collect(); + + if (!cfg.shouldAutoCommit()) { + client.commit(firstInstant, statuses); + } + TestHoodieClientBase.assertNoWriteErrors(statusList); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + List dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg); + assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead.stream().findAny().isPresent()); + validateDeltaCommit(firstInstant, fileIdToCompactionOperation, cfg); + } + + int numRecords = records.size(); + for (String instantTime : deltaInstants) { + records = dataGen.generateUpdates(instantTime, numRecords); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + createNextDeltaCommit(instantTime, records, client, metaClient, cfg, false); + validateDeltaCommit(instantTime, fileIdToCompactionOperation, cfg); + } + return records; + } + + private void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteClient client, + HoodieWriteConfig cfg) throws IOException { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan( + metaClient.getActiveTimeline().getInstantAuxiliaryDetails(compactionInstant).get()); + metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant); + HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants() + .filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get(); + assertTrue("Instant must be marked inflight", instant.isInflight()); + } + + private void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg) + throws IOException { + client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get(); + assertEquals("Last compaction instant must be the one set", + instant.getTimestamp(), compactionInstantTime); + } + + private void scheduleAndExecuteCompaction(String compactionInstantTime, + HoodieWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs, + boolean hasDeltaCommitAfterPendingCompaction) throws IOException { + scheduleCompaction(compactionInstantTime, client, cfg); + executeCompaction(compactionInstantTime, client, table, cfg, expectedNumRecs, hasDeltaCommitAfterPendingCompaction); + } + + private void executeCompaction(String compactionInstantTime, + HoodieWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs, + boolean hasDeltaCommitAfterPendingCompaction) throws IOException { + + client.compact(compactionInstantTime); + List fileSliceList = getCurrentLatestFileSlices(table, cfg); + assertTrue("Ensure latest file-slices are not empty", fileSliceList.stream().findAny().isPresent()); + assertFalse("Verify all file-slices have base-instant same as compaction instant", + fileSliceList.stream().filter(fs -> !fs.getBaseInstantTime().equals(compactionInstantTime)) + .findAny().isPresent()); + assertFalse("Verify all file-slices have data-files", + fileSliceList.stream().filter(fs -> !fs.getDataFile().isPresent()).findAny().isPresent()); + + if (hasDeltaCommitAfterPendingCompaction) { + assertFalse("Verify all file-slices have atleast one log-file", + fileSliceList.stream().filter(fs -> fs.getLogFiles().count() == 0).findAny().isPresent()); + } else { + assertFalse("Verify all file-slices have no log-files", + fileSliceList.stream().filter(fs -> fs.getLogFiles().count() > 0).findAny().isPresent()); + } + + // verify that there is a commit + table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg, jsc); + HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants(); + String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); + assertEquals("Expect compaction instant time to be the latest commit time", + latestCompactionCommitTime, compactionInstantTime); + assertEquals("Must contain expected records", expectedNumRecs, + HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count()); + + } + + private List createNextDeltaCommit(String instantTime, List records, + HoodieWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) { + JavaRDD writeRecords = jsc.parallelize(records, 1); + + client.startCommitWithTime(instantTime); + + JavaRDD statuses = client.upsert(writeRecords, instantTime); + List statusList = statuses.collect(); + TestHoodieClientBase.assertNoWriteErrors(statusList); + if (!cfg.shouldAutoCommit() && !skipCommit) { + client.commit(instantTime, statuses); + } + + Optional deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline() + .filterCompletedInstants().lastInstant(); + if (skipCommit && !cfg.shouldAutoCommit()) { + assertTrue("Delta commit should not be latest instant", + deltaCommit.get().getTimestamp().compareTo(instantTime) < 0); + } else { + assertTrue(deltaCommit.isPresent()); + assertEquals("Delta commit should be latest instant", instantTime, deltaCommit.get().getTimestamp()); + } + return statusList; + } + + private List getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException { + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(table.getMetaClient().getFs(), cfg.getBasePath()); + HoodieTableFileSystemView + view = new HoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitTimeline(), allFiles); + List dataFilesToRead = view.getLatestDataFiles().collect(Collectors.toList()); + return dataFilesToRead; + } + + private List getCurrentLatestFileSlices(HoodieTable table, HoodieWriteConfig cfg) throws IOException { + HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(), + table.getMetaClient().getActiveTimeline().reload().getCommitsAndCompactionTimeline()); + List fileSliceList = + Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS).stream().flatMap(partition -> + view.getLatestFileSlices(partition)).collect(Collectors.toList()); + return fileSliceList; + } + + protected HoodieTableType getTableType() { + return HoodieTableType.MERGE_ON_READ; + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java index 0840f7ddc0323..5545992faa305 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -26,6 +26,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -48,6 +49,7 @@ import java.util.Optional; import java.util.Set; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -71,6 +73,7 @@ public class TestHoodieClientBase implements Serializable { public void init() throws IOException { // Initialize a local spark env jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient")); + jsc.setLogLevel("ERROR"); //SQLContext stuff sqlContext = new SQLContext(jsc); @@ -80,7 +83,14 @@ public void init() throws IOException { folder.create(); basePath = folder.getRoot().getAbsolutePath(); fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); - HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + if (fs instanceof LocalFileSystem) { + LocalFileSystem lfs = (LocalFileSystem) fs; + // With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream + // This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open + // So, for the tests, we enforce checksum verification to circumvent the problem + lfs.setVerifyChecksum(true); + } + HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, getTableType()); dataGen = new HoodieTestDataGenerator(); } @@ -112,7 +122,7 @@ HoodieWriteConfig.Builder getConfigBuilder() { * * @param statuses List of Write Status */ - void assertNoWriteErrors(List statuses) { + static void assertNoWriteErrors(List statuses) { // Verify there are no errors for (WriteStatus status : statuses) { assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors()); @@ -433,4 +443,8 @@ interface Function3 { R apply(T1 v1, T2 v2, T3 v3) throws IOException; } + + protected HoodieTableType getTableType() { + return HoodieTableType.COPY_ON_WRITE; + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index 8fd0d31dca35b..e1a6d66dcb600 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -35,6 +35,7 @@ import com.uber.hoodie.config.HoodieMemoryConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.bloom.HoodieBloomIndex; import com.uber.hoodie.io.compact.HoodieCompactor; @@ -90,7 +91,9 @@ public void clean() { } private HoodieWriteConfig getConfig() { - return getConfigBuilder().build(); + return getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .build(); } private HoodieWriteConfig.Builder getConfigBuilder() { @@ -103,12 +106,14 @@ private HoodieWriteConfig.Builder getConfigBuilder() { .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = HoodieNotSupportedException.class) public void testCompactionOnCopyOnWriteFail() throws Exception { HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); - compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime(); + table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime)); } @Test @@ -123,8 +128,9 @@ public void testCompactionEmpty() throws Exception { JavaRDD recordsRDD = jsc.parallelize(records, 1); writeClient.insert(recordsRDD, newCommitTime).collect(); - JavaRDD result = compactor - .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime(); + JavaRDD result = + table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime)); assertTrue("If there is nothing to compact, result will be empty", result.isEmpty()); } @@ -171,8 +177,9 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config, jsc); - JavaRDD result = compactor - .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime(); + JavaRDD result = + table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime)); // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 980b3d07746ca..fbcdac2580160 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -41,6 +41,7 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; @@ -202,7 +203,7 @@ public void testSimpleInsertAndUpdate() throws Exception { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - String compactionCommitTime = client.startCompaction(); + String compactionCommitTime = client.scheduleCompaction(Optional.empty()).get().toString(); client.compact(compactionCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); @@ -522,14 +523,15 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - String compactionCommit = client.startCompaction(); - JavaRDD writeStatus = client.compact(compactionCommit); - client.commitCompaction(compactionCommit, writeStatus); + String compactionInstantTime = client.scheduleCompaction(Optional.empty()).get().toString(); + JavaRDD ws = client.compact(compactionInstantTime); + client.commitCompaction(compactionInstantTime, ws, Optional.empty()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); + List dataFiles2 = roView.getLatestDataFiles().collect(Collectors.toList()); final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() .getTimestamp(); @@ -679,12 +681,13 @@ public void testLogFileCountsAfterCompaction() throws Exception { } } - // Do a compaction - metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); + // Mark 2nd delta-instant as completed + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Optional.empty()); - String commitTime = writeClient.startCompaction(); - JavaRDD result = writeClient.compact(commitTime); + // Do a compaction + String compactionInstantTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); + JavaRDD result = writeClient.compact(compactionInstantTime); // Verify that recently written compacted data file has no log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -741,9 +744,9 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception Assert.assertTrue(totalUpsertTime > 0); // Do a compaction - String commitTime = writeClient.startCompaction(); - statuses = writeClient.compact(commitTime); - writeClient.commitCompaction(commitTime, statuses); + String compactionInstantTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); + statuses = writeClient.compact(compactionInstantTime); + writeClient.commitCompaction(compactionInstantTime, statuses, Optional.empty()); // total time taken for scanning log files should be greater than 0 long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime()) .reduce((a, b) -> a + b).longValue(); @@ -782,11 +785,11 @@ public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception { Assert.assertTrue(numLogFiles > 0); // Do a compaction - String commitTime = writeClient.startCompaction(); + String commitTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); statuses = writeClient.compact(commitTime); Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); Assert.assertEquals(statuses.count(), numLogFiles); - writeClient.commitCompaction(commitTime, statuses); + writeClient.commitCompaction(commitTime, statuses, Optional.empty()); } @Test @@ -824,6 +827,8 @@ public void testInsertsGeneratedIntoLogFilesRollback() throws Exception { writeClient.commit(newCommitTime, statuses); // rollback a successful commit + // Sleep for small interval to force a new rollback start time. + Thread.sleep(5); writeClient.rollback(newCommitTime); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); @@ -875,12 +880,12 @@ public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exc Assert.assertTrue(numLogFiles > 0); // Do a compaction - newCommitTime = writeClient.startCompaction(); + newCommitTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); statuses = writeClient.compact(newCommitTime); // Ensure all log files have been compacted into parquet files Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); Assert.assertEquals(statuses.count(), numLogFiles); - writeClient.commitCompaction(newCommitTime, statuses); + writeClient.commitCompaction(newCommitTime, statuses, Optional.empty()); // Trigger a rollback of compaction writeClient.rollback(newCommitTime); for (String partitionPath : dataGen.getPartitionPaths()) { diff --git a/hoodie-client/src/test/resources/log4j-surefire.properties b/hoodie-client/src/test/resources/log4j-surefire.properties index daf8d28c13c5f..23ded09c3f49a 100644 --- a/hoodie-client/src/test/resources/log4j-surefire.properties +++ b/hoodie-client/src/test/resources/log4j-surefire.properties @@ -14,12 +14,8 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=INFO -log4j.category.com.uber.hoodie.common.utils=WARN -log4j.category.com.uber.hoodie.io=WARN -log4j.category.com.uber.hoodie.common=WARN -log4j.category.com.uber.hoodie.table.log=WARN -log4j.category.org.apache.parquet.hadoop=WARN +log4j.category.com.uber=WARN +log4j.category.org.apache.parquet.hadoop=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-common/src/test/resources/log4j-surefire.properties b/hoodie-common/src/test/resources/log4j-surefire.properties index ea3e935455861..23ded09c3f49a 100644 --- a/hoodie-common/src/test/resources/log4j-surefire.properties +++ b/hoodie-common/src/test/resources/log4j-surefire.properties @@ -14,10 +14,8 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=INFO -log4j.category.com.uber.hoodie.table.log=WARN -log4j.category.com.uber.hoodie.common.util=WARN -log4j.category.org.apache.parquet.hadoop=WARN +log4j.category.com.uber=WARN +log4j.category.org.apache.parquet.hadoop=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties b/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties index 3613e7d120a53..23ded09c3f49a 100644 --- a/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties +++ b/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties @@ -14,8 +14,8 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=INFO -log4j.category.org.apache.parquet.hadoop=WARN +log4j.category.com.uber=WARN +log4j.category.org.apache.parquet.hadoop=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-hive/src/test/resources/log4j-surefire.properties b/hoodie-hive/src/test/resources/log4j-surefire.properties index 8027f04d8375d..1d7678513d15f 100644 --- a/hoodie-hive/src/test/resources/log4j-surefire.properties +++ b/hoodie-hive/src/test/resources/log4j-surefire.properties @@ -14,9 +14,9 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=INFO -log4j.category.org.apache.parquet.hadoop=WARN -log4j.category.parquet.hadoop=WARN +log4j.category.com.uber=WARN +log4j.category.org.apache.parquet.hadoop=ERROR +log4j.category.parquet.hadoop=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-spark/src/test/resources/log4j-surefire.properties b/hoodie-spark/src/test/resources/log4j-surefire.properties index daf8d28c13c5f..23ded09c3f49a 100644 --- a/hoodie-spark/src/test/resources/log4j-surefire.properties +++ b/hoodie-spark/src/test/resources/log4j-surefire.properties @@ -14,12 +14,8 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=INFO -log4j.category.com.uber.hoodie.common.utils=WARN -log4j.category.com.uber.hoodie.io=WARN -log4j.category.com.uber.hoodie.common=WARN -log4j.category.com.uber.hoodie.table.log=WARN -log4j.category.org.apache.parquet.hadoop=WARN +log4j.category.com.uber=WARN +log4j.category.org.apache.parquet.hadoop=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. From 89ecc712f034e012c9668c937ab50a6d060bfd46 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Sat, 26 May 2018 14:08:29 -0700 Subject: [PATCH 073/374] Ensure Compaction Operation compacts the data file as defined in the workload --- .../com/uber/hoodie/io/HoodieMergeHandle.java | 16 +++++++---- .../compact/HoodieRealtimeTableCompactor.java | 6 +++- .../hoodie/table/HoodieCopyOnWriteTable.java | 28 +++++++++---------- .../test/resources/log4j-surefire.properties | 8 ++++-- .../hoodie/common/table/HoodieTimeline.java | 2 ++ .../common/table/TableFileSystemView.java | 6 ++++ .../table/view/HoodieTableFileSystemView.java | 16 +++++++++++ .../test/resources/log4j-surefire.properties | 6 ++-- .../test/resources/log4j-surefire.properties | 4 +-- .../test/resources/log4j-surefire.properties | 6 ++-- .../test/resources/log4j-surefire.properties | 8 ++++-- 11 files changed, 74 insertions(+), 32 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index fbc2b2bf5b8e8..2bd5d8668b87b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -17,6 +17,7 @@ package com.uber.hoodie.io; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; @@ -71,30 +72,33 @@ public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTabl Iterator> recordItr, String fileId) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); - init(fileId, init(fileId, recordItr)); + String partitionPath = init(fileId, recordItr); + init(fileId, partitionPath, + fileSystemView.getLatestDataFiles(partitionPath) + .filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst()); } public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, - Map> keyToNewRecords, String fileId) { + Map> keyToNewRecords, String fileId, Optional dataFileToBeMerged) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); this.keyToNewRecords = keyToNewRecords; init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()) - .getPartitionPath()); + .getPartitionPath(), dataFileToBeMerged); } /** * Extract old file path, initialize StorageWriter and WriteStatus */ - private void init(String fileId, String partitionPath) { + private void init(String fileId, String partitionPath, Optional dataFileToBeMerged) { this.writtenRecordKeys = new HashSet<>(); WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieWriteStat()); this.writeStatus = writeStatus; try { - String latestValidFilePath = fileSystemView.getLatestDataFiles(partitionPath) - .filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst().get().getFileName(); + //TODO: dataFileToBeMerged must be optional. Will be fixed by Nishith's changes to support insert to log-files + String latestValidFilePath = dataFileToBeMerged.get().getFileName(); writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index ef97abff85e1a..3659437c6abce 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -124,13 +124,17 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, return Lists.newArrayList(); } + Optional oldDataFileOpt = hoodieCopyOnWriteTable.getROFileSystemView() + .getLatestDataFilesOn(operation.getPartitionPath(), operation.getBaseInstantTime()) + .filter(df -> df.getFileId().equals(operation.getFileId())).findFirst(); + // Compacting is very similar to applying updates to existing file Iterator> result; // If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a // new base parquet file. if (operation.getDataFilePath().isPresent()) { result = hoodieCopyOnWriteTable - .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); + .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(), oldDataFileOpt); } else { result = hoodieCopyOnWriteTable .handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index de9d840007f4c..fa180dba4611b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -173,26 +173,26 @@ public JavaRDD compact(JavaSparkContext jsc, String compactionInsta throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); } - public Iterator> handleUpdate(String commitTime, String fileLoc, + public Iterator> handleUpdate(String commitTime, String fileId, Iterator> recordItr) throws IOException { // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr); - return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, recordItr); + return handleUpdateInternal(upsertHandle, commitTime, fileId); } - public Iterator> handleUpdate(String commitTime, String fileLoc, - Map> keyToNewRecords) throws IOException { + public Iterator> handleUpdate(String commitTime, String fileId, + Map> keyToNewRecords, Optional dataFileOpt) throws IOException { // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords); - return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, dataFileOpt); + return handleUpdateInternal(upsertHandle, commitTime, fileId); } protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, - String commitTime, String fileLoc) + String commitTime, String fileId) throws IOException { if (upsertHandle.getOldFilePath() == null) { throw new HoodieUpsertException( - "Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc); + "Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId); } else { AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) @@ -222,14 +222,14 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups .iterator(); } - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId, Iterator> recordItr) { - return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); + return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileId); } - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, - Map> keyToNewRecords) { - return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc); + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId, + Map> keyToNewRecords, Optional dataFileToBeMerged) { + return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged); } public Iterator> handleInsert(String commitTime, diff --git a/hoodie-client/src/test/resources/log4j-surefire.properties b/hoodie-client/src/test/resources/log4j-surefire.properties index 23ded09c3f49a..daf8d28c13c5f 100644 --- a/hoodie-client/src/test/resources/log4j-surefire.properties +++ b/hoodie-client/src/test/resources/log4j-surefire.properties @@ -14,8 +14,12 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=WARN -log4j.category.org.apache.parquet.hadoop=ERROR +log4j.category.com.uber=INFO +log4j.category.com.uber.hoodie.common.utils=WARN +log4j.category.com.uber.hoodie.io=WARN +log4j.category.com.uber.hoodie.common=WARN +log4j.category.com.uber.hoodie.table.log=WARN +log4j.category.org.apache.parquet.hadoop=WARN # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java index 790ba770e0488..7adc9a1641ee1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java @@ -174,6 +174,8 @@ public interface HoodieTimeline extends Serializable { /** * Helper methods to compare instants **/ + BiPredicate EQUAL = + (commit1, commit2) -> commit1.compareTo(commit2) == 0; BiPredicate GREATER_OR_EQUAL = (commit1, commit2) -> commit1.compareTo(commit2) >= 0; BiPredicate GREATER = (commit1, commit2) -> commit1.compareTo(commit2) > 0; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java index 637f6e874ef0d..d0f2d87e0faa4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java @@ -51,6 +51,12 @@ interface ReadOptimizedView { Stream getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime); + /** + * Stream all the latest version data files in the given partition with precondition that + * instant time of file matches passed in instant time. + */ + Stream getLatestDataFilesOn(String partitionPath, String instantTime); + /** * Stream all the latest data files pass */ diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index 8aa585728b7e9..edcaa8261256d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -256,6 +256,22 @@ public Stream getLatestDataFilesInRange(List commitsToRe .map(Optional::get); } + @Override + public Stream getLatestDataFilesOn(String partitionPath, String instantTime) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> { + return fileGroup.getAllDataFiles() + .filter(dataFile -> + HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), + instantTime, + HoodieTimeline.EQUAL)) + .filter(df -> !isDataFileDueToPendingCompaction(df)) + .findFirst(); + }) + .filter(Optional::isPresent) + .map(Optional::get); + } + @Override public Stream getAllDataFiles(String partitionPath) { return getAllFileGroups(partitionPath) diff --git a/hoodie-common/src/test/resources/log4j-surefire.properties b/hoodie-common/src/test/resources/log4j-surefire.properties index 23ded09c3f49a..ea3e935455861 100644 --- a/hoodie-common/src/test/resources/log4j-surefire.properties +++ b/hoodie-common/src/test/resources/log4j-surefire.properties @@ -14,8 +14,10 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=WARN -log4j.category.org.apache.parquet.hadoop=ERROR +log4j.category.com.uber=INFO +log4j.category.com.uber.hoodie.table.log=WARN +log4j.category.com.uber.hoodie.common.util=WARN +log4j.category.org.apache.parquet.hadoop=WARN # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties b/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties index 23ded09c3f49a..3613e7d120a53 100644 --- a/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties +++ b/hoodie-hadoop-mr/src/test/resources/log4j-surefire.properties @@ -14,8 +14,8 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=WARN -log4j.category.org.apache.parquet.hadoop=ERROR +log4j.category.com.uber=INFO +log4j.category.org.apache.parquet.hadoop=WARN # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-hive/src/test/resources/log4j-surefire.properties b/hoodie-hive/src/test/resources/log4j-surefire.properties index 1d7678513d15f..8027f04d8375d 100644 --- a/hoodie-hive/src/test/resources/log4j-surefire.properties +++ b/hoodie-hive/src/test/resources/log4j-surefire.properties @@ -14,9 +14,9 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=WARN -log4j.category.org.apache.parquet.hadoop=ERROR -log4j.category.parquet.hadoop=ERROR +log4j.category.com.uber=INFO +log4j.category.org.apache.parquet.hadoop=WARN +log4j.category.parquet.hadoop=WARN # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. diff --git a/hoodie-spark/src/test/resources/log4j-surefire.properties b/hoodie-spark/src/test/resources/log4j-surefire.properties index 23ded09c3f49a..daf8d28c13c5f 100644 --- a/hoodie-spark/src/test/resources/log4j-surefire.properties +++ b/hoodie-spark/src/test/resources/log4j-surefire.properties @@ -14,8 +14,12 @@ # limitations under the License. # log4j.rootLogger=WARN, A1 -log4j.category.com.uber=WARN -log4j.category.org.apache.parquet.hadoop=ERROR +log4j.category.com.uber=INFO +log4j.category.com.uber.hoodie.common.utils=WARN +log4j.category.com.uber.hoodie.io=WARN +log4j.category.com.uber.hoodie.common=WARN +log4j.category.com.uber.hoodie.table.log=WARN +log4j.category.org.apache.parquet.hadoop=WARN # A1 is set to be a ConsoleAppender. log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. From 5b6396069603e3c43b52e4be8e5f111bb0c3bd75 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 5 Jun 2018 18:26:01 -0700 Subject: [PATCH 074/374] Add CLI support inspect, schedule and run compaction --- .../cli/commands/CompactionCommand.java | 229 +++++++++++++ .../commands/HDFSParquetImportCommand.java | 15 +- .../uber/hoodie/cli/commands/SparkMain.java | 39 ++- .../common/table/timeline/HoodieInstant.java | 4 + .../hoodie/utilities/HDFSParquetImporter.java | 307 ++++++++---------- .../hoodie/utilities/HoodieCompactor.java | 129 ++++++++ .../uber/hoodie/utilities/UtilHelpers.java | 109 +++++++ 7 files changed, 653 insertions(+), 179 deletions(-) create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java new file mode 100644 index 0000000000000..b5ed2ada5c25d --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java @@ -0,0 +1,229 @@ +/* + * 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.cli.commands; + +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; +import com.uber.hoodie.cli.commands.SparkMain.SparkCommand; +import com.uber.hoodie.cli.utils.InputStreamConsumer; +import com.uber.hoodie.cli.utils.SparkUtil; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.launcher.SparkLauncher; +import org.apache.spark.util.Utils; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +@Component +public class CompactionCommand implements CommandMarker { + + private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class); + + @CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline") + public String compactionsAll( + @CliOption(key = { + "includeExtraMetadata"}, help = "Include extra metadata", unspecifiedDefaultValue = "false") final + boolean includeExtraMetadata, + @CliOption(key = { + "limit"}, mandatory = false, 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 { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); + HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionTimeline(); + HoodieTimeline commitTimeline = activeTimeline.getCommitTimeline().filterCompletedInstants(); + Set committed = commitTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + + List instants = timeline.getInstants().collect(Collectors.toList()); + List rows = new ArrayList<>(); + Collections.reverse(instants); + for (int i = 0; i < instants.size(); i++) { + HoodieInstant instant = instants.get(i); + HoodieCompactionPlan workload = null; + if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) { + try { + // This could be a completed compaction. Assume a compaction request file is present but skip if fails + workload = AvroUtils.deserializeCompactionPlan( + activeTimeline.getInstantAuxiliaryDetails( + HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); + } catch (HoodieIOException ioe) { + // SKIP + } + } else { + workload = AvroUtils.deserializeCompactionPlan(activeTimeline.getInstantAuxiliaryDetails( + HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); + } + + if (null != workload) { + HoodieInstant.State state = instant.getState(); + if (committed.contains(instant.getTimestamp())) { + state = State.COMPLETED; + } + if (includeExtraMetadata) { + rows.add(new Comparable[]{instant.getTimestamp(), + state.toString(), + workload.getOperations() == null ? 0 : workload.getOperations().size(), + workload.getExtraMetadata().toString()}); + } else { + rows.add(new Comparable[]{instant.getTimestamp(), + state.toString(), + workload.getOperations() == null ? 0 : workload.getOperations().size()}); + } + } + } + + Map> fieldNameToConverterMap = new HashMap<>(); + TableHeader header = new TableHeader() + .addTableHeaderField("Compaction Instant Time") + .addTableHeaderField("State") + .addTableHeaderField("Total FileIds to be Compacted"); + if (includeExtraMetadata) { + header = header.addTableHeaderField("Extra Metadata"); + } + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); + } + + @CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant") + public String compactionShow( + @CliOption(key = "instant", mandatory = true, help = "Base path for the target hoodie dataset") final + String compactionInstantTime, + @CliOption(key = { + "limit"}, mandatory = false, 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 Exception { + HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); + HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan( + activeTimeline.getInstantAuxiliaryDetails( + HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get()); + + List rows = new ArrayList<>(); + if ((null != workload) && (null != workload.getOperations())) { + for (HoodieCompactionOperation op : workload.getOperations()) { + rows.add(new Comparable[]{op.getPartitionPath(), + op.getFileId(), + op.getBaseInstantTime(), + op.getDataFilePath(), + op.getDeltaFilePaths().size(), + op.getMetrics().toString() + }); + } + } + + Map> fieldNameToConverterMap = new HashMap<>(); + TableHeader header = new TableHeader() + .addTableHeaderField("Partition Path") + .addTableHeaderField("File Id") + .addTableHeaderField("Base Instant") + .addTableHeaderField("Data File Path") + .addTableHeaderField("Total Delta Files") + .addTableHeaderField("getMetrics"); + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); + } + + @CliCommand(value = "compaction schedule", help = "Schedule Compaction") + public String scheduleCompact( + @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, + @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField, + @CliOption(key = { + "parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") final String parallelism, + @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String + schemaFilePath, + @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, + @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + // First get a compaction instant time and pass it to spark launcher for scheduling compaction + String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime(); + + if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), HoodieCLI.tableMetadata.getBasePath(), + tableName, compactionInstantTime, rowKeyField, parallelism, schemaFilePath, sparkMemory, retry); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + if (exitCode != 0) { + return "Failed to run compaction for " + compactionInstantTime; + } + return "Compaction successfully completed for " + compactionInstantTime; + } else { + throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); + } + } + + @CliCommand(value = "compaction run", help = "Run Compaction for given instant time") + public String compact( + @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, + @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField, + @CliOption(key = { + "parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") final String parallelism, + @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String + schemaFilePath, + @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, + @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry, + @CliOption(key = "compactionInstant", mandatory = true, help = "Base path for the target hoodie dataset") final + String compactionInstantTime) throws Exception { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), HoodieCLI.tableMetadata.getBasePath(), + tableName, compactionInstantTime, rowKeyField, parallelism, schemaFilePath, sparkMemory, retry); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + if (exitCode != 0) { + return "Failed to run compaction for " + compactionInstantTime; + } + return "Compaction successfully completed for " + compactionInstantTime; + } else { + throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); + } + } +} \ No newline at end of file diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java index c3298c9ae4999..6750bafe915da 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java @@ -29,14 +29,17 @@ import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliOption; import org.springframework.stereotype.Component; +import scala.collection.JavaConverters; @Component public class HDFSParquetImportCommand implements CommandMarker { private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class); - @CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset") + @CliCommand(value = "hdfsparquetimport", help = "Imports Parquet dataset to a hoodie dataset") public String convert( + @CliOption(key = "upsert", mandatory = false, unspecifiedDefaultValue = "false", + help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert, @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath, @CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") final String srcType, @CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String @@ -59,10 +62,16 @@ public String convert( boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); String sparkPropertiesPath = Utils.getDefaultPropertiesFile( - scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala()); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName, tableType, rowKeyField, + String cmd = SparkCommand.IMPORT.toString(); + if (useUpsert) { + cmd = SparkCommand.UPSERT.toString(); + } + + sparkLauncher.addAppArgs(cmd, srcPath, targetPath, tableName, tableType, rowKeyField, partitionPathField, parallelism, schemaFilePath, sparkMemory, retry); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index 32e2a0091e683..526e6462064ad 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -24,6 +24,7 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.utilities.HDFSParquetImporter; +import com.uber.hoodie.utilities.HoodieCompactor; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -32,12 +33,11 @@ public class SparkMain { protected static final Logger LOG = Logger.getLogger(SparkMain.class); - /** * Commands */ enum SparkCommand { - ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT + ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN, } public static void main(String[] args) throws Exception { @@ -62,9 +62,20 @@ public static void main(String[] args) throws Exception { returnCode = rollbackToSavepoint(jsc, args[1], args[2]); break; case IMPORT: + case UPSERT: assert (args.length == 11); - returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6], Integer.parseInt(args[7]), - args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10])); + returnCode = dataLoad(jsc, command, args[1], args[2], args[3], args[4], args[5], args[6], + Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10])); + break; + case COMPACT_RUN: + assert (args.length == 9); + returnCode = compact(jsc, args[1], args[2], args[3], args[4], args[5], Integer.parseInt(args[6]), + args[7], args[8], Integer.parseInt(args[9]), false); + break; + case COMPACT_SCHEDULE: + assert (args.length == 10); + returnCode = compact(jsc, args[1], args[2], args[3], args[4], args[5], Integer.parseInt(args[6]), + args[7], args[8], Integer.parseInt(args[9]), true); break; default: break; @@ -73,10 +84,12 @@ public static void main(String[] args) throws Exception { System.exit(returnCode); } - private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath, String tableName, + private static int dataLoad(JavaSparkContext jsc, String command, + String srcPath, String targetPath, String tableName, String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster, String sparkMemory, int retry) throws Exception { HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); + cfg.command = command; cfg.srcPath = srcPath; cfg.targetPath = targetPath; cfg.tableName = tableName; @@ -89,6 +102,22 @@ private static int dataImport(JavaSparkContext jsc, String srcPath, String targe return new HDFSParquetImporter(cfg).dataImport(jsc, retry); } + private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant, + String rowKey, String partitionKey, int parallelism, String schemaFile, + String sparkMemory, int retry, boolean schedule) throws Exception { + HoodieCompactor.Config cfg = new HoodieCompactor.Config(); + cfg.basePath = basePath; + cfg.tableName = tableName; + cfg.compactionInstantTime = compactionInstant; + cfg.rowKey = rowKey; + cfg.partitionKey = partitionKey; + cfg.parallelism = parallelism; + cfg.schemaFile = schemaFile; + cfg.runSchedule = schedule; + jsc.getConf().set("spark.executor.memory", sparkMemory); + return new HoodieCompactor(cfg).compact(jsc, retry); + } + private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath, String repairedOutputPath, String basePath) throws Exception { DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java index a21683f0a7b1d..164f4486cbac6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java @@ -149,6 +149,10 @@ public boolean equals(Object o) { && Objects.equals(timestamp, that.timestamp); } + public State getState() { + return state; + } + @Override public int hashCode() { return Objects.hash(state, action, timestamp); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 70871b30a4b84..7ec989c95a5e4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -26,24 +26,21 @@ import com.uber.hoodie.common.HoodieJsonPayload; 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.HoodieTableConfig; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; -import com.uber.hoodie.config.HoodieIndexConfig; -import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; -import com.uber.hoodie.index.HoodieIndex; import java.io.IOException; import java.io.Serializable; -import java.nio.ByteBuffer; import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; import java.util.List; +import java.util.Optional; import java.util.Properties; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; @@ -52,23 +49,150 @@ import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetInputFormat; -import org.apache.spark.Accumulator; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import scala.Tuple2; +/** + * Loads data from Parquet Sources + */ public class HDFSParquetImporter implements Serializable { + public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd"); private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class); private final Config cfg; private transient FileSystem fs; - public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd"); public HDFSParquetImporter(Config cfg) throws IOException { this.cfg = cfg; } + 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); + } + HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg); + dataImporter.dataImport(UtilHelpers.buildSparkContext(cfg.tableName, cfg.sparkMaster, cfg.sparkMemory), cfg.retry); + } + + public int dataImport(JavaSparkContext jsc, int retry) throws Exception { + this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration()); + int ret = -1; + try { + // Verify that targetPath is not present. + if (fs.exists(new Path(cfg.targetPath))) { + throw new HoodieIOException(String.format("Make sure %s is not present.", cfg.targetPath)); + } + do { + ret = dataImport(jsc); + } while (ret != 0 && retry-- > 0); + } catch (Throwable t) { + logger.error(t); + } + return ret; + } + + @VisibleForTesting + protected int dataImport(JavaSparkContext jsc) throws IOException { + try { + if (fs.exists(new Path(cfg.targetPath))) { + // cleanup target directory. + fs.delete(new Path(cfg.targetPath), true); + } + + //Get schema. + String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile); + + // Initialize target hoodie table. + Properties properties = new Properties(); + properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName); + properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType); + HoodieTableMetaClient + .initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); + + HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr, + cfg.parallelism, Optional.empty()); + + JavaRDD> hoodieRecords = buildHoodieRecordsForImport(jsc, schemaStr); + // Get instant time. + String instantTime = client.startCommit(); + JavaRDD writeResponse = load(client, instantTime, hoodieRecords); + return UtilHelpers.handleErrors(jsc, instantTime, writeResponse); + } catch (Throwable t) { + logger.error("Error occurred.", t); + } + return -1; + } + + protected JavaRDD> buildHoodieRecordsForImport( + JavaSparkContext jsc, String schemaStr) throws IOException { + Job job = Job.getInstance(jsc.hadoopConfiguration()); + // Allow recursive directories to be found + job.getConfiguration().set(FileInputFormat.INPUT_DIR_RECURSIVE, "true"); + // To parallelize reading file status. + job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024"); + AvroReadSupport + .setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr))); + ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class)); + + return jsc.newAPIHadoopFile(cfg.srcPath, + ParquetInputFormat.class, Void.class, GenericRecord.class, job.getConfiguration()) + // To reduce large number of + // tasks. + .coalesce(16 * cfg.parallelism) + .map(entry -> { + GenericRecord genericRecord + = ((Tuple2) entry)._2(); + Object partitionField = + genericRecord.get(cfg.partitionKey); + if (partitionField == null) { + throw new HoodieIOException( + "partition key is missing. :" + + cfg.partitionKey); + } + Object rowField = genericRecord.get(cfg.rowKey); + if (rowField == null) { + throw new HoodieIOException( + "row field is missing. :" + cfg.rowKey); + } + String partitionPath = partitionField.toString(); + logger.info("Row Key : " + rowField + ", Partition Path is (" + partitionPath + ")"); + if (partitionField instanceof Number) { + try { + long ts = (long) (Double.parseDouble(partitionField.toString()) * 1000L); + partitionPath = + PARTITION_FORMATTER.format(new Date(ts)); + } catch (NumberFormatException nfe) { + logger.warn("Unable to parse date from partition field. Assuming partition as (" + partitionField + ")"); + } + } + return new HoodieRecord<>( + new HoodieKey( + (String) rowField, partitionPath), + new HoodieJsonPayload( + genericRecord.toString())); + }); + } + + /** + * Imports records to Hoodie dataset + * + * @param client Hoodie Client + * @param instantTime Instant Time + * @param hoodieRecords Hoodie Records + * @param Type + */ + protected JavaRDD load(HoodieWriteClient client, + String instantTime, JavaRDD> hoodieRecords) { + if (cfg.command.toLowerCase().equals("insert")) { + return client.insert(hoodieRecords, instantTime); + } + return client.upsert(hoodieRecords, instantTime); + } + public static class FormatValidator implements IValueValidator { List validFormats = Arrays.asList("parquet"); @@ -97,6 +221,10 @@ public void validate(String name, String value) throws ParameterException { public static class Config implements Serializable { + @Parameter(names = {"--command", "-c"}, + description = "Write command Valid values are insert(default)/upsert", + required = false) + public String command = "INSERT"; @Parameter(names = {"--src-path", "-sp"}, description = "Base path for the input dataset", required = true) public String srcPath = null; @@ -137,167 +265,4 @@ public static class Config implements Serializable { @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; } - - public static void main(String[] args) throws Exception { - final HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); - JCommander cmd = new JCommander(cfg, args); - if (cfg.help || args.length == 0) { - cmd.usage(); - System.exit(1); - } - HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg); - dataImporter.dataImport(dataImporter.getSparkContext(), cfg.retry); - } - - private JavaSparkContext getSparkContext() { - SparkConf sparkConf = new SparkConf().setAppName("hoodie-data-importer-" + cfg.tableName); - sparkConf.setMaster(cfg.sparkMaster); - - if (cfg.sparkMaster.startsWith("yarn")) { - sparkConf.set("spark.eventLog.overwrite", "true"); - sparkConf.set("spark.eventLog.enabled", "true"); - } - - sparkConf.set("spark.driver.maxResultSize", "2g"); - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.set("spark.executor.memory", cfg.sparkMemory); - - // Configure hadoop conf - 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", - "org.apache.hadoop.io.compress.GzipCodec"); - sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); - - sparkConf = HoodieWriteClient.registerClasses(sparkConf); - return new JavaSparkContext(sparkConf); - } - - private String getSchema() throws Exception { - // Read schema file. - Path p = new Path(cfg.schemaFile); - if (!fs.exists(p)) { - throw new Exception(String.format("Could not find - %s - schema file.", cfg.schemaFile)); - } - long len = fs.getFileStatus(p).getLen(); - ByteBuffer buf = ByteBuffer.allocate((int) len); - FSDataInputStream inputStream = null; - try { - inputStream = fs.open(p); - inputStream.readFully(0, buf.array(), 0, buf.array().length); - } finally { - if (inputStream != null) { - inputStream.close(); - } - } - return new String(buf.array()); - } - - public int dataImport(JavaSparkContext jsc, int retry) throws Exception { - this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration()); - int ret = -1; - try { - // Verify that targetPath is not present. - if (fs.exists(new Path(cfg.targetPath))) { - throw new HoodieIOException(String.format("Make sure %s is not present.", cfg.targetPath)); - } - do { - ret = dataImport(jsc); - } while (ret != 0 && retry-- > 0); - } catch (Throwable t) { - logger.error(t); - } - return ret; - } - - @VisibleForTesting - protected int dataImport(JavaSparkContext jsc) throws IOException { - try { - if (fs.exists(new Path(cfg.targetPath))) { - // cleanup target directory. - fs.delete(new Path(cfg.targetPath), true); - } - - //Get schema. - String schemaStr = getSchema(); - - // Initialize target hoodie table. - Properties properties = new Properties(); - properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName); - properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType); - HoodieTableMetaClient - .initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); - - HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr, - cfg.parallelism); - - Job job = Job.getInstance(jsc.hadoopConfiguration()); - // To parallelize reading file status. - job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024"); - AvroReadSupport - .setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr))); - ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class)); - - JavaRDD> hoodieRecords = jsc.newAPIHadoopFile(cfg.srcPath, - ParquetInputFormat.class, Void.class, GenericRecord.class, job.getConfiguration()) - // To reduce large number of - // tasks. - .coalesce(16 * cfg.parallelism) - .map(entry -> { - GenericRecord genericRecord - = ((Tuple2) entry)._2(); - Object partitionField = - genericRecord.get(cfg.partitionKey); - if (partitionField == null) { - throw new HoodieIOException( - "partition key is missing. :" - + cfg.partitionKey); - } - Object rowField = genericRecord.get(cfg.rowKey); - if (rowField == null) { - throw new HoodieIOException( - "row field is missing. :" + cfg.rowKey); - } - long ts = (long) ((Double) partitionField * 1000L); - String partitionPath = - PARTITION_FORMATTER.format(new Date(ts)); - return new HoodieRecord<>( - new HoodieKey( - (String) rowField, partitionPath), - new HoodieJsonPayload( - genericRecord.toString())); - }); - // Get commit time. - String commitTime = client.startCommit(); - - JavaRDD writeResponse = client.bulkInsert(hoodieRecords, commitTime); - Accumulator errors = jsc.accumulator(0); - writeResponse.foreach(writeStatus -> { - if (writeStatus.hasErrors()) { - errors.add(1); - logger.error(String.format("Error processing records :writeStatus:%s", - writeStatus.getStat().toString())); - } - }); - if (errors.value() == 0) { - logger.info( - String.format("Dataset imported into hoodie dataset with %s commit time.", commitTime)); - return 0; - } - logger.error(String.format("Import failed with %d errors.", errors.value())); - } catch (Throwable t) { - logger.error("Error occurred.", t); - } - return -1; - } - - private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, - String schemaStr, int parallelism) throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withParallelism(parallelism, parallelism).withSchema(schemaStr) - .combineInput(true, true).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); - return new HoodieWriteClient(jsc, config); - } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java new file mode 100644 index 0000000000000..d986a6ac4e8f7 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java @@ -0,0 +1,129 @@ +/* + * 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.WriteStatus; +import com.uber.hoodie.common.util.FSUtils; +import java.io.Serializable; +import java.util.Optional; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + + +public class HoodieCompactor { + + private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class); + private final Config cfg; + private transient FileSystem fs; + + public HoodieCompactor(Config cfg) { + this.cfg = cfg; + } + + public static class Config implements Serializable { + @Parameter(names = {"--base-path", + "-sp"}, description = "Base path for the dataset", required = true) + public String basePath = null; + @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true) + public String tableName = null; + @Parameter(names = {"--instant-time", + "-sp"}, description = "Compaction Instant time", required = true) + public String compactionInstantTime = null; + @Parameter(names = {"--row-key-field", + "-rk"}, description = "Row key field name", required = true) + public String rowKey = null; + @Parameter(names = {"--partition-key-field", + "-pk"}, description = "Partition key field name", required = true) + public String partitionKey = null; + @Parameter(names = {"--parallelism", + "-pl"}, description = "Parallelism for hoodie insert", required = true) + public int parallelism = 1; + @Parameter(names = {"--schema-file", + "-sf"}, description = "path for Avro schema file", required = true) + public String schemaFile = null; + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + public String sparkMaster = null; + @Parameter(names = {"--spark-memory", + "-sm"}, description = "spark memory to use", required = true) + public String sparkMemory = null; + @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false) + public int retry = 0; + @Parameter(names = {"--schedule", "-sc"}, description = "Schedule compaction", required = false) + public Boolean runSchedule = false; + @Parameter(names = {"--strategy", "-st"}, description = "Stratgey Class", required = false) + public String strategyClassName = null; + @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); + } + HoodieCompactor compactor = new HoodieCompactor(cfg); + compactor.compact(UtilHelpers.buildSparkContext(cfg.tableName, cfg.sparkMaster, cfg.sparkMemory), cfg.retry); + } + + public int compact(JavaSparkContext jsc, int retry) { + this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); + int ret = -1; + try { + do { + if (cfg.runSchedule) { + if (null == cfg.strategyClassName) { + throw new IllegalArgumentException("Missing Strategy class name for running compaction"); + } + ret = doSchedule(jsc); + } else { + ret = doCompact(jsc); + } + } while (ret != 0 && retry-- > 0); + } catch (Throwable t) { + logger.error(t); + } + return ret; + } + + private int doCompact(JavaSparkContext jsc) throws Exception { + //Get schema. + String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile); + HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, + Optional.empty()); + JavaRDD writeResponse = client.compact(cfg.compactionInstantTime); + return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse); + } + + private int doSchedule(JavaSparkContext jsc) throws Exception { + //Get schema. + String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile); + HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, + Optional.of(cfg.strategyClassName)); + client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Optional.empty()); + return 0; + } +} \ No newline at end of file diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index d64829ea6bf72..77952bcf0d436 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -18,24 +18,39 @@ package com.uber.hoodie.utilities; +import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.util.ReflectionUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.Source; import com.uber.hoodie.utilities.sources.SourceDataFormat; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Optional; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.lang3.reflect.ConstructorUtils; import org.apache.hadoop.fs.FSDataInputStream; 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.Accumulator; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; /** * Bunch of helper methods */ public class UtilHelpers { + private static Logger logger = LogManager.getLogger(UtilHelpers.class); public static Source createSource(String sourceClass, PropertiesConfiguration cfg, JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider) @@ -76,4 +91,98 @@ public static PropertiesConfiguration readConfig(FileSystem fs, Path cfgPath) { } } + /** + * Parse Schema from file + * + * @param fs File System + * @param schemaFile Schema File + */ + public static String parseSchema(FileSystem fs, String schemaFile) throws Exception { + // Read schema file. + Path p = new Path(schemaFile); + if (!fs.exists(p)) { + throw new Exception(String.format("Could not find - %s - schema file.", schemaFile)); + } + long len = fs.getFileStatus(p).getLen(); + ByteBuffer buf = ByteBuffer.allocate((int) len); + FSDataInputStream inputStream = null; + try { + inputStream = fs.open(p); + inputStream.readFully(0, buf.array(), 0, buf.array().length); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + return new String(buf.array()); + } + + /** + * Build Spark Context for ingestion/compaction + * @return + */ + public static JavaSparkContext buildSparkContext(String tableName, String sparkMaster, String sparkMemory) { + SparkConf sparkConf = new SparkConf().setAppName("hoodie-data-importer-" + tableName); + sparkConf.setMaster(sparkMaster); + + if (sparkMaster.startsWith("yarn")) { + sparkConf.set("spark.eventLog.overwrite", "true"); + sparkConf.set("spark.eventLog.enabled", "true"); + } + + sparkConf.set("spark.driver.maxResultSize", "2g"); + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + sparkConf.set("spark.executor.memory", sparkMemory); + + // Configure hadoop conf + 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", + "org.apache.hadoop.io.compress.GzipCodec"); + sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); + + sparkConf = HoodieWriteClient.registerClasses(sparkConf); + return new JavaSparkContext(sparkConf); + } + + /** + * Build Hoodie write client + * + * @param jsc Java Spark Context + * @param basePath Base Path + * @param schemaStr Schema + * @param parallelism Parallelism + */ + public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, + String schemaStr, int parallelism, Optional compactionStrategyClass) throws Exception { + HoodieCompactionConfig compactionConfig = + compactionStrategyClass.map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false) + .withCompactionStrategy(ReflectionUtils.loadClass(strategy)) + .build()).orElse(HoodieCompactionConfig.newBuilder().withInlineCompaction(false).build()); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withParallelism(parallelism, parallelism).withSchema(schemaStr) + .combineInput(true, true) + .withCompactionConfig(compactionConfig) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .build(); + return new HoodieWriteClient(jsc, config); + } + + public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD writeResponse) { + Accumulator errors = jsc.accumulator(0); + writeResponse.foreach(writeStatus -> { + if (writeStatus.hasErrors()) { + errors.add(1); + logger.error(String.format("Error processing records :writeStatus:%s", + writeStatus.getStat().toString())); + } + }); + if (errors.value() == 0) { + logger.info( + String.format("Dataset imported into hoodie dataset with %s instant time.", instantTime)); + return 0; + } + logger.error(String.format("Import failed with %d errors.", errors.value())); + return -1; + } } From ef6f137adcc6bd687f257ff266a379207cd01409 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Fri, 3 Aug 2018 16:40:58 -0700 Subject: [PATCH 075/374] Minor bug fixes found during testing --- hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java | 3 ++- .../src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java | 2 +- hoodie-common/src/main/avro/HoodieCommitMetadata.avsc | 5 +++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java index d02b7810a4c01..79f1424cbeb5f 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java @@ -144,7 +144,8 @@ private void sortAndLimit() { if (fieldNameToConverterMap.containsKey(fieldName)) { return fieldNameToConverterMap.get(fieldName).apply(row.get(idx)); } - return row.get(idx).toString(); + Object v = row.get(idx); + return v == null ? "null" : v.toString(); }).collect(Collectors.toList()); }).collect(Collectors.toList()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index e05e34d2fef39..a0b2321053bb6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -75,7 +75,7 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTab throw new HoodieInsertException( "Failed to initialize HoodieStorageWriter for path " + getStorageWriterPath(), e); } - logger.info("New InsertHandle for partition :" + partitionPath); + logger.info("New InsertHandle for partition :" + partitionPath + " with fileId " + fileId); } public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index 832b5fc9ccf7a..3d41765ffb9e5 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -59,7 +59,8 @@ }, { "name":"totalUpdatedRecordsCompacted", - "type":["null","long"] + "type":["null","long"], + "default" : null } ] } @@ -74,4 +75,4 @@ }] } ] -} \ No newline at end of file +} From 9ec4cba39bbf892b7614d998205bad8ee6b0e176 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 7 Aug 2018 15:57:01 -0700 Subject: [PATCH 076/374] BUGFIX - Use Guava Optional (which is Serializable) in CompactionOperation wcached to avoid NoSerializableException --- .../uber/hoodie/common/model/CompactionOperation.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java index f7f886e2a7a28..b094c49c55b11 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java @@ -16,6 +16,7 @@ package com.uber.hoodie.common.model; +import com.google.common.base.Optional; import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.common.util.FSUtils; import java.io.Serializable; @@ -23,7 +24,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; /** @@ -34,6 +34,7 @@ public class CompactionOperation implements Serializable { private String baseInstantTime; + // Using Guava Optional as it is serializable private Optional dataFileCommitTime; private List deltaFilePaths; private Optional dataFilePath; @@ -46,7 +47,7 @@ public class CompactionOperation implements Serializable { public CompactionOperation() { } - public CompactionOperation(Optional dataFile, String partitionPath, + public CompactionOperation(java.util.Optional dataFile, String partitionPath, List logFiles, Map metrics) { if (dataFile.isPresent()) { this.baseInstantTime = dataFile.get().getCommitTime(); @@ -55,10 +56,10 @@ public CompactionOperation(Optional dataFile, String partitionPa this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime()); } else { assert logFiles.size() > 0; - this.dataFilePath = Optional.empty(); + this.dataFilePath = Optional.absent(); this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath()); this.fileId = FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()); - this.dataFileCommitTime = Optional.empty(); + this.dataFileCommitTime = Optional.absent(); } this.partitionPath = partitionPath; @@ -103,7 +104,7 @@ public Map getMetrics() { public static CompactionOperation convertFromAvroRecordInstance(HoodieCompactionOperation operation) { CompactionOperation op = new CompactionOperation(); op.baseInstantTime = operation.getBaseInstantTime(); - op.dataFilePath = Optional.ofNullable(operation.getDataFilePath()); + op.dataFilePath = Optional.fromNullable(operation.getDataFilePath()); op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths()); op.fileId = operation.getFileId(); op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics()); From 997c21463e75603da55a1d76d712a8402c132c3f Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 7 Aug 2018 15:51:46 -0700 Subject: [PATCH 077/374] Adding another metric to HoodieWriteStat to determine if there were inserts converted to updates, added one test for this --- .../uber/hoodie/io/HoodieCreateHandle.java | 3 + .../com/uber/hoodie/io/HoodieMergeHandle.java | 16 +++- ...ecords.java => TestHoodieMergeHandle.java} | 83 ++++++++++++++++++- .../hoodie/table/TestMergeOnReadTable.java | 4 +- .../src/main/avro/HoodieCommitMetadata.avsc | 5 ++ .../hoodie/common/model/HoodieWriteStat.java | 13 +++ 6 files changed, 116 insertions(+), 8 deletions(-) rename hoodie-client/src/test/java/com/uber/hoodie/io/{TestHoodieMergeHandleDuplicateRecords.java => TestHoodieMergeHandle.java} (72%) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index a0b2321053bb6..0781faf36de41 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -48,6 +48,7 @@ public class HoodieCreateHandle extends HoodieIOH private final Path path; private Path tempPath = null; private long recordsWritten = 0; + private long insertRecordsWritten = 0; private long recordsDeleted = 0; private Iterator> recordIterator; @@ -100,6 +101,7 @@ public void write(HoodieRecord record, Optional avroRecord) { // update the new location of record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId())); recordsWritten++; + insertRecordsWritten++; } else { recordsDeleted++; } @@ -149,6 +151,7 @@ public WriteStatus close() { HoodieWriteStat stat = new HoodieWriteStat(); stat.setNumWrites(recordsWritten); stat.setNumDeletes(recordsDeleted); + stat.setNumInserts(insertRecordsWritten); stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); stat.setFileId(status.getFileId()); stat.setPaths(new Path(config.getBasePath()), path, tempPath); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 2bd5d8668b87b..f11fce57c1c74 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -67,6 +67,7 @@ public class HoodieMergeHandle extends HoodieIOHa private long recordsWritten = 0; private long recordsDeleted = 0; private long updatedRecordsWritten = 0; + private long insertRecordsWritten = 0; public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, Iterator> recordItr, String fileId) { @@ -173,14 +174,19 @@ private String init(String fileId, Iterator> newRecordsItr) { return partitionPath; } - private boolean writeUpdateRecord(HoodieRecord hoodieRecord, - Optional indexedRecord) { + private boolean writeUpdateRecord(HoodieRecord hoodieRecord, Optional indexedRecord) { + if (indexedRecord.isPresent()) { + updatedRecordsWritten++; + } + return writeRecord(hoodieRecord, indexedRecord); + } + + private boolean writeRecord(HoodieRecord hoodieRecord, Optional indexedRecord) { Optional recordMetadata = hoodieRecord.getData().getMetadata(); try { if (indexedRecord.isPresent()) { storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord); recordsWritten++; - updatedRecordsWritten++; } else { recordsDeleted++; } @@ -256,7 +262,8 @@ public WriteStatus close() { String key = pendingRecordsItr.next(); if (!writtenRecordKeys.contains(key)) { HoodieRecord hoodieRecord = keyToNewRecords.get(key); - writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema)); + writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema)); + insertRecordsWritten++; } } keyToNewRecords.clear(); @@ -270,6 +277,7 @@ public WriteStatus close() { writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten); + writeStatus.getStat().setNumInserts(insertRecordsWritten); writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size()); RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalUpsertTime(timer.endTimer()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandle.java similarity index 72% rename from hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java rename to hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandle.java index e5523f8beaad4..d28750108e95c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandle.java @@ -26,6 +26,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; @@ -35,6 +36,7 @@ import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.table.HoodieTable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -46,12 +48,13 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.rules.TemporaryFolder; @SuppressWarnings("unchecked") -public class TestHoodieMergeHandleDuplicateRecords { +public class TestHoodieMergeHandle { protected transient JavaSparkContext jsc = null; protected transient SQLContext sqlContext; @@ -62,7 +65,7 @@ public class TestHoodieMergeHandleDuplicateRecords { @Before public void init() throws IOException { // Initialize a local spark env - jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandleDuplicateRecords")); + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandle")); //SQLContext stuff sqlContext = new SQLContext(jsc); @@ -241,6 +244,82 @@ public void testUpsertsForMultipleRecordsInSameFile() throws Exception { assertEquals(21, record2Count); } + @Test + public void testHoodieMergeHandleWriteStatMetrics() throws Exception { + // insert 100 records + HoodieWriteConfig config = getConfigBuilder().build(); + HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); + + // All records should be inserts into new parquet + Assert.assertTrue(statuses.stream() + .filter(status -> status.getStat().getPrevCommit() != HoodieWriteStat.NULL_COMMIT).count() > 0); + // Num writes should be equal to the number of records inserted + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100); + // Num update writes should be equal to the number of records updated + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 0); + // Num update writes should be equal to the number of insert records converted to updates as part of small file + // handling + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100); + + // Update all the 100 records + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + + newCommitTime = "101"; + writeClient.startCommitWithTime(newCommitTime); + + List updatedRecords = dataGen.generateUpdates(newCommitTime, records); + JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); + statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime).collect(); + + // All records should be upserts into existing parquet + Assert.assertEquals(statuses.stream() + .filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0); + // Num writes should be equal to the number of records inserted + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100); + // Num update writes should be equal to the number of records updated + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100); + // Num update writes should be equal to the number of insert records converted to updates as part of small file + // handling + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 0); + + + newCommitTime = "102"; + writeClient.startCommitWithTime(newCommitTime); + + List allRecords = dataGen.generateInserts(newCommitTime, 100); + allRecords.addAll(updatedRecords); + JavaRDD allRecordsRDD = jsc.parallelize(allRecords, 1); + statuses = writeClient.upsert(allRecordsRDD, newCommitTime).collect(); + + // All records should be upserts into existing parquet (with inserts as updates small file handled) + Assert.assertEquals((long) statuses.stream() + .filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0); + // Num writes should be equal to the total number of records written + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 200); + // Num update writes should be equal to the number of records updated (including inserts converted as updates) + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100); + // Num update writes should be equal to the number of insert records converted to updates as part of small file + // handling + Assert.assertEquals((long) statuses.stream() + .map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100); + + } + private Dataset getRecords() { // Check the entire dataset has 8 records still String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index fbcdac2580160..f5c22c4422b46 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -827,8 +827,8 @@ public void testInsertsGeneratedIntoLogFilesRollback() throws Exception { writeClient.commit(newCommitTime, statuses); // rollback a successful commit - // Sleep for small interval to force a new rollback start time. - Thread.sleep(5); + // Sleep for small interval (at least 1 second) to force a new rollback start time. + Thread.sleep(1000); writeClient.rollback(newCommitTime); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index 3d41765ffb9e5..0429cc7d38d0e 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -61,6 +61,11 @@ "name":"totalUpdatedRecordsCompacted", "type":["null","long"], "default" : null + }, + { + "name":"numInserts", + "type":["null","long"], + "default" : null } ] } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index e03e84e3a67da..7523684817a00 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -61,6 +61,11 @@ public class HoodieWriteStat implements Serializable { */ private long numUpdateWrites; + /** + * Total number of insert records or converted to updates (for small file handling) + */ + private long numInserts; + /** * Total size of file written */ @@ -160,6 +165,10 @@ public void setNumUpdateWrites(long numUpdateWrites) { this.numUpdateWrites = numUpdateWrites; } + public void setNumInserts(long numInserts) { + this.numInserts = numInserts; + } + public long getTotalWriteBytes() { return totalWriteBytes; } @@ -192,6 +201,10 @@ public long getNumUpdateWrites() { return numUpdateWrites; } + public long getNumInserts() { + return numInserts; + } + public String getFileId() { return fileId; } From 75e8ddc0e8bfd2e9e0270dca4ecaa807ecbc860f Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 21 Aug 2018 17:27:56 -0700 Subject: [PATCH 078/374] Fixing Null pointer exception in finally block --- .../java/com/uber/hoodie/table/HoodieMergeOnReadTable.java | 4 +++- .../uber/hoodie/common/table/log/HoodieLogFormatWriter.java | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 3ead5904c95eb..078d23ad07f8e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -296,7 +296,9 @@ public List rollback(JavaSparkContext jsc, List comm "Failed to rollback for commit " + commit, io); } finally { try { - writer.close(); + if (writer != null) { + writer.close(); + } } catch (IOException io) { throw new UncheckedIOException(io); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index e0644d69fd8ce..c75e666bd6949 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -72,6 +72,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { try { this.output = fs.append(path, bufferSize); } catch (RemoteException e) { + log.warn("Remote Exception, attempting to handle or recover lease", e); handleAppendExceptionOrRecoverLease(path, e); } catch (IOException ioe) { if (ioe.getMessage().equalsIgnoreCase("Not supported")) { From 1d912955006b661d36abc99511c260070d4094fa Mon Sep 17 00:00:00 2001 From: Kaushik Devarajaiah Date: Thu, 19 Jul 2018 13:46:33 -0700 Subject: [PATCH 079/374] Throttling to limit QPS from HbaseIndex --- .../hoodie/config/HoodieHBaseIndexConfig.java | 175 ++++++++++++++++++ .../uber/hoodie/config/HoodieIndexConfig.java | 41 +--- .../uber/hoodie/config/HoodieWriteConfig.java | 32 +++- .../uber/hoodie/index/hbase/HBaseIndex.java | 158 +++++++++++++++- .../com/uber/hoodie/index/TestHbaseIndex.java | 76 +++++++- .../uber/hoodie/index/TestHoodieIndex.java | 4 +- 6 files changed, 436 insertions(+), 50 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/config/HoodieHBaseIndexConfig.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieHBaseIndexConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieHBaseIndexConfig.java new file mode 100644 index 0000000000000..48e8031c13912 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieHBaseIndexConfig.java @@ -0,0 +1,175 @@ +/* + * 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.config; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +public class HoodieHBaseIndexConfig extends DefaultHoodieConfig { + + public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; + public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; + public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; + public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; + /** + * Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not + * be honored for HBase Puts + */ + public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; + /** + * Property to set to enable auto computation of put batch size + */ + public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = "hoodie.index.hbase.put.batch.size.autocompute"; + public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = "false"; + /** + * Property to set the fraction of the global share of QPS that should be allocated to this job. + * Let's say there are 3 jobs which have input size in terms of number of rows required for + * HbaseIndexing as x, 2x, 3x respectively. Then this fraction for the jobs would be (0.17) 1/6, + * 0.33 (2/6) and 0.5 (3/6) respectively. + */ + public static final String HBASE_QPS_FRACTION_PROP = "hoodie.index.hbase.qps.fraction"; + /** + * Property to set maximum QPS allowed per Region Server. This should be same across various + * jobs. This is intended to limit the aggregate QPS generated across various jobs to an Hbase + * Region Server. It is recommended to set this value based on global indexing throughput needs + * and most importantly, how much the HBase installation in use is able to tolerate without + * Region Servers going down. + */ + public static String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = "hoodie.index.hbase.max.qps.per.region.server"; + /** + * Default batch size, used only for Get, but computed for Put + */ + public static final int DEFAULT_HBASE_BATCH_SIZE = 100; + /** + * A low default value. + */ + public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = 1000; + /** + * Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming + * Region Servers + */ + public static final float DEFAULT_HBASE_QPS_FRACTION = 0.5f; + + public HoodieHBaseIndexConfig(final Properties props) { + super(props); + } + + public static HoodieHBaseIndexConfig.Builder newBuilder() { + return new HoodieHBaseIndexConfig.Builder(); + } + + public static class Builder { + + private final Properties props = new Properties(); + + public HoodieHBaseIndexConfig.Builder fromFile(File propertiesFile) throws IOException { + FileReader reader = new FileReader(propertiesFile); + try { + this.props.load(reader); + return this; + } finally { + reader.close(); + } + } + + public HoodieHBaseIndexConfig.Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseZkQuorum(String zkString) { + props.setProperty(HBASE_ZKQUORUM_PROP, zkString); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseZkPort(int port) { + props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port)); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseTableName(String tableName) { + props.setProperty(HBASE_TABLENAME_PROP, tableName); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseIndexGetBatchSize(int getBatchSize) { + props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize)); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseIndexPutBatchSize(int putBatchSize) { + props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize)); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseIndexPutBatchSizeAutoCompute( + boolean putBatchSizeAutoCompute) { + props.setProperty(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, + String.valueOf(putBatchSizeAutoCompute)); + return this; + } + + public HoodieHBaseIndexConfig.Builder hbaseIndexQPSFraction(float qpsFraction) { + props.setProperty(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION_PROP, + String.valueOf(qpsFraction)); + return this; + } + + /** + *

+ * Method to set maximum QPS allowed per Region Server. This should be same across various + * jobs. This is intended to limit the aggregate QPS generated across various jobs to an + * Hbase Region Server. + *

+ *

+ * It is recommended to set this value based on your global indexing throughput needs and + * most importantly, how much your HBase installation is able to tolerate without Region + * Servers going down. + *

+ */ + public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer( + int maxQPSPerRegionServer) { + // This should be same across various jobs + props.setProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP, + String.valueOf(maxQPSPerRegionServer)); + return this; + } + + public HoodieHBaseIndexConfig build() { + HoodieHBaseIndexConfig config = new HoodieHBaseIndexConfig(props); + setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP), + HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); + setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), + HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); + setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP), + HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, + String.valueOf(DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE)); + setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP), + HBASE_QPS_FRACTION_PROP, String.valueOf(DEFAULT_HBASE_QPS_FRACTION)); + setDefaultOnCondition(props, + !props.containsKey(HBASE_MAX_QPS_PER_REGION_SERVER_PROP), + HBASE_MAX_QPS_PER_REGION_SERVER_PROP, String.valueOf( + DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER)); + return config; + } + + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java index 5a24cc12f2a98..f347509732994 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java @@ -49,14 +49,6 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { "hoodie.bloom.index.input.storage" + ".level"; public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; - // ***** HBase Index Configs ***** - public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; - public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; - public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; - public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; - public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; - public static final String DEFAULT_HBASE_BATCH_SIZE = "100"; - // ***** Bucketed Index Configs ***** public static final String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets"; @@ -92,6 +84,11 @@ public Builder withIndexType(HoodieIndex.IndexType indexType) { return this; } + public Builder withHBaseIndexConfig(HoodieHBaseIndexConfig hBaseIndexConfig) { + props.putAll(hBaseIndexConfig.getProps()); + return this; + } + public Builder bloomFilterNumEntries(int numEntries) { props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries)); return this; @@ -102,21 +99,6 @@ public Builder bloomFilterFPP(double fpp) { return this; } - public Builder hbaseZkQuorum(String zkString) { - props.setProperty(HBASE_ZKQUORUM_PROP, zkString); - return this; - } - - public Builder hbaseZkPort(int port) { - props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port)); - return this; - } - - public Builder hbaseTableName(String tableName) { - props.setProperty(HBASE_TABLENAME_PROP, tableName); - return this; - } - public Builder bloomIndexParallelism(int parallelism) { props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism)); return this; @@ -137,15 +119,6 @@ public Builder numBucketsPerPartition(int numBuckets) { return this; } - public Builder hbaseIndexGetBatchSize(int getBatchSize) { - props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize)); - return this; - } - - public Builder hbaseIndexPutBatchSize(int putBatchSize) { - props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize)); - return this; - } public Builder withBloomIndexInputStorageLevel(String level) { props.setProperty(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level); @@ -166,10 +139,6 @@ public HoodieIndexConfig build() { BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP), BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING); - setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP), - HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); - setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), - HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE)); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL), BLOOM_INDEX_INPUT_STORAGE_LEVEL, DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL); // Throws IllegalArgumentException if the value set is not a known Hoodie Index Type diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 6c1e394df59cf..2349860b1aa94 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -258,23 +258,45 @@ public double getBloomFilterFPP() { } public String getHbaseZkQuorum() { - return props.getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP); + return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKQUORUM_PROP); } public int getHbaseZkPort() { - return Integer.parseInt(props.getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP)); + return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKPORT_PROP)); } public String getHbaseTableName() { - return props.getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP); + return props.getProperty(HoodieHBaseIndexConfig.HBASE_TABLENAME_PROP); } public int getHbaseIndexGetBatchSize() { - return Integer.valueOf(props.getProperty(HoodieIndexConfig.HBASE_GET_BATCH_SIZE_PROP)); + return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE_PROP)); } public int getHbaseIndexPutBatchSize() { - return Integer.valueOf(props.getProperty(HoodieIndexConfig.HBASE_PUT_BATCH_SIZE_PROP)); + return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_PROP)); + } + + public Boolean getHbaseIndexPutBatchSizeAutoCompute() { + return Boolean.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP)); + } + + /** + * Fraction of the global share of QPS that should be allocated to this job. + * Let's say there are 3 jobs which have input size in terms of number of rows + * required for HbaseIndexing as x, 2x, 3x respectively. Then this fraction for + * the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively. + */ + public float getHbaseIndexQPSFraction() { + return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION_PROP)); + } + + /** + * This should be same across various jobs. This is intended to limit the aggregate + * QPS generated across various Hoodie jobs to an Hbase Region Server + */ + public int getHbaseIndexMaxQPSPerRegionServer() { + return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP)); } public int getBloomIndexParallelism() { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java index e929b81d0dcf6..c706858822907 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java @@ -28,12 +28,14 @@ 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.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.Iterator; import java.util.LinkedList; @@ -47,10 +49,12 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.Bytes; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -61,19 +65,44 @@ */ public class HBaseIndex extends HoodieIndex { + public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = + "spark.executor.instances"; + public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = + "spark.dynamicAllocation.enabled"; + public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME = + "spark.dynamicAllocation.maxExecutors"; + private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s"); private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts"); private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name"); private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path"); + private static final int SLEEP_TIME_MILLISECONDS = 100; private static Logger logger = LogManager.getLogger(HBaseIndex.class); private static Connection hbaseConnection = null; + private float qpsFraction; + private int maxQpsPerRegionServer; + /** + * multiPutBatchSize will be computed and re-set in updateLocation if + * {@link HoodieIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP} is set to true + */ + private Integer multiPutBatchSize; + private Integer numRegionServersForTable; private final String tableName; + private HbasePutBatchSizeCalculator putBatchSizeCalculator; public HBaseIndex(HoodieWriteConfig config) { super(config); this.tableName = config.getHbaseTableName(); addShutDownHook(); + init(config); + } + + private void init(HoodieWriteConfig config) { + multiPutBatchSize = config.getHbaseIndexGetBatchSize(); + qpsFraction = config.getHbaseIndexQPSFraction(); + maxQpsPerRegionServer = config.getHbaseIndexMaxQPSPerRegionServer(); + putBatchSizeCalculator = new HbasePutBatchSizeCalculator(); } @Override @@ -162,7 +191,7 @@ Iterator>> locationTagFunction(HoodieTableMetaClient metaClient) // iterator till we reach batch size if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) { // get results for batch from Hbase - Result[] results = hTable.get(statements); + Result[] results = doGet(hTable, statements); // clear statements to be GC'd statements.clear(); for (Result result : results) { @@ -211,6 +240,11 @@ Iterator>> locationTagFunction(HoodieTableMetaClient metaClient) }; } + private Result[] doGet(HTable hTable, List keys) throws IOException { + sleepForTime(SLEEP_TIME_MILLISECONDS); + return hTable.get(keys); + } + @Override public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { @@ -218,9 +252,8 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, } private Function2, Iterator> updateLocationFunction() { - return (Function2, Iterator>) (partition, - statusIterator) -> { - Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); + + return (Function2, Iterator>) (partition, statusIterator) -> { List writeStatusList = new ArrayList<>(); // Grab the global HBase connection @@ -303,14 +336,131 @@ private void doPutsAndDeletes(HTable hTable, List puts, List delete hTable.flushCommits(); puts.clear(); deletes.clear(); + sleepForTime(SLEEP_TIME_MILLISECONDS); + } + + private static void sleepForTime(int sleepTimeMs) { + try { + Thread.sleep(sleepTimeMs); + } catch (InterruptedException e) { + logger.error("Sleep interrupted during throttling", e); + throw new RuntimeException(e); + } } @Override public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, HoodieTable hoodieTable) { + setPutBatchSize(writeStatusRDD, jsc); return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); } + private void setPutBatchSize(JavaRDD writeStatusRDD, + final JavaSparkContext jsc) { + if (config.getHbaseIndexPutBatchSizeAutoCompute()) { + SparkConf conf = jsc.getConf(); + int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1); + if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) { + maxExecutors = Math.max(maxExecutors, conf.getInt( + DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME, 1)); + } + + /* + Each writeStatus represents status information from a write done in one of the IOHandles. + If a writeStatus has any insert, it implies that the corresponding task contacts HBase for + doing puts, since we only do puts for inserts from HBaseIndex. + */ + int hbasePutAccessParallelism = getHBasePutAccessParallelism(writeStatusRDD); + multiPutBatchSize = putBatchSizeCalculator + .getBatchSize( + getNumRegionServersAliveForTable(), + maxQpsPerRegionServer, + hbasePutAccessParallelism, + maxExecutors, + SLEEP_TIME_MILLISECONDS, + qpsFraction); + } + } + + @VisibleForTesting + public int getHBasePutAccessParallelism(final JavaRDD writeStatusRDD) { + return Math.toIntExact(Math.max(writeStatusRDD + .filter(w -> w.getStat().getNumInserts() > 0).count(), 1)); + } + + public static class HbasePutBatchSizeCalculator implements Serializable { + + private static final int MILLI_SECONDS_IN_A_SECOND = 1000; + private static Logger logger = LogManager.getLogger(HbasePutBatchSizeCalculator.class); + + /** + * Calculate putBatch size so that sum of requests across multiple jobs in a second does not exceed + * maxQpsPerRegionServer for each Region Server. Multiplying qpsFraction to reduce the aggregate load on common RS + * across topics. Assumption here is that all tables have regions across all RS, which is not necessarily true for + * smaller tables. So, they end up getting a smaller share of QPS than they deserve, but it might be ok. + *

+ * Example: int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f) + *

+ *

+ * Expected batchSize is 8 because in that case, total request sent to a Region Server in one second is: + * + * 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) => + * 16000. We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 requests which + * happens to be 10% of 16667 (maxQPSPerRegionServer), as expected. + *

+ *

Assumptions made here

  • In a batch, writes get evenly distributed to each RS for that + * table. Since we do writes only in the case of inserts and not updates, for this assumption to fail, inserts would + * have to be skewed towards few RS, likelihood of which is less if Hbase table is pre-split and rowKeys are UUIDs + * (random strings). If this assumption fails, then it is possible for some RS to receive more than + * maxQpsPerRegionServer QPS, but for simplicity, we are going ahead with this model, since this is meant to be a + * lightweight distributed throttling mechanism without maintaining a global context. So if this assumption breaks, + * we are hoping the HBase Master relocates hot-spot regions to new Region Servers. + * + *
  • For Region Server stability, throttling at a second level granularity is fine. + * Although, within a second, the sum of queries might be within maxQpsPerRegionServer, there could be peaks at some + * sub second intervals. So, the assumption is that these peaks are tolerated by the Region Server (which at max can + * be maxQpsPerRegionServer).
  • + */ + public int getBatchSize(int numRegionServersForTable, int maxQpsPerRegionServer, + int numTasksDuringPut, int maxExecutors, int sleepTimeMs, float qpsFraction) { + int numRSAlive = numRegionServersForTable; + int maxReqPerSec = (int) (qpsFraction * numRSAlive * maxQpsPerRegionServer); + int numTasks = numTasksDuringPut; + int maxParallelPuts = Math.max(1, Math.min(numTasks, maxExecutors)); + int maxReqsSentPerTaskPerSec = MILLI_SECONDS_IN_A_SECOND / sleepTimeMs; + int multiPutBatchSize = Math.max(1, maxReqPerSec / (maxParallelPuts * maxReqsSentPerTaskPerSec)); + logger.info("HbaseIndexThrottling: qpsFraction :" + qpsFraction); + logger.info("HbaseIndexThrottling: numRSAlive :" + numRSAlive); + logger.info("HbaseIndexThrottling: maxReqPerSec :" + maxReqPerSec); + logger.info("HbaseIndexThrottling: numTasks :" + numTasks); + logger.info("HbaseIndexThrottling: maxExecutors :" + maxExecutors); + logger.info("HbaseIndexThrottling: maxParallelPuts :" + maxParallelPuts); + logger.info("HbaseIndexThrottling: maxReqsSentPerTaskPerSec :" + maxReqsSentPerTaskPerSec); + logger.info("HbaseIndexThrottling: numRegionServersForTable :" + numRegionServersForTable); + logger.info("HbaseIndexThrottling: multiPutBatchSize :" + multiPutBatchSize); + return multiPutBatchSize; + } + } + + private Integer getNumRegionServersAliveForTable() { + // This is being called in the driver, so there is only one connection + // from the driver, so ok to use a local connection variable. + if (numRegionServersForTable == null) { + try (Connection conn = getHBaseConnection()) { + RegionLocator regionLocator = conn + .getRegionLocator(TableName.valueOf(tableName)); + numRegionServersForTable = Math.toIntExact( + regionLocator.getAllRegionLocations().stream().map(e -> e.getServerName()).distinct() + .count()); + return numRegionServersForTable; + } catch (IOException e) { + logger.error(e); + throw new RuntimeException(e); + } + } + return numRegionServersForTable; + } + @Override public boolean rollbackCommit(String commitTime) { // Rollback in HbaseIndex is managed via method {@link #checkIfValidCommit()} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java index 6f681564bc3f9..bde2bf6ad1f73 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -27,14 +27,18 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieHBaseIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.hbase.HBaseIndex; +import com.uber.hoodie.index.hbase.HBaseIndex.HbasePutBatchSizeCalculator; import com.uber.hoodie.table.HoodieTable; import java.io.File; +import java.util.Arrays; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -51,6 +55,7 @@ import org.apache.spark.api.java.JavaSparkContext; import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.FixMethodOrder; @@ -281,6 +286,66 @@ public void testTotalPutsBatching() throws Exception { Mockito.verify(table, atMost(numberOfDataFileIds)).put((List) anyObject()); } + @Test + public void testPutBatchSizeCalculation() { + HbasePutBatchSizeCalculator batchSizeCalculator = new HbasePutBatchSizeCalculator(); + + // All asserts cases below are derived out of the first + // example below, with change in one parameter at a time. + + int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f); + // Expected batchSize is 8 because in that case, total request sent in one second is below + // 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) => 16000 + // We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 request + // 1600 happens to be 10% of 16667 (maxQPSPerRegionServer) as expected. + Assert.assertEquals(putBatchSize, 8); + + // Number of Region Servers are halved, total requests sent in a second are also halved, so batchSize is also halved + int putBatchSize2 = batchSizeCalculator.getBatchSize(5, 16667, 1200, 200, 100, 0.1f); + Assert.assertEquals(putBatchSize2, 4); + + // If the parallelism is halved, batchSize has to double + int putBatchSize3 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 100, 100, 0.1f); + Assert.assertEquals(putBatchSize3, 16); + + // If the parallelism is halved, batchSize has to double. + // This time parallelism is driven by numTasks rather than numExecutors + int putBatchSize4 = batchSizeCalculator.getBatchSize(10, 16667, 100, 200, 100, 0.1f); + Assert.assertEquals(putBatchSize4, 16); + + // If sleepTimeMs is halved, batchSize has to halve + int putBatchSize5 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.05f); + Assert.assertEquals(putBatchSize5, 4); + + // If maxQPSPerRegionServer is doubled, batchSize also doubles + int putBatchSize6 = batchSizeCalculator.getBatchSize(10, 33334, 1200, 200, 100, 0.1f); + Assert.assertEquals(putBatchSize6, 16); + } + + @Test + public void testsHBasePutAccessParallelism() { + HoodieWriteConfig config = getConfig(); + HBaseIndex index = new HBaseIndex(config); + final JavaRDD writeStatusRDD = jsc.parallelize( + Arrays.asList( + getSampleWriteStatus(1, 2), + getSampleWriteStatus(0, 3), + getSampleWriteStatus(10, 0)), + 10); + final int hbasePutAccessParallelism = index.getHBasePutAccessParallelism(writeStatusRDD); + Assert.assertEquals(10, writeStatusRDD.getNumPartitions()); + Assert.assertEquals(2, hbasePutAccessParallelism); + } + + private WriteStatus getSampleWriteStatus(final int numInserts, final int numUpdateWrites) { + final WriteStatus writeStatus = new WriteStatus(); + HoodieWriteStat hoodieWriteStat = new HoodieWriteStat(); + hoodieWriteStat.setNumInserts(numInserts); + hoodieWriteStat.setNumUpdateWrites(numUpdateWrites); + writeStatus.setStat(hoodieWriteStat); + return writeStatus; + } + private void assertNoWriteErrors(List statuses) { // Verify there are no errors for (WriteStatus status : statuses) { @@ -300,8 +365,11 @@ private HoodieWriteConfig.Builder getConfigBuilder() { .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) .forTable("test-trip-table").withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE) - .hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort"))) - .hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName) - .hbaseIndexGetBatchSize(100).hbaseIndexPutBatchSize(100).build()); + .withHBaseIndexConfig( + new HoodieHBaseIndexConfig.Builder() + .hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort"))) + .hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName) + .hbaseIndexGetBatchSize(100).build()) + .build()); } -} \ No newline at end of file +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java index 002c1e4351ca2..f7ec0d826a61c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java @@ -20,6 +20,7 @@ import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.config.HoodieHBaseIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.bloom.HoodieBloomIndex; @@ -63,7 +64,8 @@ public void testCreateIndex() throws Exception { HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); // Different types HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig( - indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build(); + indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE) + .withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()).build(); assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex); config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); From 6a652eb0fa0a856d95642455dff13997f3460cc4 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 22 Aug 2018 21:11:43 -0700 Subject: [PATCH 080/374] Update Release notes for 0.4.3 release --- RELEASE_NOTES.md | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index c8d0564550942..4fffd75b2002e 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -1,3 +1,25 @@ +Release 0.4.3 +------------------------------------ + +### Highlights + * Ability to run compactions asynchrously & in-parallel to ingestion/write added!!! + * Day based compaction does not respect IO budgets i.e agnostic of them + * Adds ability to throttle writes to HBase via the HBaseIndex + * (Merge on read) Inserts are sent to log files, if they are indexable. + +### Full PR List + + * **@n3nash** - Adding ability for inserts to be written to log files #400 + * **@n3nash** - Fixing bug introducted in rollback for MOR table type with inserts into log files #417 + * **@n3nash** - Changing Day based compaction strategy to be IO agnostic #398 + * **@ovj** - Changing access level to protected so that subclasses can access it #421 + * **@n3nash** - Fixing missing hoodie record location in HoodieRecord when record is read from disk after being spilled #419 + * **@bvaradar** - Async compaction - Single Consolidated PR #404 + * **@bvaradar** - BUGFIX - Use Guava Optional (which is Serializable) in CompactionOperation to avoid NoSerializableException #435 + * **@n3nash** - Adding another metric to HoodieWriteStat #434 + * **@n3nash** - Fixing Null pointer exception in finally block #440 + * **@kaushikd49** - Throttling to limit QPS from HbaseIndex #427 + Release 0.4.2 ------------------------------------ From f6a9fe5266f7d768b3f4294fa9c95c29e36e5071 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 22 Aug 2018 21:29:53 -0700 Subject: [PATCH 081/374] [maven-release-plugin] prepare release hoodie-0.4.3 --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- pom.xml | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index c6fe41fe10e65..1c8d99d437358 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index 6dff57c98247a..1abd989a923d4 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index bbf8447112410..69321fbc464b2 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index da902e883290a..6fbaa285cc127 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index cf5a979ebc941..b504d96799715 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index b9aa9d81c53d8..a78a3257a57b3 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 5aa00f1ac0d22..174f65512f502 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.3 4.0.0 diff --git a/pom.xml b/pom.xml index 570a76924cd65..ce7ccc6fa9ad1 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.3-SNAPSHOT + 0.4.3 Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - HEAD + hoodie-0.4.3 From 5ac4b5305fdd1d2dffba060f3d763f39e68ba08c Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 22 Aug 2018 21:30:05 -0700 Subject: [PATCH 082/374] [maven-release-plugin] prepare for next development iteration --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- pom.xml | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 1c8d99d437358..c696972f0bc71 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index 1abd989a923d4..7b0f7ab57c1d7 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 69321fbc464b2..0276263e3eabe 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 6fbaa285cc127..fca84ad4d7ba8 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index b504d96799715..cbaedb9fd5529 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index a78a3257a57b3..330f8547f6f05 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 174f65512f502..0f28561d7cfef 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3 + 0.4.4-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index ce7ccc6fa9ad1..0fd1ef5239011 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.3 + 0.4.4-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - hoodie-0.4.3 + HEAD From 7f279f9af4e42b8fb807a9f053a533a693643784 Mon Sep 17 00:00:00 2001 From: Saravanan Elumalai Date: Wed, 29 Aug 2018 23:42:06 +0530 Subject: [PATCH 083/374] Updated jcommander version to fix NPE in HoodieDeltaStreamer tool --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0fd1ef5239011..f0e960fa17e3b 100644 --- a/pom.xml +++ b/pom.xml @@ -362,7 +362,7 @@ com.beust jcommander - 1.48 + 1.72 From 983bf15a6bea421c3cca8725a4c437e4d7c3a7ab Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 4 Sep 2018 02:05:15 -0700 Subject: [PATCH 084/374] Removing dependency on apache-commons lang 3, adding necessary classes as needed --- .../compact/HoodieRealtimeTableCompactor.java | 2 +- .../com/uber/hoodie/TestAsyncCompaction.java | 2 +- .../java/com/uber/hoodie/TestCleaner.java | 2 +- hoodie-common/pom.xml | 4 - .../hoodie/common/table/HoodieTimeline.java | 2 +- .../table/log/block/HoodieDeleteBlock.java | 4 +- .../table/timeline/HoodieActiveTimeline.java | 5 +- .../table/view/HoodieTableFileSystemView.java | 2 +- .../hoodie/common/util/CompactionUtils.java | 2 +- .../common/util/SerializationUtils.java | 171 +++++++++++++++++ .../uber/hoodie/common/util/StringUtils.java | 51 +++++ .../common/util/collection/ImmutablePair.java | 109 +++++++++++ .../util/collection/ImmutableTriple.java | 112 +++++++++++ .../hoodie/common/util/collection/Pair.java | 179 ++++++++++++++++++ .../hoodie/common/util/collection/Triple.java | 162 ++++++++++++++++ .../converter/HoodieRecordConverter.java | 12 +- .../util/queue/BoundedInMemoryExecutor.java | 4 +- ...java => HoodieSerializationException.java} | 10 +- .../hoodie/common/model/HoodieTestUtils.java | 2 +- .../view/HoodieTableFileSystemViewTest.java | 2 +- .../common/util/TestCompactionUtils.java | 2 +- hoodie-spark/pom.xml | 5 - pom.xml | 5 - 23 files changed, 812 insertions(+), 39 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/SerializationUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutablePair.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutableTriple.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Pair.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Triple.java rename hoodie-common/src/main/java/com/uber/hoodie/exception/{HoodieNotSerializableException.java => HoodieSerializationException.java} (76%) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 3659437c6abce..4d47e76055a1e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.table.HoodieCopyOnWriteTable; @@ -49,7 +50,6 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.avro.Schema; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java index f1a1b8ea8e8a9..26fddc17a6065 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; @@ -49,7 +50,6 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.spark.api.java.JavaRDD; import org.junit.Test; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 2408333be8fc5..0a669683c9507 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -46,6 +46,7 @@ import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; @@ -64,7 +65,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 0276263e3eabe..1e2f4fec4ee75 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -110,10 +110,6 @@ hadoop-common tests - - org.apache.commons - commons-lang3 - com.esotericsoftware kryo diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java index 7adc9a1641ee1..4cd8ba7099dfe 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java @@ -19,11 +19,11 @@ import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.StringUtils; import java.io.Serializable; import java.util.Optional; import java.util.function.BiPredicate; import java.util.stream.Stream; -import org.apache.commons.lang3.StringUtils; /** * HoodieTimeline is a view of meta-data instants in the hoodie dataset. Instants are specific diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index 2f4ed7421f0c0..eddae761d0366 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -18,6 +18,7 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.storage.SizeAwareDataInputStream; +import com.uber.hoodie.common.util.StringUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -28,7 +29,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FSDataInputStream; /** @@ -64,7 +64,7 @@ public byte[] getContentBytes() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); - byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',') + byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ",") .getBytes(Charset.forName("utf-8")); output.writeInt(HoodieLogBlock.version); output.writeInt(bytesToWrite.length); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index f2c1f34328c81..3640ad6363bb9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -24,6 +24,7 @@ import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; +import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; import java.util.HashSet; @@ -32,7 +33,6 @@ import java.util.function.Function; import java.util.stream.Stream; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.time.FastDateFormat; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -49,8 +49,7 @@ */ public class HoodieActiveTimeline extends HoodieDefaultTimeline { - public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat - .getInstance("yyyyMMddHHmmss"); + public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index edcaa8261256d..d33667e319ff6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -28,6 +28,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; @@ -42,7 +43,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java index 2ee99eb8dc0e5..6ce2aa8d395ab 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -23,6 +23,7 @@ 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 com.uber.hoodie.exception.HoodieException; import java.io.IOException; import java.util.HashMap; @@ -32,7 +33,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.commons.lang3.tuple.Pair; /** * Helper class to generate compaction plan from FileGroup/FileSlice abstraction diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SerializationUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SerializationUtils.java new file mode 100644 index 0000000000000..c52a666747138 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SerializationUtils.java @@ -0,0 +1,171 @@ +/* + * 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.common.util; + +import com.uber.hoodie.exception.HoodieSerializationException; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * (NOTE: Adapted from Apache commons-lang3) + * This class defines API's to serde an object. + */ +public class SerializationUtils { + // Serialize + //----------------------------------------------------------------------- + + /** + *

    Serializes an {@code Object} to the specified stream.

    + * + *

    The stream will be closed once the object is written. + * This avoids the need for a finally clause, and maybe also exception + * handling, in the application code.

    + * + *

    The stream passed in is not buffered internally within this method. + * This is the responsibility of your application if desired.

    + * + * @param obj the object to serialize to bytes, may be null + * @param outputStream the stream to write to, must not be null + * @throws IllegalArgumentException if {@code outputStream} is {@code null} + * @throws HoodieSerializationException (runtime) if the serialization fails + */ + public static void serialize(final Serializable obj, final OutputStream outputStream) { + if (outputStream == null) { + throw new IllegalArgumentException("The OutputStream must not be null"); + } + ObjectOutputStream out = null; + try { + // stream closed in the finally + out = new ObjectOutputStream(outputStream); + out.writeObject(obj); + + } catch (final IOException ex) { + throw new HoodieSerializationException("unable to serialize object", ex); + } finally { + try { + if (out != null) { + out.close(); + } + } catch (final IOException ex) { // NOPMD + // ignore close exception + } + } + } + + /** + *

    Serializes an {@code Object} to a byte array for + * storage/serialization.

    + * + * @param obj the object to serialize to bytes + * @return a byte[] with the converted Serializable + * @throws HoodieSerializationException (runtime) if the serialization fails + */ + public static byte[] serialize(final Serializable obj) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(512); + serialize(obj, baos); + return baos.toByteArray(); + } + + // Deserialize + //----------------------------------------------------------------------- + + /** + *

    + * Deserializes an {@code Object} from the specified stream. + *

    + * + *

    + * The stream will be closed once the object is written. This avoids the need for a finally clause, and maybe also + * exception handling, in the application code. + *

    + * + *

    + * The stream passed in is not buffered internally within this method. This is the responsibility of your + * application if desired. + *

    + * + *

    + * If the call site incorrectly types the return value, a {@link ClassCastException} is thrown from the call site. + * Without Generics in this declaration, the call site must type cast and can cause the same ClassCastException. + * Note that in both cases, the ClassCastException is in the call site, not in this method. + *

    + * + * @param the object type to be deserialized + * @param inputStream the serialized object input stream, must not be null + * @return the deserialized object + * @throws IllegalArgumentException if {@code inputStream} is {@code null} + * @throws HoodieSerializationException (runtime) if the serialization fails + */ + public static T deserialize(final InputStream inputStream) { + if (inputStream == null) { + throw new IllegalArgumentException("The InputStream must not be null"); + } + ObjectInputStream in = null; + try { + // stream closed in the finally + in = new ObjectInputStream(inputStream); + @SuppressWarnings("unchecked") // may fail with CCE if serialised form is incorrect + final T obj = (T) in.readObject(); + return obj; + + } catch (final ClassCastException ex) { + throw new HoodieSerializationException("cannot cast class", ex); + } catch (final ClassNotFoundException ex) { + throw new HoodieSerializationException("class not found", ex); + } catch (final IOException ex) { + throw new HoodieSerializationException("unable to deserialize to object", ex); + } finally { + try { + if (in != null) { + in.close(); + } + } catch (final IOException ex) { // NOPMD + // ignore close exception + } + } + } + + /** + *

    + * Deserializes a single {@code Object} from an array of bytes. + *

    + * + *

    + * If the call site incorrectly types the return value, a {@link ClassCastException} is thrown from the call site. + * Without Generics in this declaration, the call site must type cast and can cause the same ClassCastException. + * Note that in both cases, the ClassCastException is in the call site, not in this method. + *

    + * + * @param the object type to be deserialized + * @param objectData the serialized object, must not be null + * @return the deserialized object + * @throws IllegalArgumentException if {@code objectData} is {@code null} + * @throws HoodieSerializationException (runtime) if the serialization fails + */ + public static T deserialize(final byte[] objectData) { + if (objectData == null) { + throw new IllegalArgumentException("The byte[] must not be null"); + } + return deserialize(new ByteArrayInputStream(objectData)); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java new file mode 100644 index 0000000000000..e9921a210fb34 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java @@ -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.common.util; + +/** + * Simple utility for operations on strings + */ +public class StringUtils { + + /** + *

    Joins the elements of the provided array into a single String + * containing the provided list of elements.

    + * + *

    No separator is added to the joined String. + * Null objects or empty strings within the array are represented by + * empty strings.

    + * + *
    +   * StringUtils.join(null)            = null
    +   * StringUtils.join([])              = ""
    +   * StringUtils.join([null])          = ""
    +   * StringUtils.join(["a", "b", "c"]) = "abc"
    +   * StringUtils.join([null, "", "a"]) = "a"
    +   * 
    + */ + public static String join(final String... elements) { + return join(elements, ""); + } + + public static String join(final String[] array, final String separator) { + if (array == null) { + return null; + } + return org.apache.hadoop.util.StringUtils.join(separator, array); + } + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutablePair.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutablePair.java new file mode 100644 index 0000000000000..d2d4df07b6d5b --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutablePair.java @@ -0,0 +1,109 @@ +/* + * 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.common.util.collection; + +/** + * (NOTE: Adapted from Apache commons-lang3) + *

    An immutable pair consisting of two {@code Object} elements.

    + * + *

    Although the implementation is immutable, there is no restriction on the objects + * that may be stored. If mutable objects are stored in the pair, then the pair + * itself effectively becomes mutable. The class is also {@code final}, so a subclass + * can not add undesirable behaviour.

    + * + *

    #ThreadSafe# if both paired objects are thread-safe

    + * + * @param the left element type + * @param the right element type + */ +public final class ImmutablePair extends Pair { + + /** + * Serialization version + */ + private static final long serialVersionUID = 4954918890077093841L; + + /** + * Left object + */ + public final L left; + /** + * Right object + */ + public final R right; + + /** + *

    Obtains an immutable pair of from two objects inferring the generic types.

    + * + *

    This factory allows the pair to be created using inference to + * obtain the generic types.

    + * + * @param the left element type + * @param the right element type + * @param left the left element, may be null + * @param right the right element, may be null + * @return a pair formed from the two parameters, not null + */ + public static ImmutablePair of(final L left, final R right) { + return new ImmutablePair(left, right); + } + + /** + * Create a new pair instance. + * + * @param left the left value, may be null + * @param right the right value, may be null + */ + public ImmutablePair(final L left, final R right) { + super(); + this.left = left; + this.right = right; + } + + //----------------------------------------------------------------------- + + /** + * {@inheritDoc} + */ + @Override + public L getLeft() { + return left; + } + + /** + * {@inheritDoc} + */ + @Override + public R getRight() { + return right; + } + + /** + *

    Throws {@code UnsupportedOperationException}.

    + * + *

    This pair is immutable, so this operation is not supported.

    + * + * @param value the value to set + * @return never + * @throws UnsupportedOperationException as this operation is not supported + */ + @Override + public R setValue(final R value) { + throw new UnsupportedOperationException(); + } + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutableTriple.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutableTriple.java new file mode 100644 index 0000000000000..f986cc9f61550 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ImmutableTriple.java @@ -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.common.util.collection; + +/** + * (NOTE: Adapted from Apache commons-lang3) + *

    An immutable triple consisting of three {@code Object} elements.

    + * + *

    Although the implementation is immutable, there is no restriction on the objects + * that may be stored. If mutable objects are stored in the triple, then the triple + * itself effectively becomes mutable. The class is also {@code final}, so a subclass + * can not add undesirable behaviour.

    + * + *

    #ThreadSafe# if all three objects are thread-safe

    + * + * @param the left element type + * @param the middle element type + * @param the right element type + */ +public final class ImmutableTriple extends Triple { + + /** + * Serialization version + */ + private static final long serialVersionUID = 1L; + + /** + * Left object + */ + public final L left; + /** + * Middle object + */ + public final M middle; + /** + * Right object + */ + public final R right; + + /** + *

    Obtains an immutable triple of from three objects inferring the generic types.

    + * + *

    This factory allows the triple to be created using inference to + * obtain the generic types.

    + * + * @param the left element type + * @param the middle element type + * @param the right element type + * @param left the left element, may be null + * @param middle the middle element, may be null + * @param right the right element, may be null + * @return a triple formed from the three parameters, not null + */ + public static ImmutableTriple of(final L left, final M middle, final R right) { + return new ImmutableTriple(left, middle, right); + } + + /** + * Create a new triple instance. + * + * @param left the left value, may be null + * @param middle the middle value, may be null + * @param right the right value, may be null + */ + public ImmutableTriple(final L left, final M middle, final R right) { + super(); + this.left = left; + this.middle = middle; + this.right = right; + } + + //----------------------------------------------------------------------- + + /** + * {@inheritDoc} + */ + @Override + public L getLeft() { + return left; + } + + /** + * {@inheritDoc} + */ + @Override + public M getMiddle() { + return middle; + } + + /** + * {@inheritDoc} + */ + @Override + public R getRight() { + return right; + } +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Pair.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Pair.java new file mode 100644 index 0000000000000..60d26fe61cba0 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Pair.java @@ -0,0 +1,179 @@ +/* + * 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.common.util.collection; + +import java.io.Serializable; +import java.util.Map; +import org.apache.commons.lang.builder.CompareToBuilder; + +/** + * (NOTE: Adapted from Apache commons-lang3) + *

    A pair consisting of two elements.

    + * + *

    This class is an abstract implementation defining the basic API. + * It refers to the elements as 'left' and 'right'. It also implements the + * {@code Map.Entry} interface where the key is 'left' and the value is 'right'.

    + * + *

    Subclass implementations may be mutable or immutable. + * However, there is no restriction on the type of the stored objects that may be stored. + * If mutable objects are stored in the pair, then the pair itself effectively becomes mutable.

    + * + * @param the left element type + * @param the right element type + */ +public abstract class Pair implements Map.Entry, Comparable>, Serializable { + + /** + * Serialization version + */ + private static final long serialVersionUID = 4954918890077093841L; + + /** + *

    Obtains an immutable pair of from two objects inferring the generic types.

    + * + *

    This factory allows the pair to be created using inference to + * obtain the generic types.

    + * + * @param the left element type + * @param the right element type + * @param left the left element, may be null + * @param right the right element, may be null + * @return a pair formed from the two parameters, not null + */ + public static Pair of(final L left, final R right) { + return new ImmutablePair(left, right); + } + + //----------------------------------------------------------------------- + + /** + *

    Gets the left element from this pair.

    + * + *

    When treated as a key-value pair, this is the key.

    + * + * @return the left element, may be null + */ + public abstract L getLeft(); + + /** + *

    Gets the right element from this pair.

    + * + *

    When treated as a key-value pair, this is the value.

    + * + * @return the right element, may be null + */ + public abstract R getRight(); + + /** + *

    Gets the key from this pair.

    + * + *

    This method implements the {@code Map.Entry} interface returning the + * left element as the key.

    + * + * @return the left element as the key, may be null + */ + @Override + public final L getKey() { + return getLeft(); + } + + /** + *

    Gets the value from this pair.

    + * + *

    This method implements the {@code Map.Entry} interface returning the + * right element as the value.

    + * + * @return the right element as the value, may be null + */ + @Override + public R getValue() { + return getRight(); + } + + //----------------------------------------------------------------------- + + /** + *

    Compares the pair based on the left element followed by the right element. + * The types must be {@code Comparable}.

    + * + * @param other the other pair, not null + * @return negative if this is less, zero if equal, positive if greater + */ + @Override + public int compareTo(final Pair other) { + return new CompareToBuilder().append(getLeft(), other.getLeft()) + .append(getRight(), other.getRight()).toComparison(); + } + + /** + *

    Compares this pair to another based on the two elements.

    + * + * @param obj the object to compare to, null returns false + * @return true if the elements of the pair are equal + */ + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof Map.Entry) { + final Map.Entry other = (Map.Entry) obj; + return getKey().equals(other.getKey()) + && getValue().equals(other.getValue()); + } + return false; + } + + /** + *

    Returns a suitable hash code. + * The hash code follows the definition in {@code Map.Entry}.

    + * + * @return the hash code + */ + @Override + public int hashCode() { + // see Map.Entry API specification + return (getKey() == null ? 0 : getKey().hashCode()) + ^ (getValue() == null ? 0 : getValue().hashCode()); + } + + /** + *

    Returns a String representation of this pair using the format {@code ($left,$right)}.

    + * + * @return a string describing this object, not null + */ + @Override + public String toString() { + return new StringBuilder().append('(').append(getLeft()).append(',').append(getRight()).append(')').toString(); + } + + /** + *

    Formats the receiver using the given format.

    + * + *

    This uses {@link java.util.Formattable} to perform the formatting. Two variables may + * be used to embed the left and right elements. Use {@code %1$s} for the left + * element (key) and {@code %2$s} for the right element (value). + * The default format used by {@code toString()} is {@code (%1$s,%2$s)}.

    + * + * @param format the format string, optionally containing {@code %1$s} and {@code %2$s}, not null + * @return the formatted string, not null + */ + public String toString(final String format) { + return String.format(format, getLeft(), getRight()); + } + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Triple.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Triple.java new file mode 100644 index 0000000000000..eeec5fd49cade --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/Triple.java @@ -0,0 +1,162 @@ +/* + * 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.common.util.collection; + +import java.io.Serializable; +import org.apache.commons.lang.builder.CompareToBuilder; + +/** + * (NOTE: Adapted from Apache commons-lang3) + *

    A triple consisting of three elements.

    + * + *

    This class is an abstract implementation defining the basic API. + * It refers to the elements as 'left', 'middle' and 'right'.

    + * + *

    Subclass implementations may be mutable or immutable. + * However, there is no restriction on the type of the stored objects that may be stored. + * If mutable objects are stored in the triple, then the triple itself effectively becomes mutable.

    + * + * @param the left element type + * @param the middle element type + * @param the right element type + */ +public abstract class Triple implements Comparable>, Serializable { + + /** + * Serialization version + */ + private static final long serialVersionUID = 1L; + + /** + *

    Obtains an immutable triple of from three objects inferring the generic types.

    + * + *

    This factory allows the triple to be created using inference to + * obtain the generic types.

    + * + * @param the left element type + * @param the middle element type + * @param the right element type + * @param left the left element, may be null + * @param middle the middle element, may be null + * @param right the right element, may be null + * @return a triple formed from the three parameters, not null + */ + public static Triple of(final L left, final M middle, final R right) { + return new ImmutableTriple(left, middle, right); + } + + //----------------------------------------------------------------------- + + /** + *

    Gets the left element from this triple.

    + * + * @return the left element, may be null + */ + public abstract L getLeft(); + + /** + *

    Gets the middle element from this triple.

    + * + * @return the middle element, may be null + */ + public abstract M getMiddle(); + + /** + *

    Gets the right element from this triple.

    + * + * @return the right element, may be null + */ + public abstract R getRight(); + + //----------------------------------------------------------------------- + + /** + *

    Compares the triple based on the left element, followed by the middle element, + * finally the right element. + * The types must be {@code Comparable}.

    + * + * @param other the other triple, not null + * @return negative if this is less, zero if equal, positive if greater + */ + @Override + public int compareTo(final Triple other) { + return new CompareToBuilder().append(getLeft(), other.getLeft()) + .append(getMiddle(), other.getMiddle()) + .append(getRight(), other.getRight()).toComparison(); + } + + /** + *

    Compares this triple to another based on the three elements.

    + * + * @param obj the object to compare to, null returns false + * @return true if the elements of the triple are equal + */ + @SuppressWarnings("deprecation") // ObjectUtils.equals(Object, Object) has been deprecated in 3.2 + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof Triple) { + final Triple other = (Triple) obj; + return getLeft().equals(other.getLeft()) + && getMiddle().equals(other.getMiddle()) + && getRight().equals(other.getRight()); + } + return false; + } + + /** + *

    Returns a suitable hash code.

    + * + * @return the hash code + */ + @Override + public int hashCode() { + return (getLeft() == null ? 0 : getLeft().hashCode()) + ^ (getMiddle() == null ? 0 : getMiddle().hashCode()) + ^ (getRight() == null ? 0 : getRight().hashCode()); + } + + /** + *

    Returns a String representation of this triple using the format {@code ($left,$middle,$right)}.

    + * + * @return a string describing this object, not null + */ + @Override + public String toString() { + return new StringBuilder().append('(').append(getLeft()).append(',').append(getMiddle()).append(',') + .append(getRight()).append(')').toString(); + } + + /** + *

    Formats the receiver using the given format.

    + * + *

    This uses {@link java.util.Formattable} to perform the formatting. Three variables may + * be used to embed the left and right elements. Use {@code %1$s} for the left + * element, {@code %2$s} for the middle and {@code %3$s} for the right element. + * The default format used by {@code toString()} is {@code (%1$s,%2$s,%3$s)}.

    + * + * @param format the format string, optionally containing {@code %1$s}, {@code %2$s} and {@code %3$s}, not null + * @return the formatted string, not null + */ + public String toString(final String format) { + return String.format(format, getLeft(), getMiddle(), getRight()); + } + +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java index 6aac7b365a475..1f5ad6b1f9f58 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -22,14 +22,14 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.ReflectionUtils; -import com.uber.hoodie.exception.HoodieNotSerializableException; +import com.uber.hoodie.common.util.SerializationUtils; +import com.uber.hoodie.common.util.collection.Pair; +import com.uber.hoodie.common.util.collection.Triple; +import com.uber.hoodie.exception.HoodieSerializationException; import java.io.IOException; import java.util.Optional; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.lang3.SerializationUtils; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.commons.lang3.tuple.Triple; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -71,7 +71,7 @@ public byte[] getBytes(HoodieRecord hoodieRecord) { hoodieRecord.getKey().getPartitionPath()), Pair.of(currentLocation, newLocation), val); return SerializationUtils.serialize(data); } catch (IOException io) { - throw new HoodieNotSerializableException("Cannot serialize value to bytes", io); + throw new HoodieSerializationException("Cannot serialize value to bytes", io); } } @@ -103,7 +103,7 @@ public HoodieRecord getData(byte[] bytes) { hoodieRecord.setNewLocation(newLocation); return hoodieRecord; } catch (IOException io) { - throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io); + throw new HoodieSerializationException("Cannot de-serialize value from bytes", io); } } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java index 1e7664f481c08..add92005f4de3 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; @@ -30,7 +31,6 @@ import java.util.concurrent.Future; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.commons.lang3.concurrent.ConcurrentUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -130,7 +130,7 @@ private Future startConsumer() { throw e; } }); - }).orElse(ConcurrentUtils.constantFuture(null)); + }).orElse(CompletableFuture.completedFuture(null)); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieSerializationException.java similarity index 76% rename from hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java rename to hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieSerializationException.java index 17650c79d4b76..fc09a3cc721e4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieNotSerializableException.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieSerializationException.java @@ -18,16 +18,20 @@ import java.io.IOException; -public class HoodieNotSerializableException extends HoodieException { +public class HoodieSerializationException extends HoodieException { private IOException ioException; - public HoodieNotSerializableException(String msg, IOException t) { + public HoodieSerializationException(String msg, IOException t) { super(msg, t); this.ioException = t; } - public HoodieNotSerializableException(String msg) { + public HoodieSerializationException(String msg, Exception t) { + super(msg, t); + } + + public HoodieSerializationException(String msg) { super(msg); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 475cc4f804f82..62877bd2c0c89 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -43,6 +43,7 @@ import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.collection.Pair; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; @@ -65,7 +66,6 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 3035712ea07b5..b650695f8c66f 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -37,6 +37,7 @@ import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -48,7 +49,6 @@ import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.junit.Before; diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java index 65edb2de5be3d..7557b50878dd5 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Arrays; @@ -42,7 +43,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.junit.Assert; import org.junit.Before; diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 330f8547f6f05..3942dac3ea73f 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -177,11 +177,6 @@ avro
    - - org.apache.commons - commons-lang3 - - org.apache.commons commons-configuration2 diff --git a/pom.xml b/pom.xml index f0e960fa17e3b..9f3394a7272e9 100644 --- a/pom.xml +++ b/pom.xml @@ -585,11 +585,6 @@ hive-metastore ${hive.version}-cdh${cdh.version} - - org.apache.commons - commons-lang3 - 3.4 - junit From 219035e142fe0c5fd2894f7e16855a1e7587820b Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Mon, 11 Jun 2018 20:27:56 -0700 Subject: [PATCH 085/374] 1. Small file size handling for inserts into log files. In summary, the total size of the log file is compared with the parquet max file size and if there is scope to add inserts the add it. --- .../hoodie/cli/commands/CommitsCommand.java | 9 +- .../hoodie/cli/commands/StatsCommand.java | 3 +- .../com/uber/hoodie/cli/utils/CommitUtil.java | 3 +- .../com/uber/hoodie/HoodieWriteClient.java | 45 ++- .../uber/hoodie/index/InMemoryHashIndex.java | 6 +- .../uber/hoodie/io/HoodieAppendHandle.java | 13 + .../hoodie/io/HoodieCommitArchiveLog.java | 8 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 17 +- .../hoodie/table/HoodieMergeOnReadTable.java | 301 ++++++++++++------ .../java/com/uber/hoodie/TestCleaner.java | 2 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 82 ++++- .../hoodie/common/HoodieClientTestUtils.java | 3 +- .../hoodie/table/TestMergeOnReadTable.java | 210 ++++++++++++ .../src/main/avro/HoodieCommitMetadata.avsc | 12 + .../common/model/HoodieCommitMetadata.java | 16 +- .../common/model/HoodieRollingStat.java | 98 ++++++ .../model/HoodieRollingStatMetadata.java | 101 ++++++ .../model/TestHoodieCommitMetadata.java | 3 +- .../uber/hoodie/hive/HoodieHiveClient.java | 8 +- .../com/uber/hoodie/IncrementalRelation.scala | 5 +- .../deltastreamer/HoodieDeltaStreamer.java | 2 +- 21 files changed, 824 insertions(+), 123 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index fe6a7736947aa..a3c201c78de2b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -82,7 +82,8 @@ public String showCommits(@CliOption(key = { Collections.reverse(commits); for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(), + HoodieCommitMetadata.class); rows.add(new Comparable[]{commit.getTimestamp(), commitMetadata.fetchTotalBytesWritten(), commitMetadata.fetchTotalFilesInsert(), @@ -160,7 +161,8 @@ public String showCommitPartitions( if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } - HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(), + HoodieCommitMetadata.class); List rows = new ArrayList<>(); for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); @@ -221,7 +223,8 @@ public String showCommitFiles( if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } - HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(), + HoodieCommitMetadata.class); List rows = new ArrayList<>(); for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java index eeb6bff5c21ad..212e2d2773720 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java @@ -75,7 +75,8 @@ public String writeAmplificationStats( DecimalFormat df = new DecimalFormat("#.00"); for (HoodieInstant commitTime : timeline.getInstants().collect(Collectors.toList())) { String waf = "0"; - HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get()); + HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get(), + HoodieCommitMetadata.class); if (commit.fetchTotalUpdateRecordsWritten() > 0) { waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten()); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java index 55c2471a9ae76..7be09d171cff3 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java @@ -30,7 +30,8 @@ public static long countNewRecords(HoodieTableMetaClient target, List co HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants(); for (String commit : commitsToCatchup) { HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes( - timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get()); + timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get(), + HoodieCommitMetadata.class); totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten(); } return totalNew; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 84bceac825c69..846db2b40ba2c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -32,6 +32,8 @@ 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.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -501,9 +503,8 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, writeStatus.getPartitionPath(), writeStatus.getStat())).collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - for (Tuple2 stat : stats) { - metadata.addWriteStat(stat._1(), stat._2()); - } + updateMetadataAndRollingStats(actionType, metadata, stats); + // Finalize write final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); @@ -1256,4 +1257,42 @@ private Optional forceCompact(Optional> extraMetadat }); return compactionInstantTimeOpt; } + + private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetadata metadata, List> stats) { + // TODO : make sure we cannot rollback / archive last commit file + try { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + // 0. All of the rolling stat management is only done by the DELTA commit for MOR and COMMIT for COW other wise + // there may be race conditions + HoodieRollingStatMetadata rollingStatMetadata = new HoodieRollingStatMetadata(actionType); + // 1. Look up the previous compaction/commit and get the HoodieCommitMetadata from there. + // 2. Now, first read the existing rolling stats and merge with the result of current metadata. + + // Need to do this on every commit (delta or commit) to support COW and MOR. + for (Tuple2 stat : stats) { + metadata.addWriteStat(stat._1(), stat._2()); + HoodieRollingStat hoodieRollingStat = new HoodieRollingStat(stat._2().getFileId(), + stat._2().getNumWrites() - (stat._2().getNumUpdateWrites() - stat._2.getNumDeletes()), + stat._2().getNumUpdateWrites(), stat._2.getNumDeletes(), stat._2().getTotalWriteBytes()); + rollingStatMetadata.addRollingStat(stat._1, hoodieRollingStat); + } + // The last rolling stat should be present in the completed timeline + Optional lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .lastInstant(); + if (lastInstant.isPresent()) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(table.getActiveTimeline().getInstantDetails(lastInstant + .get()).get(), HoodieCommitMetadata.class); + rollingStatMetadata = rollingStatMetadata + .merge(HoodieCommitMetadata.fromBytes(commitMetadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class)); + } + metadata.addMetadata(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, rollingStatMetadata.toJsonString()); + } catch (IOException io) { + throw new HoodieCommitException("Unable to save rolling stats"); + } + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java index a3d0429ea6719..3e4833964eeb6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java @@ -45,7 +45,11 @@ public class InMemoryHashIndex extends HoodieInde public InMemoryHashIndex(HoodieWriteConfig config) { super(config); - recordLocationMap = new ConcurrentHashMap<>(); + synchronized (InMemoryHashIndex.class) { + if (recordLocationMap == null) { + recordLocationMap = new ConcurrentHashMap<>(); + } + } } @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index fa7d857d6f20e..9f4f0924d9a61 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -75,6 +75,8 @@ public class HoodieAppendHandle extends HoodieIOH private long recordsWritten = 0; // Total number of records deleted during an append private long recordsDeleted = 0; + // Total number of records updated during an append + private long updatedRecordsWritten = 0; // Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk private long averageRecordSize = 0; private HoodieLogFile currentLogFile; @@ -89,6 +91,8 @@ public class HoodieAppendHandle extends HoodieIOH private int maxBlockSize = config.getLogFileDataBlockMaxSize(); // Header metadata for a log block private Map header = Maps.newHashMap(); + // Total number of new records inserted into the delta file + private long insertRecordsWritten = 0; public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String fileId, Iterator> recordItr) { @@ -111,6 +115,7 @@ private void init(HoodieRecord record) { // extract some information from the first record Optional fileSlice = fileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst(); + // Set the base commit time as the current commitTime for new inserts into log files String baseInstantTime = commitTime; if (fileSlice.isPresent()) { baseInstantTime = fileSlice.get().getBaseInstantTime(); @@ -156,6 +161,12 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { hoodieRecord.getPartitionPath(), fileId); HoodieAvroUtils .addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId); + // If currentLocation is present, then this is an update + if (hoodieRecord.getCurrentLocation() != null) { + updatedRecordsWritten++; + } else { + insertRecordsWritten++; + } recordsWritten++; } else { recordsDeleted++; @@ -238,6 +249,8 @@ public WriteStatus close() { writeStatus.getStat().setPrevCommit(commitTime); writeStatus.getStat().setFileId(this.fileId); writeStatus.getStat().setNumWrites(recordsWritten); + writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten); + writeStatus.getStat().setNumInserts(insertRecordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten); writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index bf5febd1a2689..a65008b4c2a8a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -31,6 +31,7 @@ import com.uber.hoodie.common.model.ActionType; import com.uber.hoodie.common.model.HoodieArchivedLogFile; import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; @@ -55,6 +56,7 @@ import java.util.stream.Stream; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -273,7 +275,7 @@ private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, } case COMMIT_ACTION: { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get()); + .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; @@ -294,7 +296,7 @@ private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, } case HoodieTimeline.DELTA_COMMIT_ACTION: { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get()); + .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; @@ -312,6 +314,8 @@ private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter( mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper .convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class); + // Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer + avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, StringUtils.EMPTY); return avroMetaData; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index fa180dba4611b..1f9ea2f05377d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -28,6 +28,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -643,12 +644,18 @@ class UpsertPartitioner extends Partitioner { */ private HashMap bucketInfoMap; + /** + * Rolling stats for files + */ + protected HoodieRollingStatMetadata rollingStatMetadata; + protected long averageRecordSize; + UpsertPartitioner(WorkloadProfile profile) { updateLocationToBucket = new HashMap<>(); partitionPathToInsertBuckets = new HashMap<>(); bucketInfoMap = new HashMap<>(); globalStat = profile.getGlobalStat(); - + rollingStatMetadata = getRollingStats(); assignUpdates(profile); assignInserts(profile); @@ -792,7 +799,7 @@ protected List getSmallFiles(String partitionPath) { * Obtains the average record size based on records written during last commit. Used for * estimating how many records pack into one file. */ - private long averageBytesPerRecord() { + protected long averageBytesPerRecord() { long avgSize = 0L; HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline() .filterCompletedInstants(); @@ -800,7 +807,7 @@ private long averageBytesPerRecord() { if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(latestCommitTime).get()); + .fromBytes(commitTimeline.getInstantDetails(latestCommitTime).get(), HoodieCommitMetadata.class); avgSize = (long) Math.ceil( (1.0 * commitMetadata.fetchTotalBytesWritten()) / commitMetadata .fetchTotalRecordsWritten()); @@ -852,4 +859,8 @@ public int getPartition(Object key) { } } } + + protected HoodieRollingStatMetadata getRollingStats() { + return null; + } } \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 078d23ad07f8e..b7d7a5080eeec 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -16,6 +16,7 @@ package com.uber.hoodie.table; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.uber.hoodie.WriteStatus; @@ -27,10 +28,11 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; -import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum; import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; @@ -40,6 +42,7 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; +import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.func.MergeOnReadLazyInsertIterable; @@ -103,7 +106,7 @@ public Iterator> handleUpdate(String commitTime, String fileId Iterator> recordItr) throws IOException { logger.info("Merging updates for commit " + commitTime + " for file " + fileId); - if (mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { + if (!index.canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { logger.info( "Small file corrections for updates for commit " + commitTime + " for file " + fileId); return super.handleUpdate(commitTime, fileId, recordItr); @@ -188,8 +191,6 @@ public List rollback(JavaSparkContext jsc, List comm .filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight); logger.info("Unpublished " + commits); Long startTime = System.currentTimeMillis(); - // TODO (NA) : remove this once HoodieIndex is a member of HoodieTable - HoodieIndex hoodieIndex = HoodieIndex.createIndex(config, jsc); List allRollbackStats = jsc.parallelize(FSUtils .getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) @@ -224,90 +225,53 @@ public List rollback(JavaSparkContext jsc, List comm throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } case HoodieTimeline.DELTA_COMMIT_ACTION: + // -------------------------------------------------------------------------------------------------- + // (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal + // -------------------------------------------------------------------------------------------------- + // (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In + // this scenario we would want to delete these log files. + // (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario, + // HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks. + // (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is + // being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and + // and hence will end up deleting these log files. This is done so there are no orphan log files + // lying around. + // (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions + // taken in this scenario is a combination of (A.2) and (A.3) + // --------------------------------------------------------------------------------------------------- + // (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal + // --------------------------------------------------------------------------------------------------- + // (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries. + // In this scenario, we delete all the parquet files written for the failed commit. + // (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In + // this scenario, perform (A.1) and for updates written to log files, write rollback blocks. + // (B.3) Rollback triggered for first commit - Same as (B.1) + // (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files + // as well if the base parquet file gets deleted. try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( metaClient.getCommitTimeline().getInstantDetails( new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) - .get()); + .get(), HoodieCommitMetadata.class); // read commit file and (either append delete blocks or delete file) final Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); // In case all data was inserts and the commit failed, delete the file belonging to that commit + // We do not know fileIds for inserts (first inserts are either log files or parquet files), + // delete all files for the corresponding failed commit, if present (same as COW) super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); - final Set deletedFiles = filesToDeletedStatus.entrySet().stream() .map(entry -> { Path filePath = entry.getKey().getPath(); return FSUtils.getFileIdFromFilePath(filePath); }).collect(Collectors.toSet()); + // append rollback blocks for updates if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { - // This needs to be done since GlobalIndex at the moment does not store the latest commit time - Map fileIdToLatestCommitTimeMap = - hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime)) : null; - commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() - .filter(wStat -> { - if (wStat != null - && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT - && wStat.getPrevCommit() != null && !deletedFiles.contains(wStat.getFileId())) { - return true; - } - return false; - }) - .forEach(wStat -> { - Writer writer = null; - String baseCommitTime = wStat.getPrevCommit(); - if (hoodieIndex.isGlobal()) { - baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId()); - } - try { - // TODO : wStat.getPrevCommit() might not give the right commit time in the following - // scenario if a compaction was scheduled, the new commitTime will be used to write the - // new log file. In this case, the commit time for the log file is the - // getBaseCommitTime() - writer = HoodieLogFormat.newWriterBuilder().onParentPath( - new Path(this.getMetaClient().getBasePath(), partitionPath)) - .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) - .withFs(this.metaClient.getFs()) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - Long numRollbackBlocks = 0L; - // generate metadata - Map header = - Maps.newHashMap(); - header.put(HeaderMetadataType.INSTANT_TIME, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - header.put(HeaderMetadataType.TARGET_INSTANT_TIME, - commit); - header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String - .valueOf( - HoodieCommandBlockTypeEnum - .ROLLBACK_PREVIOUS_BLOCK - .ordinal())); - // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock(header)); - numRollbackBlocks++; - filesToNumBlocksRollback.put(this.getMetaClient().getFs() - .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); - } catch (IOException | InterruptedException io) { - throw new HoodieRollbackException( - "Failed to rollback for commit " + commit, io); - } finally { - try { - if (writer != null) { - writer.close(); - } - } catch (IOException io) { - throw new UncheckedIOException(io); - } - } - }); - hoodieRollbackStats = HoodieRollbackStat.newBuilder() - .withPartitionPath(partitionPath) - .withDeletedFileResults(filesToDeletedStatus) - .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); + hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus, + filesToNumBlocksRollback, deletedFiles); } break; } catch (IOException io) { @@ -335,6 +299,25 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) return Optional.empty(); } + @Override + protected HoodieRollingStatMetadata getRollingStats() { + try { + Optional lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() + .lastInstant(); + if (lastInstant.isPresent()) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + this.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); + HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata + .fromBytes(commitMetadata.getExtraMetadata().get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY) + .getBytes(), HoodieRollingStatMetadata.class); + return rollingStatMetadata; + } + return null; + } catch (IOException e) { + throw new HoodieException(); + } + } + /** * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet * files to larger ones without the need for an index in the logFile. @@ -345,7 +328,6 @@ class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitio super(profile); } - @Override protected List getSmallFiles(String partitionPath) { // smallFiles only for partitionPath @@ -354,32 +336,59 @@ protected List getSmallFiles(String partitionPath) { // Init here since this class (and member variables) might not have been initialized HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + // Find out all eligible small file slices if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); // find smallest file in partition and append to it - - // TODO - check if index.isglobal then small files are log files too - Optional smallFileSlice = getRTFileSystemView() - // Use the merged file-slice for small file selection - .getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( - fileSlice -> fileSlice.getLogFiles().count() < 1 - && fileSlice.getDataFile().get().getFileSize() < config - .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> - left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() - ? -1 : 1).findFirst(); - - if (smallFileSlice.isPresent()) { - String filename = smallFileSlice.get().getDataFile().get().getFileName(); + List allSmallFileSlices = new ArrayList<>(); + // If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to + // it. Doing this overtime for a partition, we ensure that we handle small file issues + if (!index.canIndexLogFiles()) { + // TODO : choose last N small files since there can be multiple small files written to a single partition + // by different spark partitions in a single batch + Optional smallFileSlice = getRTFileSystemView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( + fileSlice -> fileSlice.getLogFiles().count() < 1 + && fileSlice.getDataFile().get().getFileSize() < config + .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> + left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() + ? -1 : 1).findFirst(); + if (smallFileSlice.isPresent()) { + allSmallFileSlices.add(smallFileSlice.get()); + } + } else { + // If we can index log files, we can add more inserts to log files. + List allFileSlices = getRTFileSystemView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()) + .collect(Collectors.toList()); + for (FileSlice fileSlice : allFileSlices) { + if (isSmallFile(partitionPath, fileSlice)) { + allSmallFileSlices.add(fileSlice); + } + } + } + // Create SmallFiles from the eligible file slices + for (FileSlice smallFileSlice : allSmallFileSlices) { SmallFile sf = new SmallFile(); - sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), - FSUtils.getFileId(filename)); - sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize(); - smallFileLocations.add(sf); - // Update the global small files list - smallFiles.add(sf); + if (smallFileSlice.getDataFile().isPresent()) { + // TODO : Move logic of file name, file id, base commit time handling inside file slice + String filename = smallFileSlice.getDataFile().get().getFileName(); + sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename)); + sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice); + smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); + } else { + HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get(); + sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()), + FSUtils.getFileIdFromLogPath(logFile.getPath())); + sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice); + smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); + } } } - return smallFileLocations; } @@ -388,6 +397,116 @@ public List getSmallFileIds() { .map(smallFile -> ((SmallFile) smallFile).location.getFileId()) .collect(Collectors.toList()); } + + private long getTotalFileSize(String partitionPath, FileSlice fileSlice) { + if (rollingStatMetadata != null) { + Map partitionRollingStats = + rollingStatMetadata.getPartitionToRollingStats().get(partitionPath); + if (partitionRollingStats != null) { + HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId()); + if (rollingStatForFile != null) { + long inserts = rollingStatForFile.getInserts(); + long totalSize = averageRecordSize * inserts; + return totalSize; + } + } + } + // In case Rolling Stats is not present, fall back to sizing log files based on heuristics + if (!fileSlice.getDataFile().isPresent()) { + return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList())); + } else { + return fileSlice.getDataFile().get().getFileSize() + convertLogFilesSizeToExpectedParquetSize(fileSlice + .getLogFiles().collect(Collectors.toList())); + } + } + + private boolean isSmallFile(String partitionPath, FileSlice fileSlice) { + long totalSize = getTotalFileSize(partitionPath, fileSlice); + if (totalSize < config.getParquetMaxFileSize()) { + return true; + } + return false; + } + + // TODO (NA) : Make this static part of utility + @VisibleForTesting + public long convertLogFilesSizeToExpectedParquetSize(List hoodieLogFiles) { + long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize().get()) + .reduce((a, b) -> (a + b)).orElse(0L); + // Here we assume that if there is no base parquet file, all log files contain only inserts. + // We can then just get the parquet equivalent size of these log files, compare that with + // {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows + long logFilesEquivalentParquetFileSize = (long) (totalSizeOfLogFiles * config + .getLogFileToParquetCompressionRatio()); + return logFilesEquivalentParquetFileSize; + } + } + + private Map generateHeader(String commit) { + // generate metadata + Map header = Maps.newHashMap(); + header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); + header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit); + header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK + .ordinal())); + return header; + } + + private HoodieRollbackStat rollback(HoodieIndex hoodieIndex, String partitionPath, String commit, + HoodieCommitMetadata commitMetadata, final Map filesToDeletedStatus, + Map filesToNumBlocksRollback, Set deletedFiles) { + // The following needs to be done since GlobalIndex at the moment does not store the latest commit time. + // Also, wStat.getPrevCommit() might not give the right commit time in the following + // scenario : If a compaction was scheduled, the new commitTime associated with the requested compaction will be + // used to write the new log files. In this case, the commit time for the log file is the compaction requested time. + Map fileIdToBaseCommitTimeForLogMap = + hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime)) : null; + commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() + .filter(wStat -> { + // Filter out stats without prevCommit since they are all inserts + if (wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null + && !deletedFiles.contains(wStat.getFileId())) { + return true; + } + return false; + }).forEach(wStat -> { + HoodieLogFormat.Writer writer = null; + String baseCommitTime = wStat.getPrevCommit(); + if (hoodieIndex.isGlobal()) { + baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId()); + } + try { + writer = HoodieLogFormat.newWriterBuilder().onParentPath( + new Path(this.getMetaClient().getBasePath(), partitionPath)) + .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) + .withFs(this.metaClient.getFs()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + Long numRollbackBlocks = 0L; + // generate metadata + Map header = generateHeader(commit); + // if update belongs to an existing log file + writer = writer.appendBlock(new HoodieCommandBlock(header)); + numRollbackBlocks++; + filesToNumBlocksRollback.put(this.getMetaClient().getFs() + .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); + } catch (IOException | InterruptedException io) { + throw new HoodieRollbackException( + "Failed to rollback for commit " + commit, io); + } finally { + try { + if (writer != null) { + writer.close(); + } + } catch (IOException io) { + throw new UncheckedIOException(io); + } + } + }); + return HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath) + .withDeletedFileResults(filesToDeletedStatus) + .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 0a669683c9507..6b5440b3d5c26 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -251,7 +251,7 @@ private void testInsertAndCleanByVersions( HashMap> fileIdToVersions = new HashMap<>(); for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(entry).get()); + .fromBytes(timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class); for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) { if (!fileIdToVersions.containsKey(wstat.getFileId())) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 9fbd0871a41d8..09b8795ab4309 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -28,6 +28,8 @@ import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -46,6 +48,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -54,6 +57,7 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; import org.junit.Test; import scala.Option; @@ -563,7 +567,7 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); + .fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class); String basePath = table.getMetaClient().getBasePath(); Collection commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values(); @@ -571,7 +575,8 @@ public void testCommitWritesRelativePaths() throws Exception { String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime); FileInputStream inputStream = new FileInputStream(filename); String everything = IOUtils.toString(inputStream); - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString()); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), + HoodieCommitMetadata.class); HashMap paths = metadata.getFileIdAndFullPaths(basePath); inputStream.close(); @@ -581,6 +586,79 @@ public void testCommitWritesRelativePaths() throws Exception { } } + /** + * Test to ensure commit metadata points to valid files + */ + @Test + public void testRollingStatsInMetadata() throws Exception { + + HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + + String commitTime = "000"; + client.startCommitWithTime(commitTime); + + List records = dataGen.generateInserts(commitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + JavaRDD result = client.bulkInsert(writeRecords, commitTime); + + assertTrue("Commit should succeed", client.commit(commitTime, result)); + assertTrue("After explicit commit, commit file should be created", + HoodieTestUtils.doesCommitExist(basePath, commitTime)); + + // Read from commit file + String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime); + FileInputStream inputStream = new FileInputStream(filename); + String everything = IOUtils.toString(inputStream); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), + HoodieCommitMetadata.class); + HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class); + int inserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + } + } + Assert.assertEquals(inserts, 200); + + // Update + Inserts such that they just expand file1 + commitTime = "001"; + client.startCommitWithTime(commitTime); + + records = dataGen.generateUpdates(commitTime, records); + writeRecords = jsc.parallelize(records, 1); + result = client.upsert(writeRecords, commitTime); + + assertTrue("Commit should succeed", client.commit(commitTime, result)); + assertTrue("After explicit commit, commit file should be created", + HoodieTestUtils.doesCommitExist(basePath, commitTime)); + + // Read from commit file + filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime); + inputStream = new FileInputStream(filename); + everything = IOUtils.toString(inputStream); + metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class); + rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class); + inserts = 0; + int upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + Assert.assertEquals(inserts, 200); + Assert.assertEquals(upserts, 200); + + } + /** * Build Hoodie Write Config for small data file sizes */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index a85bd3c23bff1..cc1dad8e18a34 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -109,7 +109,8 @@ public static HashMap getLatestFileIDsToFullPath(String basePath List commitsToReturn) throws IOException { HashMap fileIdToFullPath = new HashMap<>(); for (HoodieInstant commit : commitsToReturn) { - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get()); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(), + HoodieCommitMetadata.class); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath)); } return fileIdToFullPath; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index f5c22c4422b46..aa27440433aed 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -31,9 +31,12 @@ import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus; import com.uber.hoodie.common.minicluster.HdfsTestService; import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -896,6 +899,213 @@ public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exc } } + /** + * Test to ensure rolling stats are correctly written to metadata file + */ + @Test + public void testRollingStatsInMetadata() throws Exception { + + HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + + String commitTime = "000"; + client.startCommitWithTime(commitTime); + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + List records = dataGen.generateInserts(commitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + JavaRDD statuses = client.insert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + int inserts = 0; + for (Map.Entry> pstat : + rollingStatMetadata.getPartitionToRollingStats().entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + } + } + Assert.assertEquals(inserts, 200); + + commitTime = "001"; + client.startCommitWithTime(commitTime); + records = dataGen.generateUpdates(commitTime, records); + writeRecords = jsc.parallelize(records, 1); + statuses = client.upsert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + inserts = 0; + int upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + + Assert.assertEquals(inserts, 200); + Assert.assertEquals(upserts, 200); + + client.rollback(commitTime); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + inserts = 0; + upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + Assert.assertEquals(inserts, 200); + Assert.assertEquals(upserts, 0); + } + + /** + * Test to ensure rolling stats are correctly written to the metadata file, identifies small files and corrects them + */ + @Test + public void testRollingStatsWithSmallFileHandling() throws Exception { + + HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + Map fileIdToInsertsMap = new HashMap<>(); + Map fileIdToUpsertsMap = new HashMap<>(); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + + String commitTime = "000"; + client.startCommitWithTime(commitTime); + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + List records = dataGen.generateInserts(commitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + JavaRDD statuses = client.insert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + int inserts = 0; + for (Map.Entry> pstat : + rollingStatMetadata.getPartitionToRollingStats().entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + fileIdToInsertsMap.put(stat.getKey(), stat.getValue().getInserts()); + fileIdToUpsertsMap.put(stat.getKey(), stat.getValue().getUpserts()); + } + } + Assert.assertEquals(inserts, 200); + + commitTime = "001"; + client.startCommitWithTime(commitTime); + // generate updates + inserts. inserts should be handled into small files + records = dataGen.generateUpdates(commitTime, records); + records.addAll(dataGen.generateInserts(commitTime, 200)); + writeRecords = jsc.parallelize(records, 1); + statuses = client.upsert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + inserts = 0; + int upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + // No new file id should be created, all the data should be written to small files already there + assertTrue(fileIdToInsertsMap.containsKey(stat.getKey())); + assertTrue(fileIdToUpsertsMap.containsKey(stat.getKey())); + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + + Assert.assertEquals(inserts, 400); + Assert.assertEquals(upserts, 200); + + // Test small file handling after compaction + commitTime = "002"; + client.scheduleCompactionAtInstant(commitTime, Optional.of(metadata.getExtraMetadata())); + statuses = client.compact(commitTime); + client.commitCompaction(commitTime, statuses, Optional.empty()); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + HoodieRollingStatMetadata rollingStatMetadata1 = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + + // Ensure that the rolling stats from the extra metadata of delta commits is copied over to the compaction commit + for (Map.Entry> entry : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + Assert.assertTrue(rollingStatMetadata1.getPartitionToRollingStats().containsKey(entry.getKey())); + Assert.assertEquals(rollingStatMetadata1.getPartitionToRollingStats().get(entry.getKey()).size(), entry + .getValue().size()); + } + + // Write inserts + updates + commitTime = "003"; + client.startCommitWithTime(commitTime); + // generate updates + inserts. inserts should be handled into small files + records = dataGen.generateUpdates(commitTime, records); + records.addAll(dataGen.generateInserts(commitTime, 200)); + writeRecords = jsc.parallelize(records, 1); + statuses = client.upsert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), HoodieCommitMetadata.class); + rollingStatMetadata = HoodieCommitMetadata.fromBytes(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class); + inserts = 0; + upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats() + .entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + // No new file id should be created, all the data should be written to small files already there + assertTrue(fileIdToInsertsMap.containsKey(stat.getKey())); + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + + Assert.assertEquals(inserts, 600); + Assert.assertEquals(upserts, 600); + + } private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit).build(); diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index 0429cc7d38d0e..3f44732005dff 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -66,6 +66,18 @@ "name":"numInserts", "type":["null","long"], "default" : null + }, + { + "name":"totalLogBlocks", + "type":["null","long"] + }, + { + "name":"totalCorruptLogBlock", + "type":["null","long"] + }, + { + "name":"totalRollbackBlocks", + "type":["null","long"] } ] } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index fe97a4e3427a5..c75c19d770d50 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -119,12 +119,12 @@ public String toJsonString() throws IOException { return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); } - public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOException { + public static T fromJsonString(String jsonStr, Class clazz) throws Exception { if (jsonStr == null || jsonStr.isEmpty()) { // For empty commit file (no data or somethings bad happen). - return new HoodieCommitMetadata(); + return clazz.newInstance(); } - return getObjectMapper().readValue(jsonStr, HoodieCommitMetadata.class); + return getObjectMapper().readValue(jsonStr, clazz); } // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. @@ -319,11 +319,15 @@ public int hashCode() { return result; } - public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException { - return fromJsonString(new String(bytes, Charset.forName("utf-8"))); + public static T fromBytes(byte[] bytes, Class clazz) throws IOException { + try { + return fromJsonString(new String(bytes, Charset.forName("utf-8")), clazz); + } catch (Exception e) { + throw new IOException("unable to read commit metadata", e); + } } - private static ObjectMapper getObjectMapper() { + protected static ObjectMapper getObjectMapper() { ObjectMapper mapper = new ObjectMapper(); mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java new file mode 100644 index 0000000000000..4d51a799c3ba4 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java @@ -0,0 +1,98 @@ +/* + * 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.common.model; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import java.io.Serializable; +import javax.annotation.Nullable; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieRollingStat implements Serializable { + + private String fileId; + private long inserts; + private long upserts; + private long deletes; + // TODO + @Nullable + private long totalInputWriteBytesToDisk; + @Nullable + private long totalInputWriteBytesOnDisk; + + public HoodieRollingStat() { + // called by jackson json lib + } + + public HoodieRollingStat(String fileId, long inserts, long upserts, long deletes, long totalInputWriteBytesOnDisk) { + this.fileId = fileId; + this.inserts = inserts; + this.upserts = upserts; + this.deletes = deletes; + this.totalInputWriteBytesOnDisk = totalInputWriteBytesOnDisk; + } + + public String getFileId() { + return fileId; + } + + public void setFileId(String fileId) { + this.fileId = fileId; + } + + public long getInserts() { + return inserts; + } + + public void setInserts(long inserts) { + this.inserts = inserts; + } + + public long getUpserts() { + return upserts; + } + + public void setUpserts(long upserts) { + this.upserts = upserts; + } + + public long getDeletes() { + return deletes; + } + + public void setDeletes(long deletes) { + this.deletes = deletes; + } + + public long addInserts(long inserts) { + this.inserts += inserts; + return this.inserts; + } + + public long addUpserts(long upserts) { + this.upserts += upserts; + return this.upserts; + } + + public long addDeletes(long deletes) { + this.deletes += deletes; + return this.deletes; + } + + public long getTotalInputWriteBytesOnDisk() { + return totalInputWriteBytesOnDisk; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java new file mode 100644 index 0000000000000..ba4b1cfe911d9 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java @@ -0,0 +1,101 @@ +/* + * 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.common.model; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * This class holds statistics about files belonging to a dataset + */ +public class HoodieRollingStatMetadata implements Serializable { + + private static volatile Logger log = LogManager.getLogger(HoodieRollingStatMetadata.class); + protected Map> partitionToRollingStats; + private String actionType = "DUMMY_ACTION"; + public static final String ROLLING_STAT_METADATA_KEY = "ROLLING_STAT"; + + public void addRollingStat(String partitionPath, HoodieRollingStat stat) { + if (!partitionToRollingStats.containsKey(partitionPath)) { + partitionToRollingStats.put(partitionPath, new RollingStatsHashMap<>()); + } + partitionToRollingStats.get(partitionPath).put(stat.getFileId(), stat); + } + + public HoodieRollingStatMetadata() { + partitionToRollingStats = new HashMap<>(); + } + + public HoodieRollingStatMetadata(String actionType) { + this(); + this.actionType = actionType; + } + + class RollingStatsHashMap extends HashMap { + + @Override + public V put(K key, V value) { + V v = this.get(key); + if (v == null) { + super.put(key, value); + } else if (v instanceof HoodieRollingStat) { + long inserts = ((HoodieRollingStat) v).getInserts(); + long upserts = ((HoodieRollingStat) v).getUpserts(); + long deletes = ((HoodieRollingStat) v).getDeletes(); + ((HoodieRollingStat) value).addInserts(inserts); + ((HoodieRollingStat) value).addUpserts(upserts); + ((HoodieRollingStat) value).addDeletes(deletes); + super.put(key, value); + } + return value; + } + } + + public static HoodieRollingStatMetadata fromBytes(byte[] bytes) throws IOException { + return HoodieCommitMetadata.fromBytes(bytes, HoodieRollingStatMetadata.class); + } + + public String toJsonString() throws IOException { + if (partitionToRollingStats.containsKey(null)) { + log.info("partition path is null for " + partitionToRollingStats.get(null)); + partitionToRollingStats.remove(null); + } + return HoodieCommitMetadata.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + } + + public HoodieRollingStatMetadata merge(HoodieRollingStatMetadata rollingStatMetadata) { + for (Map.Entry> stat : rollingStatMetadata.partitionToRollingStats + .entrySet()) { + for (Map.Entry innerStat : stat.getValue().entrySet()) { + this.addRollingStat(stat.getKey(), innerStat.getValue()); + } + } + return this; + } + + public Map> getPartitionToRollingStats() { + return partitionToRollingStats; + } + + public String getActionType() { + return actionType; + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java index 5f92e83bf56e9..506416e213e04 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommitMetadata.java @@ -34,7 +34,8 @@ public void testPerfStatPresenceInHoodieMetadata() throws Exception { Assert.assertTrue(commitMetadata.getTotalLogFilesCompacted() > 0); String serializedCommitMetadata = commitMetadata.toJsonString(); - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(serializedCommitMetadata); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(serializedCommitMetadata, + HoodieCommitMetadata.class); // Make sure timing metrics are not written to instant file Assert.assertTrue(metadata.getTotalScanTime() == 0); Assert.assertTrue(metadata.getTotalLogFilesCompacted() > 0); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index c3d202b5bdd37..55d74d8f122cf 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -307,7 +307,7 @@ public MessageType getDataSchema() { HoodieInstant lastCommit = activeTimeline.lastInstant().orElseThrow( () -> new InvalidDatasetException(syncConfig.basePath)); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - activeTimeline.getInstantDetails(lastCommit).get()); + activeTimeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class); String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for commit " + lastCommit @@ -339,7 +339,7 @@ public MessageType getDataSchema() { HoodieInstant lastDeltaInstant = lastDeltaCommit.get(); // read from the log file wrote commitMetadata = HoodieCommitMetadata.fromBytes( - activeTimeline.getInstantDetails(lastDeltaInstant).get()); + activeTimeline.getInstantDetails(lastDeltaInstant).get(), HoodieCommitMetadata.class); filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() .stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION)) .findAny().orElseThrow(() -> new IllegalArgumentException( @@ -372,7 +372,7 @@ private MessageType readSchemaFromLastCompaction(Optional lastCom // Read from the compacted file wrote HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata.fromBytes( - activeTimeline.getInstantDetails(lastCompactionCommit).get()); + activeTimeline.getInstantDetails(lastCompactionCommit).get(), HoodieCommitMetadata.class); String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for compaction " + lastCompactionCommit @@ -539,7 +539,7 @@ List getPartitionsWrittenToSince(Optional lastCommitTimeSynced) Integer.MAX_VALUE); return timelineToSync.getInstants().map(s -> { try { - return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get()); + return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get(), HoodieCommitMetadata.class); } catch (IOException e) { throw new HoodieIOException( "Failed to get partitions written since " + lastCommitTimeSynced, e); diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index cace267a9bcdd..b0014d3b2fe3e 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -73,7 +73,7 @@ class IncrementalRelation(val sqlContext: SQLContext, // use schema from a file produced in the latest instant val latestSchema = { val latestMeta = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get) + .fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get, classOf[HoodieCommitMetadata]) val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next() AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema( sqlContext.sparkContext.hadoopConfiguration, new Path(metaFilePath))) @@ -84,7 +84,8 @@ class IncrementalRelation(val sqlContext: SQLContext, override def buildScan(): RDD[Row] = { val fileIdToFullPath = mutable.HashMap[String, String]() for (commit <- commitsToReturn) { - val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get) + val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit) + .get, classOf[HoodieCommitMetadata]) fileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap } val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index f4619465cae3f..7f58a721eeeae 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -187,7 +187,7 @@ private void sync() throws Exception { Optional lastCommit = commitTimelineOpt.get().lastInstant(); if (lastCommit.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get()); + commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get(), HoodieCommitMetadata.class); if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) { resumeCheckpointStr = Optional.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); } else { From e95fee808605d68666ac32d1112c5f66a60921d6 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Thu, 6 Sep 2018 08:54:32 +0800 Subject: [PATCH 086/374] Update Gemfile.lock with higher ffi version --- docs/Gemfile.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index 7a62573949228..8fac90615c295 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -19,7 +19,7 @@ GEM execjs (2.7.0) faraday (0.13.1) multipart-post (>= 1.2, < 3) - ffi (1.9.18) + ffi (1.9.24) forwardable-extended (2.6.0) gemoji (2.1.0) github-pages (106) From 4f65da6618f340fc6ae6ba6252838f6d6a0db045 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 6 Sep 2018 12:02:09 -0700 Subject: [PATCH 087/374] Simplify and fix CLI to schedule and run compactions --- hoodie-cli/hoodie-cli.sh | 2 +- .../cli/commands/CompactionCommand.java | 34 ++++++++----------- .../uber/hoodie/cli/commands/SparkMain.java | 20 +++++------ .../hoodie/utilities/HoodieCompactor.java | 11 ++---- 4 files changed, 27 insertions(+), 40 deletions(-) diff --git a/hoodie-cli/hoodie-cli.sh b/hoodie-cli/hoodie-cli.sh index 73ffe333f54db..b67a06741df20 100755 --- a/hoodie-cli/hoodie-cli.sh +++ b/hoodie-cli/hoodie-cli.sh @@ -13,4 +13,4 @@ fi if [ -z "$CLIENT_JAR" ]; then echo "client jar location not set" fi -java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HOODIE_JAR:${CLIENT_JAR} org.springframework.shell.Bootstrap +java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HOODIE_JAR:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java index b5ed2ada5c25d..b5f344e136e32 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java @@ -162,14 +162,8 @@ public String compactionShow( @CliCommand(value = "compaction schedule", help = "Schedule Compaction") public String scheduleCompact( - @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, - @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField, - @CliOption(key = { - "parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") final String parallelism, - @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String - schemaFilePath, - @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, - @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception { + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1G", help = "Spark executor memory") + final String sparkMemory) throws Exception { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); @@ -181,7 +175,7 @@ public String scheduleCompact( scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), HoodieCLI.tableMetadata.getBasePath(), - tableName, compactionInstantTime, rowKeyField, parallelism, schemaFilePath, sparkMemory, retry); + HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, sparkMemory); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); @@ -196,16 +190,15 @@ public String scheduleCompact( @CliCommand(value = "compaction run", help = "Run Compaction for given instant time") public String compact( - @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, - @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField, - @CliOption(key = { - "parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") final String parallelism, - @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String - schemaFilePath, - @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, - @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry, - @CliOption(key = "compactionInstant", mandatory = true, help = "Base path for the target hoodie dataset") final - String compactionInstantTime) throws Exception { + @CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") + final String parallelism, + @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") + final String schemaFilePath, + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G", help = "Spark executor memory") + final String sparkMemory, + @CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") final String retry, + @CliOption(key = "compactionInstant", mandatory = true, help = "Base path for the target hoodie dataset") + final String compactionInstantTime) throws Exception { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); @@ -214,7 +207,8 @@ public String compact( scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), HoodieCLI.tableMetadata.getBasePath(), - tableName, compactionInstantTime, rowKeyField, parallelism, schemaFilePath, sparkMemory, retry); + HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, parallelism, schemaFilePath, + sparkMemory, retry); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index 526e6462064ad..bb9189ccb1d17 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -23,6 +23,7 @@ import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; import com.uber.hoodie.utilities.HDFSParquetImporter; import com.uber.hoodie.utilities.HoodieCompactor; import org.apache.log4j.Logger; @@ -68,14 +69,14 @@ public static void main(String[] args) throws Exception { Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10])); break; case COMPACT_RUN: - assert (args.length == 9); - returnCode = compact(jsc, args[1], args[2], args[3], args[4], args[5], Integer.parseInt(args[6]), - args[7], args[8], Integer.parseInt(args[9]), false); + assert (args.length == 8); + returnCode = compact(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), + args[5], args[6], Integer.parseInt(args[7]), false); break; case COMPACT_SCHEDULE: - assert (args.length == 10); - returnCode = compact(jsc, args[1], args[2], args[3], args[4], args[5], Integer.parseInt(args[6]), - args[7], args[8], Integer.parseInt(args[9]), true); + assert (args.length == 5); + returnCode = compact(jsc, args[1], args[2], args[3], 1, + "", args[4], 0, true); break; default: break; @@ -103,14 +104,13 @@ private static int dataLoad(JavaSparkContext jsc, String command, } private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant, - String rowKey, String partitionKey, int parallelism, String schemaFile, - String sparkMemory, int retry, boolean schedule) throws Exception { + int parallelism, String schemaFile, String sparkMemory, int retry, boolean schedule) throws Exception { HoodieCompactor.Config cfg = new HoodieCompactor.Config(); cfg.basePath = basePath; cfg.tableName = tableName; cfg.compactionInstantTime = compactionInstant; - cfg.rowKey = rowKey; - cfg.partitionKey = partitionKey; + // TODO: Make this configurable along with strategy specific config - For now, this is a generic enough strategy + cfg.strategyClassName = UnBoundedCompactionStrategy.class.getCanonicalName(); cfg.parallelism = parallelism; cfg.schemaFile = schemaFile; cfg.runSchedule = schedule; diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java index d986a6ac4e8f7..d327e6d0d0b3a 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java @@ -34,7 +34,7 @@ public class HoodieCompactor { - private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class); + private static volatile Logger logger = LogManager.getLogger(HoodieCompactor.class); private final Config cfg; private transient FileSystem fs; @@ -51,12 +51,6 @@ public static class Config implements Serializable { @Parameter(names = {"--instant-time", "-sp"}, description = "Compaction Instant time", required = true) public String compactionInstantTime = null; - @Parameter(names = {"--row-key-field", - "-rk"}, description = "Row key field name", required = true) - public String rowKey = null; - @Parameter(names = {"--partition-key-field", - "-pk"}, description = "Partition key field name", required = true) - public String partitionKey = null; @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = true) public int parallelism = 1; @@ -120,8 +114,7 @@ private int doCompact(JavaSparkContext jsc) throws Exception { private int doSchedule(JavaSparkContext jsc) throws Exception { //Get schema. - String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile); - HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, + HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, Optional.of(cfg.strategyClassName)); client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Optional.empty()); return 0; From ba6c141127beeec08523b2878266dbcffa8c9933 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 7 Sep 2018 00:17:49 -0700 Subject: [PATCH 088/374] Fix a failing test case intermittenly in TestMergeOnRead due to incorrect prev commit time --- .../src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java | 1 - .../test/java/com/uber/hoodie/table/TestMergeOnReadTable.java | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 9f4f0924d9a61..c60384475b2c8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -246,7 +246,6 @@ public WriteStatus close() { if (writer != null) { writer.close(); } - writeStatus.getStat().setPrevCommit(commitTime); writeStatus.getStat().setFileId(this.fileId); writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index aa27440433aed..92c40e7e899cc 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -488,6 +488,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); + writeRecords = jsc.parallelize(copyOfRecords, 1); writeStatusJavaRDD = client.upsert(writeRecords, commitTime2); client.commit(commitTime2, writeStatusJavaRDD); statuses = writeStatusJavaRDD.collect(); From aabc2cb14a8c4c0ee8f7949547444589fb4c77cc Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 6 Sep 2018 10:24:32 -0700 Subject: [PATCH 089/374] CLI to create and desc hoodie table --- .../cli/commands/CompactionCommand.java | 7 ++ .../hoodie/cli/commands/DatasetsCommand.java | 71 +++++++++++++++++++ .../common/table/HoodieTableConfig.java | 7 ++ 3 files changed, 85 insertions(+) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java index b5f344e136e32..866e9315b418a 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java @@ -45,6 +45,7 @@ import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.util.Utils; import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliOption; import org.springframework.stereotype.Component; @@ -54,6 +55,12 @@ public class CompactionCommand implements CommandMarker { private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class); + @CliAvailabilityIndicator({"compactions show all", "compaction show", "compaction run", "compaction schedule"}) + public boolean isAvailable() { + return (HoodieCLI.tableMetadata != null) + && (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ); + } + @CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline") public String compactionsAll( @CliOption(key = { diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java index 1bcdfa5bc7574..e370f5b429821 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -17,9 +17,17 @@ package com.uber.hoodie.cli.commands; import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; +import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.exception.DatasetNotFoundException; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliOption; import org.springframework.stereotype.Component; @@ -37,4 +45,67 @@ public String connect( HoodieCLI.state = HoodieCLI.CLIState.DATASET; return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; } + + /** + * Create a Hoodie Table if it does not exist + * + * @param path Base Path + * @param name Hoodie Table Name + * @param tableTypeStr Hoodie Table Type + * @param payloadClass Payload Class + */ + @CliCommand(value = "create", help = "Create a hoodie table if not present") + public String createTable( + @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path, + @CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name, + @CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE", + help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr, + @CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "com.uber.hoodie.common.model.HoodieAvroPayload", + help = "Payload Class") final String payloadClass) throws IOException { + + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + boolean existing = false; + try { + new HoodieTableMetaClient(HoodieCLI.conf, path); + existing = true; + } catch (DatasetNotFoundException dfe) { + // expected + } + + // Do not touch table that already exist + if (existing) { + throw new IllegalStateException("Dataset already existing in path : " + path); + } + + final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr); + HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass); + + // Now connect to ensure loading works + return connect(path); + } + + @CliAvailabilityIndicator({"desc"}) + public boolean isDescAvailable() { + return HoodieCLI.tableMetadata != null; + } + + /** + * Describes table properties + */ + @CliCommand(value = "desc", help = "Describle Hoodie Table properties") + public String descTable() { + TableHeader header = new TableHeader() + .addTableHeaderField("Property") + .addTableHeaderField("Value"); + List rows = new ArrayList<>(); + rows.add(new Comparable[]{"basePath", HoodieCLI.tableMetadata.getBasePath()}); + rows.add(new Comparable[]{"metaPath", HoodieCLI.tableMetadata.getMetaPath()}); + rows.add(new Comparable[]{"fileSystem", HoodieCLI.tableMetadata.getFs().getScheme()}); + HoodieCLI.tableMetadata.getTableConfig().getProps().entrySet().forEach(e -> { + rows.add(new Comparable[]{e.getKey(), e.getValue()}); + }); + return HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 56921fb0ca2e5..007a0f825ed41 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -23,7 +23,9 @@ import java.io.IOException; import java.io.Serializable; import java.util.Date; +import java.util.Map; import java.util.Properties; +import java.util.stream.Collectors; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -172,4 +174,9 @@ public HoodieFileFormat getRTFileFormat() { public String getArchivelogFolder() { return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); } + + public Map getProps() { + return props.entrySet().stream().collect( + Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue()))); + } } From 8366e8f27d5dca94bcaf578f3b0cb020a38931c8 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Sat, 4 Aug 2018 03:35:30 -0700 Subject: [PATCH 090/374] Reworking the deltastreamer tool - Standardize version of jackson - DFSPropertiesConfiguration replaces usage of commons PropertiesConfiguration - Remove dependency on ConstructorUtils - Throw error if ordering value is not present, during key generation - Switch to shade plugin for hoodie-utilities - Added support for consumption for Confluent avro kafka serdes - Support for Confluent schema registry - KafkaSource now deals with skews nicely, by doing round robin allocation of source limit across partitions - Added support for BULK_INSERT operations as well - Pass in the payload class config properly into HoodieWriteClient - Fix documentation based on new usage - Adding tests on deltastreamer, sources and all new util classes. --- docs/comparison.md | 2 +- docs/incremental_processing.md | 183 +++++++------- .../commands/HDFSParquetImportCommand.java | 8 +- .../common/HoodieTestDataGenerator.java | 11 +- hoodie-common/pom.xml | 1 + .../util/DFSPropertiesConfiguration.java | 102 ++++++++ .../hoodie/common/util/ReflectionUtils.java | 53 ++-- .../hoodie/common/util/TypedProperties.java | 87 +++++++ .../util/TestDFSPropertiesConfiguration.java | 138 ++++++++++ hoodie-spark/pom.xml | 5 +- .../java/com/uber/hoodie/BaseAvroPayload.java | 4 + .../java/com/uber/hoodie/DataSourceUtils.java | 17 +- .../java/com/uber/hoodie/KeyGenerator.java | 6 +- .../com/uber/hoodie/SimpleKeyGenerator.java | 25 +- .../scala/com/uber/hoodie/DefaultSource.scala | 12 +- .../test/scala/DataSourceDefaultsTest.scala | 32 ++- hoodie-utilities/pom.xml | 122 +++++---- hoodie-utilities/src/assembly/src.xml | 50 ---- .../hoodie/utilities/HDFSParquetImporter.java | 21 +- .../utilities/HiveIncrementalPuller.java | 9 +- .../hoodie/utilities/HoodieCompactor.java | 5 +- .../uber/hoodie/utilities/UtilHelpers.java | 78 +++--- .../deltastreamer/HoodieDeltaStreamer.java | 239 +++++------------- .../keygen/TimestampBasedKeyGenerator.java | 4 +- .../schema/FilebasedSchemaProvider.java | 43 ++-- .../utilities/schema/SchemaProvider.java | 17 +- .../schema/SchemaRegistryProvider.java | 71 ++++++ .../utilities/sources/AvroConvertor.java | 6 +- .../utilities/sources/AvroDFSSource.java | 47 ++++ .../utilities/sources/AvroKafkaSource.java | 47 ++++ .../hoodie/utilities/sources/DFSSource.java | 66 ++--- .../utilities/sources/HiveIncrPullSource.java | 32 ++- ...urceDataFormat.java => JsonDFSSource.java} | 23 +- .../utilities/sources/JsonKafkaSource.java | 45 ++++ .../hoodie/utilities/sources/KafkaSource.java | 158 ++++++------ .../uber/hoodie/utilities/sources/Source.java | 23 +- .../key-generator.properties | 19 -- .../delta-streamer-config/target.avsc | 29 --- .../utilities/TestHoodieDeltaStreamer.java | 162 ++++++++++++ .../hoodie/utilities/UtilitiesTestBase.java | 137 ++++++++++ .../utilities/sources/TestDFSSource.java | 104 ++++++++ .../utilities/sources/TestDataSource.java | 99 ++++++++ .../utilities/sources/TestKafkaSource.java | 186 ++++++++++++++ .../resources/IncrementalPull.sqltemplate | 0 .../delta-streamer-config/base.properties} | 3 + .../dfs-source.properties} | 13 +- .../kafka-source.properties} | 13 +- .../delta-streamer-config/source.avsc | 6 +- pom.xml | 86 ++++++- 49 files changed, 1907 insertions(+), 742 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/TestDFSPropertiesConfiguration.java delete mode 100644 hoodie-utilities/src/assembly/src.xml create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java rename hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/{SourceDataFormat.java => JsonDFSSource.java} (51%) create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java delete mode 100644 hoodie-utilities/src/main/resources/delta-streamer-config/key-generator.properties delete mode 100644 hoodie-utilities/src/main/resources/delta-streamer-config/target.avsc create mode 100644 hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java create mode 100644 hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java create mode 100644 hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java create mode 100644 hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java create mode 100644 hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java rename hoodie-utilities/src/{main => test}/resources/IncrementalPull.sqltemplate (100%) rename hoodie-utilities/src/{main/resources/delta-streamer-config/hoodie-client.properties => test/resources/delta-streamer-config/base.properties} (86%) rename hoodie-utilities/src/{main/resources/delta-streamer-config/schema-provider.properties => test/resources/delta-streamer-config/dfs-source.properties} (60%) rename hoodie-utilities/src/{main/resources/delta-streamer-config/source.properties => test/resources/delta-streamer-config/kafka-source.properties} (60%) rename hoodie-utilities/src/{main => test}/resources/delta-streamer-config/source.avsc (86%) diff --git a/docs/comparison.md b/docs/comparison.md index b5aa0d1d047ec..0120bcb2e7df8 100644 --- a/docs/comparison.md +++ b/docs/comparison.md @@ -47,7 +47,7 @@ just for analytics. Finally, HBase does not support incremental processing primi A popular question, we get is : "How does Hoodie relate to stream processing systems?", which we will try to answer here. Simply put, Hoodie can integrate with batch (`copy-on-write storage`) and streaming (`merge-on-read storage`) jobs of today, to store the computed results in Hadoop. For Spark apps, this can happen via direct integration of Hoodie library with Spark/Spark streaming DAGs. In case of Non-Spark processing systems (eg: Flink, Hive), the processing can be done in the respective systems -and later sent into a Hoodie table via a Kafka topic/HDFS intermediate file. (TODO: Need link to SQLStreamer/DeltaStreamer after reworking). In more conceptual level, data processing +and later sent into a Hoodie table via a Kafka topic/HDFS intermediate file. In more conceptual level, data processing pipelines just consist of three components : `source`, `processing`, `sink`, with users ultimately running queries against the sink to use the results of the pipeline. Hoodie can act as either a source or sink, that stores data on HDFS. Applicability of Hoodie to a given stream processing pipeline ultimately boils down to suitability of Presto/SparkSQL/Hive for your queries. diff --git a/docs/incremental_processing.md b/docs/incremental_processing.md index 753dd2044c259..3ea9b82de06b0 100644 --- a/docs/incremental_processing.md +++ b/docs/incremental_processing.md @@ -4,45 +4,22 @@ keywords: incremental processing sidebar: mydoc_sidebar permalink: incremental_processing.html toc: false -summary: In this page, we will discuss incremental processing primitives that Hoodie has to offer. +summary: In this page, we will discuss some available tools for ingesting data incrementally & consuming the changes. --- As discussed in the concepts section, the two basic primitives needed for [incrementally processing](https://www.oreilly.com/ideas/ubers-case-for-incremental-processing-on-hadoop), data using Hoodie are `upserts` (to apply changes to a dataset) and `incremental pulls` (to obtain a change stream/log from a dataset). This section discusses a few tools that can be used to achieve these on different contexts. -{% include callout.html content="Instructions are currently only for Copy-on-write storage. When merge-on-read storage is added, these tools would be revised to add that support" type="info" %} +## Ingesting/Writing Data - -## Upserts - -Upserts can be used to apply a delta or an incremental change to a Hoodie dataset. For e.g, the incremental changes could be from a Kafka topic or files uploaded to HDFS or +Following means can be used to apply a delta or an incremental change to a Hoodie dataset. For e.g, the incremental changes could be from a Kafka topic or files uploaded to HDFS or even changes pulled from another Hoodie dataset. -#### Via Spark Job - -The `hoodie-spark` module offers the DataSource API to write any data frame into a Hoodie dataset. Following is how we can upsert a dataframe, while specifying the field names that need to be used -for `recordKey => _row_key`, `partitionPath => partition` and `precombineKey => timestamp` - - -``` -inputDF.write() - .format("com.uber.hoodie") - .options(clientOpts) // any of the hoodie client opts can be passed in as well - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") - .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") - .option(HoodieWriteConfig.TABLE_NAME, tableName) - .mode(SaveMode.Append) - .save(basePath); -``` - -Please refer to [configurations](configurations.html) section, to view all datasource options. - - #### DeltaStreamer Tool -The `HoodieDeltaStreamer` utility provides the way to achieve all of these, by using the capabilities of `HoodieWriteClient`. +The `HoodieDeltaStreamer` utility provides the way to achieve all of these, by using the capabilities of `HoodieWriteClient`, and support simply row-row ingestion (no transformations) +from different sources such as DFS or Kafka. The tool is a spark job (part of hoodie-utilities), that provides the following functionality @@ -50,88 +27,112 @@ The tool is a spark job (part of hoodie-utilities), that provides the following - Support json, avro or a custom payload types for the incoming data - New data is written to a Hoodie dataset, with support for checkpointing & schemas and registered onto Hive - To understand more +Command line options describe capabilities in more detail (first build hoodie-utilities using `mvn clean package`). ``` - -[hoodie]$ spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer hoodie-utilities/target/hoodie-utilities-0.3.6-SNAPSHOT-bin.jar --help +[hoodie]$ spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer `ls hoodie-utilities/target/hoodie-utilities-*-SNAPSHOT.jar` --help Usage:
    [options] Options: --help, -h - Default: false - --hoodie-client-config - path to properties file on localfs or dfs, with hoodie client config. - Sane defaultsare used, but recommend use to provide basic things like metrics - endpoints, hive configs etc + --key-generator-class - Subclass of com.uber.hoodie.utilities.common.KeyExtractor to generatea - HoodieKey from the given avro record. Built in: SimpleKeyGenerator (Uses provided - field names as recordkey & partitionpath. Nested fields specified via dot - notation, e.g: a.b.c) - Default: com.uber.hoodie.utilities.keygen.SimpleKeyGenerator - --key-generator-config - Path to properties file on localfs or dfs, with KeyGenerator configs. For - list of acceptable properites, refer the KeyGenerator class - --max-input-bytes - Maximum number of bytes to read from source. Default: 1TB - Default: 1099511627776 + Subclass of com.uber.hoodie.KeyGenerator to generate a HoodieKey from + the given avro record. Built in: SimpleKeyGenerator (uses provided field + names as recordkey & partitionpath. Nested fields specified via dot + notation, e.g: a.b.c) + Default: com.uber.hoodie.SimpleKeyGenerator --op - Takes one of these values : UPSERT (default), INSERT (use when input is - purely new data/inserts to gain speed) - Default: UPSERT - Possible Values: [UPSERT, INSERT] + Takes one of these values : UPSERT (default), INSERT (use when input is + purely new data/inserts to gain speed) + Default: UPSERT + Possible Values: [UPSERT, INSERT, BULK_INSERT] --payload-class - subclass of HoodieRecordPayload, that works off a GenericRecord. Default: - SourceWrapperPayload. Implement your own, if you want to do something other than overwriting - existing value - Default: com.uber.hoodie.utilities.deltastreamer.DeltaStreamerAvroPayload + subclass of HoodieRecordPayload, that works off a GenericRecord. + Implement your own, if you want to do something other than overwriting + existing value + Default: com.uber.hoodie.OverwriteWithLatestAvroPayload + --props + path to properties file on localfs or dfs, with configurations for + hoodie client, schema provider, key generator and data source. For + hoodie client props, sane defaults are used, but recommend use to + provide basic things like metrics endpoints, hive configs etc. For + sources, referto individual classes, for supported properties. + Default: file:///Users/vinoth/bin/hoodie/src/test/resources/delta-streamer-config/dfs-source.properties --schemaprovider-class - subclass of com.uber.hoodie.utilities.schema.SchemaProvider to attach - schemas to input & target table data, built in options: FilebasedSchemaProvider - Default: com.uber.hoodie.utilities.schema.FilebasedSchemaProvider - --schemaprovider-config - path to properties file on localfs or dfs, with schema configs. For list - of acceptable properties, refer the schema provider class + subclass of com.uber.hoodie.utilities.schema.SchemaProvider to attach + schemas to input & target table data, built in options: + FilebasedSchemaProvider + Default: com.uber.hoodie.utilities.schema.FilebasedSchemaProvider --source-class - subclass of com.uber.hoodie.utilities.sources.Source to use to read data. - built-in options: com.uber.hoodie.utilities.common.{DFSSource (default), - KafkaSource, HiveIncrPullSource} - Default: com.uber.hoodie.utilities.sources.DFSSource - --source-config - path to properties file on localfs or dfs, with source configs. For list - of acceptable properties, refer the source class - --source-format - Format of data in source, JSON (default), Avro. All source data is - converted to Avro using the provided schema in any case - Default: JSON - Possible Values: [AVRO, JSON, ROW, CUSTOM] + Subclass of com.uber.hoodie.utilities.sources to read data. Built-in + options: com.uber.hoodie.utilities.sources.{JsonDFSSource (default), + AvroDFSSource, JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource} + Default: com.uber.hoodie.utilities.sources.JsonDFSSource + --source-limit + 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: 9223372036854775807 --source-ordering-field - Field within source record to decide how to break ties between records - with same key in input data. Default: 'ts' holding unix timestamp of record - Default: ts - --target-base-path - base path for the target hoodie dataset - --target-table - name of the target table in Hive + Field within source record to decide how to break ties between records + with same key in input data. Default: 'ts' holding unix timestamp of + record + Default: ts + --spark-master + spark master to use. + Default: local[2] + * --target-base-path + base path for the target hoodie dataset. (Will be created if did not + exist first time around. If exists, expected to be a hoodie dataset) + * --target-table + name of the target table in Hive ``` -For e.g, followings ingests data from Kafka (avro records as the client example) +The tool takes a hierarchically composed property file and has pluggable interfaces for extracting data, key generation and providing schema. Sample configs for ingesting from kafka and dfs are +provided under `hoodie-utilities/src/test/resources/delta-streamer-config`. + +For e.g: once you have Confluent Kafka, Schema registry up & running, produce some test data using ([impressions.avro](https://docs.confluent.io/current/ksql/docs/tutorials/generate-custom-test-data.html) provided by schema-registry repo) ``` -[hoodie]$ spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer hoodie-utilities/target/hoodie-utilities-0.3.6-SNAPSHOT-bin.jar \ - --hoodie-client-config hoodie-utilities/src/main/resources/delta-streamer-config/hoodie-client.properties \ - --key-generator-config hoodie-utilities/src/main/resources/delta-streamer-config/key-generator.properties \ - --schemaprovider-config hoodie-utilities/src/main/resources/delta-streamer-config/schema-provider.properties \ - --source-class com.uber.hoodie.utilities.sources.KafkaSource \ - --source-config hoodie-utilities/src/main/resources/delta-streamer-config/source.properties \ - --source-ordering-field rider \ - --target-base-path file:///tmp/hoodie-deltastreamer-op \ - --target-table uber.trips +[confluent-5.0.0]$ bin/ksql-datagen schema=../impressions.avro format=avro topic=impressions key=impressionid ``` +and then ingest it as follows. + +``` +[hoodie]$ spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer `ls hoodie-utilities/target/hoodie-utilities-*-SNAPSHOT.jar` \ + --props file://${PWD}/hoodie-utilities/src/test/resources/delta-streamer-config/kafka-source.properties \ + --schemaprovider-class com.uber.hoodie.utilities.schema.SchemaRegistryProvider \ + --source-class com.uber.hoodie.utilities.sources.AvroKafkaSource \ + --source-ordering-field impresssiontime \ + --target-base-path file:///tmp/hoodie-deltastreamer-op --target-table uber.impressions \ + --op BULK_INSERT +``` + +In some cases, you may want to convert your existing dataset into Hoodie, before you can begin ingesting new data. This can be accomplished using the `hdfsparquetimport` command on the `hoodie-cli`. +Currently, there is support for converting parquet datasets. + +#### Via Custom Spark Job + +The `hoodie-spark` module offers the DataSource API to write any data frame into a Hoodie dataset. Following is how we can upsert a dataframe, while specifying the field names that need to be used +for `recordKey => _row_key`, `partitionPath => partition` and `precombineKey => timestamp` + + +``` +inputDF.write() + .format("com.uber.hoodie") + .options(clientOpts) // any of the hoodie client opts can be passed in as well + .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") + .option(HoodieWriteConfig.TABLE_NAME, tableName) + .mode(SaveMode.Append) + .save(basePath); +``` + +Please refer to [configurations](configurations.html) section, to view all datasource options. #### Syncing to Hive @@ -164,7 +165,7 @@ Usage:
    [options] {% include callout.html content="Note that for now, due to jar mismatches between Spark & Hive, its recommended to run this as a separate Java task in your workflow manager/cron. This is getting fix [here](https://github.com/uber/hoodie/issues/123)" type="info" %} -## Incremental Pull +## Incrementally Pulling Hoodie datasets can be pulled incrementally, which means you can get ALL and ONLY the updated & new rows since a specified commit timestamp. This, together with upserts, are particularly useful for building data pipelines where 1 or more source hoodie tables are incrementally pulled (streams/facts), diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java index 6750bafe915da..76a19fa70a2b8 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java @@ -41,7 +41,6 @@ public String convert( @CliOption(key = "upsert", mandatory = false, unspecifiedDefaultValue = "false", help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert, @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath, - @CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") final String srcType, @CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String targetPath, @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, @@ -57,7 +56,7 @@ public String convert( @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception { - validate(format, srcType); + (new HDFSParquetImporter.FormatValidator()).validate("format", format); boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); @@ -81,9 +80,4 @@ public String convert( } return "Dataset imported to hoodie format"; } - - private void validate(String format, String srcType) { - (new HDFSParquetImporter.FormatValidator()).validate("format", format); - (new HDFSParquetImporter.SourceTypeValidator()).validate("srcType", srcType); - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index a36faef1cadef..b056bd805923d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -65,7 +65,6 @@ public class HoodieTestDataGenerator { DEFAULT_THIRD_PARTITION_PATH }; public static final int DEFAULT_PARTITION_DEPTH = 3; - public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"double\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"}," @@ -78,6 +77,7 @@ public class HoodieTestDataGenerator { + "{\"name\":\"fare\",\"type\": \"double\"}]}"; public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); private static Random rand = new Random(46474747); + private List existingKeysList = new ArrayList<>(); private String[] partitionPaths; @@ -162,10 +162,6 @@ public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInst } } - public static void createSavepointFile(String basePath, String commitTime) throws IOException { - createSavepointFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); - } - public static void createSavepointFile(String basePath, String commitTime, Configuration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME @@ -289,8 +285,11 @@ public String[] getPartitionPaths() { return partitionPaths; } - static class KeyPartition { + public List getExistingKeysList() { + return existingKeysList; + } + public static class KeyPartition { HoodieKey key; String partitionPath; } diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 1e2f4fec4ee75..071ae7a636b6d 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -88,6 +88,7 @@ com.fasterxml.jackson.core jackson-annotations + ${fasterxml.version} org.apache.parquet diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java new file mode 100644 index 0000000000000..758ba0fcaabd8 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java @@ -0,0 +1,102 @@ +/* + * 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.common.util; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.HashSet; +import java.util.Set; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * A simplified versions of Apache commons - PropertiesConfiguration, that supports limited field types and hierarchical + * configurations within the same folder as the root file. + * + * Includes denoted by the same include=filename.properties syntax, with relative path from root file's folder. Lines + * beginning with '#' are ignored as comments. Final values for properties are resolved by the order in which they are + * specified in the files, with included files treated as if they are inline. + * + * Note: Not reusing commons-configuration since it has too many conflicting runtime deps. + */ +public class DFSPropertiesConfiguration { + + private static volatile Logger log = LogManager.getLogger(DFSPropertiesConfiguration.class); + + private final FileSystem fs; + + private final Path rootFile; + + private final TypedProperties props; + + // Keep track of files visited, to detect loops + private final Set visitedFiles; + + public DFSPropertiesConfiguration(FileSystem fs, Path rootFile, TypedProperties defaults) { + this.fs = fs; + this.rootFile = rootFile; + this.props = defaults; + this.visitedFiles = new HashSet<>(); + visitFile(rootFile); + } + + private String[] splitProperty(String line) { + int ind = line.indexOf('='); + String k = line.substring(0, ind).trim(); + String v = line.substring(ind + 1).trim(); + return new String[]{k, v}; + } + + private void visitFile(Path file) { + try { + if (visitedFiles.contains(file.getName())) { + throw new IllegalStateException("Loop detected; file " + file + " already referenced"); + } + visitedFiles.add(file.getName()); + BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file))); + String line; + while ((line = reader.readLine()) != null) { + if (line.startsWith("#") || line.equals("") || !line.contains("=")) { + continue; + } + String[] split = splitProperty(line); + if (line.startsWith("include=") || line.startsWith("include =")) { + visitFile(new Path(rootFile.getParent(), split[1])); + } else { + props.setProperty(split[0], split[1]); + } + } + reader.close(); + } catch (IOException ioe) { + log.error("Error reading in properies from dfs", ioe); + throw new IllegalArgumentException("Cannot read properties from dfs", ioe); + } + } + + public DFSPropertiesConfiguration(FileSystem fs, Path rootFile) { + this(fs, rootFile, new TypedProperties()); + } + + public TypedProperties getConfig() { + return props; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java index 1d5d3b58f8a98..7416067e0e600 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java @@ -19,6 +19,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.exception.HoodieException; import java.lang.reflect.InvocationTargetException; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -26,18 +27,22 @@ public class ReflectionUtils { private static Map> clazzCache = new HashMap<>(); + private static Class getClass(String clazzName) { + if (!clazzCache.containsKey(clazzName)) { + try { + Class clazz = Class.forName(clazzName); + clazzCache.put(clazzName, clazz); + } catch (ClassNotFoundException e) { + throw new HoodieException("Unable to load class", e); + } + } + return clazzCache.get(clazzName); + } + public static T loadClass(String fqcn) { try { - if (clazzCache.get(fqcn) == null) { - Class clazz = Class.forName(fqcn); - clazzCache.put(fqcn, clazz); - } - return (T) clazzCache.get(fqcn).newInstance(); - } catch (ClassNotFoundException e) { - throw new HoodieException("Could not load class " + fqcn, e); - } catch (InstantiationException e) { - throw new HoodieException("Could not load class " + fqcn, e); - } catch (IllegalAccessException e) { + return (T) getClass(fqcn).newInstance(); + } catch (InstantiationException | IllegalAccessException e) { throw new HoodieException("Could not load class " + fqcn, e); } } @@ -49,18 +54,32 @@ public static T loadPayload(String recordPayload Object[] payloadArgs, Class... constructorArgTypes) { try { - if (clazzCache.get(recordPayloadClass) == null) { - Class clazz = Class.forName(recordPayloadClass); - clazzCache.put(recordPayloadClass, clazz); - } - return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes) + return (T) getClass(recordPayloadClass).getConstructor(constructorArgTypes) .newInstance(payloadArgs); } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { throw new HoodieException("Unable to instantiate payload class ", e); - } catch (ClassNotFoundException e) { - throw new HoodieException("Unable to instantiate payload class ", e); } } + /** + * Creates an instnace of the given class. Use this version when dealing with interface types as constructor args. + */ + public static Object loadClass(String clazz, Class[] constructorArgTypes, Object... constructorArgs) { + try { + return getClass(clazz).getConstructor(constructorArgTypes).newInstance(constructorArgs); + } catch (InstantiationException | IllegalAccessException + | InvocationTargetException | NoSuchMethodException e) { + throw new HoodieException("Unable to instantiate class ", e); + } + } + + /** + * Creates an instance of the given class. Constructor arg types are inferred. + */ + public static Object loadClass(String clazz, Object... constructorArgs) { + Class[] constructorArgTypes = Arrays.stream(constructorArgs) + .map(arg -> arg.getClass()).toArray(Class[]::new); + return loadClass(clazz, constructorArgTypes, constructorArgs); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java new file mode 100644 index 0000000000000..5674d2382484c --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java @@ -0,0 +1,87 @@ +/* + * 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.common.util; + +import java.io.Serializable; +import java.util.Properties; + +/** + * Type-aware extension of {@link java.util.Properties} + */ +public class TypedProperties extends Properties implements Serializable { + + public TypedProperties() { + super(null); + } + + public TypedProperties(Properties defaults) { + super(defaults); + } + + private void checkKey(String property) { + if (!containsKey(property)) { + throw new IllegalArgumentException("Property " + property + " not found"); + } + } + + public String getString(String property) { + checkKey(property); + return getProperty(property); + } + + public String getString(String property, String defaultValue) { + return containsKey(property) ? getProperty(property) : defaultValue; + } + + public int getInteger(String property) { + checkKey(property); + return Integer.valueOf(getProperty(property)); + } + + public int getInteger(String property, int defaultValue) { + return containsKey(property) ? Integer.valueOf(getProperty(property)) : defaultValue; + } + + public long getLong(String property) { + checkKey(property); + return Long.valueOf(getProperty(property)); + } + + public long getLong(String property, long defaultValue) { + return containsKey(property) ? Long.valueOf(getProperty(property)) : defaultValue; + } + + public boolean getBoolean(String property) { + checkKey(property); + return Boolean.valueOf(getProperty(property)); + } + + public boolean getBoolean(String property, boolean defaultValue) { + return containsKey(property) ? Boolean.valueOf(getProperty(property)) : defaultValue; + } + + public double getDouble(String property) { + checkKey(property); + return Double.valueOf(getProperty(property)); + } + + public double getDouble(String property, double defaultValue) { + return containsKey(property) ? Double.valueOf(getProperty(property)) : defaultValue; + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestDFSPropertiesConfiguration.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestDFSPropertiesConfiguration.java new file mode 100644 index 0000000000000..1146abfe18c82 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestDFSPropertiesConfiguration.java @@ -0,0 +1,138 @@ +/* + * 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.common.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import com.uber.hoodie.common.minicluster.HdfsTestService; +import java.io.IOException; +import java.io.PrintStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests basic functionality of {@link DFSPropertiesConfiguration} + */ +public class TestDFSPropertiesConfiguration { + + private static String dfsBasePath; + private static HdfsTestService hdfsTestService; + private static MiniDFSCluster dfsCluster; + private static DistributedFileSystem dfs; + + + @BeforeClass + public static void initClass() throws Exception { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + // Create a temp folder as the base path + dfs = dfsCluster.getFileSystem(); + dfsBasePath = dfs.getWorkingDirectory().toString(); + dfs.mkdirs(new Path(dfsBasePath)); + + // create some files. + Path filePath = new Path(dfsBasePath + "/t1.props"); + writePropertiesFile(filePath, new String[]{ + "", "#comment", "abc",// to be ignored + "int.prop=123", "double.prop=113.4", "string.prop=str", "boolean.prop=true", "long.prop=1354354354" + }); + + filePath = new Path(dfsBasePath + "/t2.props"); + writePropertiesFile(filePath, new String[]{ + "string.prop=ignored", "include=t1.props" + }); + + filePath = new Path(dfsBasePath + "/t3.props"); + writePropertiesFile(filePath, new String[]{ + "double.prop=838.3", "include = t2.props", "double.prop=243.4", "string.prop=t3.value" + }); + + filePath = new Path(dfsBasePath + "/t4.props"); + writePropertiesFile(filePath, new String[]{ + "double.prop=838.3", "include = t4.props" + }); + } + + @AfterClass + public static void cleanupClass() throws Exception { + if (hdfsTestService != null) { + hdfsTestService.stop(); + } + } + + private static void writePropertiesFile(Path path, String[] lines) throws IOException { + PrintStream out = new PrintStream(dfs.create(path, true)); + for (String line : lines) { + out.println(line); + } + out.flush(); + out.close(); + } + + @Test + public void testParsing() throws IOException { + DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t1.props")); + TypedProperties props = cfg.getConfig(); + assertEquals(5, props.size()); + try { + props.getString("invalid.key"); + fail("Should error out here."); + } catch (IllegalArgumentException iae) { /* ignore */ } + + assertEquals(123, props.getInteger("int.prop")); + assertEquals(113.4, props.getDouble("double.prop"), 0.001); + assertEquals(true, props.getBoolean("boolean.prop")); + assertEquals("str", props.getString("string.prop")); + assertEquals(1354354354, props.getLong("long.prop")); + + assertEquals(123, props.getInteger("int.prop", 456)); + assertEquals(113.4, props.getDouble("double.prop", 223.4), 0.001); + assertEquals(true, props.getBoolean("boolean.prop", false)); + assertEquals("str", props.getString("string.prop", "default")); + assertEquals(1354354354, props.getLong("long.prop", 8578494434L)); + + assertEquals(456, props.getInteger("bad.int.prop", 456)); + assertEquals(223.4, props.getDouble("bad.double.prop", 223.4), 0.001); + assertEquals(false, props.getBoolean("bad.boolean.prop", false)); + assertEquals("default", props.getString("bad.string.prop", "default")); + assertEquals(8578494434L, props.getLong("bad.long.prop", 8578494434L)); + } + + @Test + public void testIncludes() { + DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t3.props")); + TypedProperties props = cfg.getConfig(); + + assertEquals(123, props.getInteger("int.prop")); + assertEquals(243.4, props.getDouble("double.prop"), 0.001); + assertEquals(true, props.getBoolean("boolean.prop")); + assertEquals("t3.value", props.getString("string.prop")); + assertEquals(1354354354, props.getLong("long.prop")); + + try { + new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t4.props")); + fail("Should error out on a self-included file."); + } catch (IllegalStateException ise) { /* ignore */ } + } +} diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 3942dac3ea73f..5ed93b3b05189 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -148,7 +148,10 @@ com.fasterxml.jackson.core jackson-annotations - + + com.fasterxml.jackson.module + jackson-module-scala_2.11 + org.apache.hadoop hadoop-client diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java index 8e452c8dc11d1..191b2d236e22c 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java @@ -18,6 +18,7 @@ package com.uber.hoodie; +import com.uber.hoodie.exception.HoodieException; import java.io.Serializable; import org.apache.avro.generic.GenericRecord; @@ -44,5 +45,8 @@ public abstract class BaseAvroPayload implements Serializable { public BaseAvroPayload(GenericRecord record, Comparable orderingVal) { this.record = record; this.orderingVal = orderingVal; + if (orderingVal == null) { + throw new HoodieException("Ordering value is null for record: " + record); + } } } diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index 19226af91b168..551a1865d101e 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -21,6 +21,8 @@ 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.util.ReflectionUtils; +import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -31,8 +33,6 @@ import java.util.List; import java.util.Map; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.reflect.ConstructorUtils; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -72,10 +72,9 @@ public static String getNestedFieldValAsString(GenericRecord record, String fiel * Create a key generator class via reflection, passing in any configs needed */ public static KeyGenerator createKeyGenerator(String keyGeneratorClass, - PropertiesConfiguration cfg) throws IOException { + TypedProperties props) throws IOException { try { - return (KeyGenerator) ConstructorUtils - .invokeConstructor(Class.forName(keyGeneratorClass), (Object) cfg); + return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props); } catch (Throwable e) { throw new IOException("Could not load key generator class " + keyGeneratorClass, e); } @@ -87,17 +86,17 @@ public static KeyGenerator createKeyGenerator(String keyGeneratorClass, public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) throws IOException { try { - return (HoodieRecordPayload) ConstructorUtils.invokeConstructor(Class.forName(payloadClass), - (Object) record, (Object) orderingVal); + return (HoodieRecordPayload) ReflectionUtils + .loadClass(payloadClass, new Class[]{GenericRecord.class, Comparable.class}, record, orderingVal); } catch (Throwable e) { throw new IOException("Could not create payload for class: " + payloadClass, e); } } - public static void checkRequiredProperties(PropertiesConfiguration configuration, + public static void checkRequiredProperties(TypedProperties props, List checkPropNames) { checkPropNames.stream().forEach(prop -> { - if (!configuration.containsKey(prop)) { + if (!props.containsKey(prop)) { throw new HoodieNotSupportedException("Required property " + prop + " is missing"); } }); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java b/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java index 74aa74560c6bb..3c6e2d65d03ba 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java @@ -19,9 +19,9 @@ package com.uber.hoodie; import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.util.TypedProperties; import java.io.Serializable; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; /** * Abstract class to extend for plugging in extraction of @@ -30,9 +30,9 @@ */ public abstract class KeyGenerator implements Serializable { - protected transient PropertiesConfiguration config; + protected transient TypedProperties config; - protected KeyGenerator(PropertiesConfiguration config) { + protected KeyGenerator(TypedProperties config) { this.config = config; } diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/SimpleKeyGenerator.java b/hoodie-spark/src/main/java/com/uber/hoodie/SimpleKeyGenerator.java index 6c15ce434272a..0017f20b33721 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/SimpleKeyGenerator.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/SimpleKeyGenerator.java @@ -19,9 +19,9 @@ package com.uber.hoodie; import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieException; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; /** * Simple key generator, which takes names of fields to be used for recordKey and partitionPath as @@ -29,14 +29,16 @@ */ public class SimpleKeyGenerator extends KeyGenerator { + private static final String DEFAULT_PARTITION_PATH = "default"; + protected final String recordKeyField; protected final String partitionPathField; - public SimpleKeyGenerator(PropertiesConfiguration config) { - super(config); - this.recordKeyField = config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); - this.partitionPathField = config + public SimpleKeyGenerator(TypedProperties props) { + super(props); + this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); + this.partitionPathField = props .getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()); } @@ -46,7 +48,16 @@ public HoodieKey getKey(GenericRecord record) { throw new HoodieException( "Unable to find field names for record key or partition path in cfg"); } - return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField), - DataSourceUtils.getNestedFieldValAsString(record, partitionPathField)); + + String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField); + String partitionPath; + try { + partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField); + } catch (HoodieException e) { + // if field is not found, lump it into default partition + partitionPath = DEFAULT_PARTITION_PATH; + } + + return new HoodieKey(recordKey, partitionPath); } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index dd744da6d5079..5a94f9e60e9b8 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -25,10 +25,10 @@ import java.util.{Optional, Properties} import com.uber.hoodie.DataSourceReadOptions._ import com.uber.hoodie.DataSourceWriteOptions._ import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import com.uber.hoodie.common.util.TypedProperties import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException import org.apache.avro.generic.GenericRecord -import org.apache.commons.configuration.PropertiesConfiguration import org.apache.hadoop.fs.Path import org.apache.log4j.LogManager import org.apache.spark.api.java.JavaSparkContext @@ -121,10 +121,10 @@ class DefaultSource extends RelationProvider mapAsScalaMap(defaultsMap) } - def toPropertiesConfiguration(params: Map[String, String]): PropertiesConfiguration = { - val propCfg = new PropertiesConfiguration() - params.foreach(kv => propCfg.addProperty(kv._1, kv._2)) - propCfg + def toProperties(params: Map[String, String]): TypedProperties = { + val props = new TypedProperties() + params.foreach(kv => props.setProperty(kv._1, kv._2)) + props } @@ -161,7 +161,7 @@ class DefaultSource extends RelationProvider // Convert to RDD[HoodieRecord] val keyGenerator = DataSourceUtils.createKeyGenerator( parameters(KEYGENERATOR_CLASS_OPT_KEY), - toPropertiesConfiguration(parameters) + toProperties(parameters) ) val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) val hoodieRecords = genericRecords.map(gr => { diff --git a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala index a2f82af4165b6..734cc7f5366cf 100644 --- a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala @@ -16,11 +16,10 @@ * */ -import com.uber.hoodie.common.util.SchemaTestUtil +import com.uber.hoodie.common.util.{SchemaTestUtil, TypedProperties} import com.uber.hoodie.exception.HoodieException import com.uber.hoodie.{DataSourceWriteOptions, OverwriteWithLatestAvroPayload, SimpleKeyGenerator} import org.apache.avro.generic.GenericRecord -import org.apache.commons.configuration.PropertiesConfiguration import org.junit.Assert._ import org.junit.{Before, Test} import org.scalatest.junit.AssertionsForJUnit @@ -39,10 +38,10 @@ class DataSourceDefaultsTest extends AssertionsForJUnit { } - private def getKeyConfig(recordKeyFieldName: String, paritionPathField: String): PropertiesConfiguration = { - val props = new PropertiesConfiguration() - props.addProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName) - props.addProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, paritionPathField) + private def getKeyConfig(recordKeyFieldName: String, paritionPathField: String): TypedProperties = { + val props = new TypedProperties() + props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName) + props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, paritionPathField) props } @@ -52,24 +51,26 @@ class DataSourceDefaultsTest extends AssertionsForJUnit { assertEquals("field1", hk1.getRecordKey) assertEquals("name1", hk1.getPartitionPath) - // recordKey field not specified + // partition path field not specified try { - val props = new PropertiesConfiguration() - props.addProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") + val props = new TypedProperties() + props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") new SimpleKeyGenerator(props).getKey(baseRecord) fail("Should have errored out") } catch { - case e: HoodieException => { + case e: IllegalArgumentException => { // do nothing } }; - // partitionPath field is null + // recordkey field not specified try { - new SimpleKeyGenerator(getKeyConfig("field1", null)).getKey(baseRecord) + val props = new TypedProperties() + props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField") + new SimpleKeyGenerator(props).getKey(baseRecord) fail("Should have errored out") } catch { - case e: HoodieException => { + case e: IllegalArgumentException => { // do nothing } }; @@ -90,6 +91,11 @@ class DataSourceDefaultsTest extends AssertionsForJUnit { // do nothing } }; + + // if partition path can't be found, return default partition path + val hk3 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere")) + .getKey(baseRecord); + assertEquals("default", hk3.getPartitionPath) } @Test def testOverwriteWithLatestAvroPayload() = { diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 0f28561d7cfef..e8159d54080f1 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -42,27 +42,43 @@ org.apache.maven.plugins - maven-assembly-plugin - 2.4.1 - - - src/assembly/src.xml - - - - com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer - - - - + maven-shade-plugin + 3.1.1 - make-assembly - package - single + shade + + ${project.build.directory}/dependency-reduced-pom.xml + + true + + + com.uber.hoodie:hoodie-common + com.uber.hoodie:hoodie-client + com.uber.hoodie:hoodie-spark + com.uber.hoodie:hoodie-hive + com.uber.hoodie:hoodie-hadoop-mr + com.beust:jcommander + com.twitter:bijection-avro_2.11 + com.twitter:bijection-core_2.11 + org.apache.parquet:parquet-avro + com.twitter:parquet-avro + com.twitter.common:objectsize + io.confluent:kafka-avro-serializer + io.confluent:common-config + io.confluent:common-utils + io.confluent:kafka-schema-registry-client + org.apache.spark:spark-streaming-kafka-0-8_2.11 + org.apache.kafka:kafka_2.11 + com.yammer.metrics:metrics-core + com.101tec:zkclient + org.apache.kafka:kafka-clients + + + @@ -78,10 +94,18 @@ + + + confluent + http://packages.confluent.io/maven/ + + + + - org.apache.spark - spark-sql_2.11 + com.fasterxml.jackson.module + jackson-module-scala_2.11 @@ -108,21 +132,6 @@ org.apache.hadoop hadoop-hdfs tests - - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - org.apache.hadoop @@ -184,6 +193,10 @@ javax.servlet servlet-api + + com.fasterxml.jackson.* + * + @@ -219,23 +232,16 @@ org.apache.hadoop hadoop-client - - - javax.servlet - servlet-api - - - + org.apache.spark spark-core_2.11 - - - javax.servlet - servlet-api - - + + + + org.apache.spark + spark-sql_2.11 @@ -291,6 +297,30 @@ 0.9.2 + + io.confluent + kafka-avro-serializer + 3.0.0 + + + + io.confluent + common-config + 3.0.0 + + + + io.confluent + common-utils + 3.0.0 + + + + io.confluent + kafka-schema-registry-client + 3.0.0 + + diff --git a/hoodie-utilities/src/assembly/src.xml b/hoodie-utilities/src/assembly/src.xml deleted file mode 100644 index aa2fbcd21c830..0000000000000 --- a/hoodie-utilities/src/assembly/src.xml +++ /dev/null @@ -1,50 +0,0 @@ - - - - bin - - jar - - - false - - - / - true - runtime - - junit:junit - com.google.code.findbugs:* - org.apache.hadoop:* - org.apache.hbase:* - - - - - - - - - - - - - - - diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 7ec989c95a5e4..9e6f455129b9d 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -75,7 +75,9 @@ public static void main(String[] args) throws Exception { System.exit(1); } HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg); - dataImporter.dataImport(UtilHelpers.buildSparkContext(cfg.tableName, cfg.sparkMaster, cfg.sparkMemory), cfg.retry); + dataImporter + .dataImport(UtilHelpers.buildSparkContext("data-importer-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory), + cfg.retry); } public int dataImport(JavaSparkContext jsc, int retry) throws Exception { @@ -206,19 +208,6 @@ public void validate(String name, String value) throws ParameterException { } } - public static class SourceTypeValidator implements IValueValidator { - - List validSourceTypes = Arrays.asList("hdfs"); - - @Override - public void validate(String name, String value) throws ParameterException { - if (value == null || !validSourceTypes.contains(value)) { - throw new ParameterException(String.format( - "Invalid source type: value:%s: supported source types:%s", value, validSourceTypes)); - } - } - } - public static class Config implements Serializable { @Parameter(names = {"--command", "-c"}, @@ -228,10 +217,6 @@ public static class Config implements Serializable { @Parameter(names = {"--src-path", "-sp"}, description = "Base path for the input dataset", required = true) public String srcPath = null; - @Parameter(names = {"--src-type", - "-st"}, description = "Source type for the input dataset", required = true, - validateValueWith = SourceTypeValidator.class) - public String srcType = null; @Parameter(names = {"--target-path", "-tp"}, description = "Base path for the target hoodie dataset", required = true) public String targetPath = null; diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index 07d3ecbe9d3e8..72c50f77ab71b 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -93,8 +93,6 @@ public static class Config implements Serializable { public int maxCommits = 3; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; - @Parameter(names = {"--storageFormat"}) - public String tempTableStorageFormat = "AVRO"; } static { @@ -207,12 +205,7 @@ private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, S } private String getStoredAsClause() { - if (config.tempTableStorageFormat.equalsIgnoreCase("json")) { - // Special case for json - // default json serde does not support having same key even if its under multiple depths - return "ROW FORMAT SERDE 'org.openx.data.jsonserde.JsonSerDe' STORED AS TEXTFILE"; - } - return "STORED AS " + config.tempTableStorageFormat; + return "STORED AS AVRO"; } private void initHiveBeelineProperties(Statement stmt) throws SQLException { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java index d327e6d0d0b3a..8aa596793faf6 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java @@ -80,7 +80,8 @@ public static void main(String[] args) throws Exception { System.exit(1); } HoodieCompactor compactor = new HoodieCompactor(cfg); - compactor.compact(UtilHelpers.buildSparkContext(cfg.tableName, cfg.sparkMaster, cfg.sparkMemory), cfg.retry); + compactor.compact(UtilHelpers.buildSparkContext("compactor-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory), + cfg.retry); } public int compact(JavaSparkContext jsc, int retry) { @@ -119,4 +120,4 @@ private int doSchedule(JavaSparkContext jsc) throws Exception { client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Optional.empty()); return 0; } -} \ No newline at end of file +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index 77952bcf0d436..7cd040dfed98d 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -20,22 +20,20 @@ import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.util.DFSPropertiesConfiguration; import com.uber.hoodie.common.util.ReflectionUtils; +import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.index.HoodieIndex; -import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.Source; -import com.uber.hoodie.utilities.sources.SourceDataFormat; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.util.Optional; -import org.apache.commons.configuration.ConfigurationException; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.reflect.ConstructorUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,42 +50,34 @@ public class UtilHelpers { private static Logger logger = LogManager.getLogger(UtilHelpers.class); - public static Source createSource(String sourceClass, PropertiesConfiguration cfg, - JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider) + public static Source createSource(String sourceClass, TypedProperties cfg, + JavaSparkContext jssc, SchemaProvider schemaProvider) throws IOException { try { - return (Source) ConstructorUtils.invokeConstructor(Class.forName(sourceClass), (Object) cfg, - (Object) jssc, (Object) dataFormat, (Object) schemaProvider); + return (Source) ReflectionUtils.loadClass(sourceClass, + new Class[]{TypedProperties.class, JavaSparkContext.class, SchemaProvider.class}, + cfg, jssc, schemaProvider); } catch (Throwable e) { throw new IOException("Could not load source class " + sourceClass, e); } } public static SchemaProvider createSchemaProvider(String schemaProviderClass, - PropertiesConfiguration cfg) throws IOException { + TypedProperties cfg, JavaSparkContext jssc) throws IOException { try { - return (SchemaProvider) ConstructorUtils.invokeConstructor(Class.forName(schemaProviderClass), - (Object) cfg); + return (SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc); } catch (Throwable e) { throw new IOException("Could not load schema provider class " + schemaProviderClass, e); } } /** - * TODO: Support hierarchical config files (see CONFIGURATION-609 for sample) */ - public static PropertiesConfiguration readConfig(FileSystem fs, Path cfgPath) { + public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath) { try { - FSDataInputStream in = fs.open(cfgPath); - PropertiesConfiguration config = new PropertiesConfiguration(); - config.load(in); - in.close(); - return config; - } catch (IOException e) { - throw new HoodieIOException("Unable to read config file at :" + cfgPath, e); - } catch (ConfigurationException e) { - throw new HoodieDeltaStreamerException("Invalid configs found in config file at :" + cfgPath, - e); + return new DFSPropertiesConfiguration(fs, cfgPath); + } catch (Exception e) { + throw new HoodieException("Unable to read props file at :" + cfgPath, e); } } @@ -117,24 +107,16 @@ public static String parseSchema(FileSystem fs, String schemaFile) throws Except return new String(buf.array()); } - /** - * Build Spark Context for ingestion/compaction - * @return - */ - public static JavaSparkContext buildSparkContext(String tableName, String sparkMaster, String sparkMemory) { - SparkConf sparkConf = new SparkConf().setAppName("hoodie-data-importer-" + tableName); - sparkConf.setMaster(sparkMaster); - - if (sparkMaster.startsWith("yarn")) { + private static SparkConf buildSparkConf(String appName, String defaultMaster) { + SparkConf sparkConf = new SparkConf().setAppName(appName); + String master = sparkConf.get("spark.master", defaultMaster); + sparkConf.setMaster(master); + if (master.startsWith("yarn")) { sparkConf.set("spark.eventLog.overwrite", "true"); sparkConf.set("spark.eventLog.enabled", "true"); } - sparkConf.set("spark.driver.maxResultSize", "2g"); sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.set("spark.executor.memory", sparkMemory); - - // Configure hadoop conf 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", @@ -142,6 +124,20 @@ public static JavaSparkContext buildSparkContext(String tableName, String sparkM sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); sparkConf = HoodieWriteClient.registerClasses(sparkConf); + return sparkConf; + } + + public static JavaSparkContext buildSparkContext(String appName, String defaultMaster) { + return new JavaSparkContext(buildSparkConf(appName, defaultMaster)); + } + + /** + * Build Spark Context for ingestion/compaction + * @return + */ + public static JavaSparkContext buildSparkContext(String appName, String sparkMaster, String sparkMemory) { + SparkConf sparkConf = buildSparkConf(appName, sparkMaster); + sparkConf.set("spark.executor.memory", sparkMemory); return new JavaSparkContext(sparkConf); } @@ -185,4 +181,10 @@ public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD logger.error(String.format("Import failed with %d errors.", errors.value())); return -1; } + + public static TypedProperties readConfig(InputStream in) throws IOException { + TypedProperties defaults = new TypedProperties(); + defaults.load(in); + return defaults; + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index 7f58a721eeeae..211a171e0e148 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -35,6 +35,8 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; +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.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -44,9 +46,8 @@ import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider; -import com.uber.hoodie.utilities.sources.DFSSource; +import com.uber.hoodie.utilities.sources.JsonDFSSource; import com.uber.hoodie.utilities.sources.Source; -import com.uber.hoodie.utilities.sources.SourceDataFormat; import java.io.IOException; import java.io.Serializable; import java.util.Arrays; @@ -56,13 +57,10 @@ import java.util.Properties; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.tuple.Pair; 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.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import scala.collection.JavaConversions; @@ -77,7 +75,7 @@ public class HoodieDeltaStreamer implements Serializable { private static volatile Logger log = LogManager.getLogger(HoodieDeltaStreamer.class); - private static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; + public static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; private final Config cfg; @@ -113,9 +111,15 @@ public class HoodieDeltaStreamer implements Serializable { private transient JavaSparkContext jssc; - public HoodieDeltaStreamer(Config cfg) throws IOException { + /** + * Bag of properties with source, hoodie client, key generator etc. + */ + TypedProperties props; + + + public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException { this.cfg = cfg; - this.jssc = getSparkContext(); + this.jssc = jssc; this.fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); if (fs.exists(new Path(cfg.targetBasePath))) { @@ -126,61 +130,19 @@ public HoodieDeltaStreamer(Config cfg) throws IOException { this.commitTimelineOpt = Optional.empty(); } - //TODO(vc) Should these be passed from outside? - initSchemaProvider(); - initKeyGenerator(); - initSource(); - } - - private void initSource() throws IOException { - // Create the source & schema providers - PropertiesConfiguration sourceCfg = UtilHelpers.readConfig(fs, new Path(cfg.sourceConfigProps)); - log.info("Creating source " + cfg.sourceClassName + " with configs : " + sourceCfg.toString()); - this.source = UtilHelpers.createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, - schemaProvider); - } - - private void initSchemaProvider() throws IOException { - PropertiesConfiguration schemaCfg = UtilHelpers.readConfig(fs, - new Path(cfg.schemaProviderConfigProps)); - log.info( - "Creating schema provider " + cfg.schemaProviderClassName + " with configs : " + schemaCfg - .toString()); - this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, schemaCfg); - } - - private void initKeyGenerator() throws IOException { - PropertiesConfiguration keygenCfg = UtilHelpers.readConfig(fs, new Path(cfg.keyGeneratorProps)); - log.info("Creating key generator " + cfg.keyGeneratorClass + " with configs : " + keygenCfg - .toString()); - this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, keygenCfg); - } - - - private JavaSparkContext getSparkContext() { - SparkConf sparkConf = new SparkConf() - .setAppName("hoodie-delta-streamer-" + cfg.targetTableName); - //sparkConf.setMaster(cfg.sparkMaster); - sparkConf.setMaster("local[2]"); - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - sparkConf.set("spark.driver.maxResultSize", "2g"); - - // Configure hadoop conf - 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", - "org.apache.hadoop.io.compress.GzipCodec"); - sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); + this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath)).getConfig(); + log.info("Creating delta streamer with configs : " + props.toString()); + this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc); + this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, props); + this.source = UtilHelpers.createSource(cfg.sourceClassName, props, jssc, schemaProvider); - sparkConf = HoodieWriteClient.registerClasses(sparkConf); // register the schemas, so that shuffle does not serialize the full schemas List schemas = Arrays.asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema()); - sparkConf.registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList()); - return new JavaSparkContext(sparkConf); + jssc.sc().getConf().registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList()); } - private void sync() throws Exception { + public void sync() throws Exception { // Retrieve the previous round checkpoints, if any Optional resumeCheckpointStr = Optional.empty(); if (commitTimelineOpt.isPresent()) { @@ -207,7 +169,7 @@ private void sync() throws Exception { // Pull the data from the source & prepare the write Pair>, String> dataAndCheckpoint = source.fetchNewData( - resumeCheckpointStr, cfg.maxInputBytes); + resumeCheckpointStr, cfg.sourceLimit); if (!dataAndCheckpoint.getKey().isPresent()) { log.info("No new data, nothing to commit.. "); @@ -222,7 +184,7 @@ private void sync() throws Exception { }); // Perform the write - HoodieWriteConfig hoodieCfg = getHoodieClientConfig(cfg.hoodieClientProps); + HoodieWriteConfig hoodieCfg = getHoodieClientConfig(); HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg); String commitTime = client.startCommit(); log.info("Starting commit : " + commitTime); @@ -232,6 +194,8 @@ private void sync() throws Exception { writeStatusRDD = client.insert(records, commitTime); } else if (cfg.operation == Operation.UPSERT) { writeStatusRDD = client.upsert(records, commitTime); + } else if (cfg.operation == Operation.BULK_INSERT) { + writeStatusRDD = client.bulkInsert(records, commitTime); } else { throw new HoodieDeltaStreamerException("Unknown operation :" + cfg.operation); } @@ -245,157 +209,84 @@ private void sync() throws Exception { if (success) { log.info("Commit " + commitTime + " successful!"); // TODO(vc): Kick off hive sync from here. - } else { log.info("Commit " + commitTime + " failed!"); } client.close(); } - private HoodieWriteConfig getHoodieClientConfig(String hoodieClientCfgPath) throws Exception { + private HoodieWriteConfig getHoodieClientConfig() throws Exception { return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) - .withAutoCommit(false).withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass( - OverwriteWithLatestAvroPayload - .class - .getName()).build()) + .withAutoCommit(false) .withSchema(schemaProvider.getTargetSchema().toString()) - .forTable(cfg.targetTableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .fromInputStream(fs.open(new Path(hoodieClientCfgPath))).build(); + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) + .forTable(cfg.targetTableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withProps(props).build(); } - private enum Operation { - UPSERT, INSERT + public enum Operation { + UPSERT, INSERT, BULK_INSERT } private class OperationConvertor implements IStringConverter { - @Override public Operation convert(String value) throws ParameterException { return Operation.valueOf(value); } } - private class SourceFormatConvertor implements IStringConverter { - - @Override - public SourceDataFormat convert(String value) throws ParameterException { - return SourceDataFormat.valueOf(value); - } - } - public static class Config implements Serializable { - /** - * TARGET CONFIGS - **/ - @Parameter(names = { - "--target-base-path"}, description = "base path for the target hoodie dataset", required = true) + @Parameter(names = {"--target-base-path"}, description = "base path for the target hoodie dataset. " + + "(Will be created if did not exist first time around. If exists, expected to be a hoodie dataset)", + required = true) public String targetBasePath; // TODO: How to obtain hive configs to register? - @Parameter(names = { - "--target-table"}, description = "name of the target table in Hive", required = true) + @Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true) public String targetTableName; - @Parameter(names = {"--hoodie-client-config"}, description = - "path to properties file on localfs or " - + "dfs, with hoodie client config. " - + "Sane defaults" - + "are used, but recommend use to " - + "provide basic things like metrics " - + "endpoints, hive configs etc") - public String hoodieClientProps = null; - - /** - * SOURCE CONFIGS - **/ - @Parameter(names = {"--source-class"}, description = - "subclass of com.uber.hoodie.utilities.sources" - + ".Source to use to read data. " - + "built-in options: com.uber.hoodie.utilities" - + ".common.{DFSSource (default), KafkaSource, " - + "HiveIncrPullSource}") - public String sourceClassName = DFSSource.class.getName(); - - @Parameter(names = {"--source-config"}, description = - "path to properties file on localfs or dfs, with " - + "source configs. " - + "For list of acceptable properties, refer " - + "the source class", required = true) - public String sourceConfigProps = null; - - @Parameter(names = {"--source-format"}, description = - "Format of data in source, JSON (default), Avro. " - + "All source data is " - + "converted to Avro using the provided " - + "schema in any case", converter = SourceFormatConvertor.class) - public SourceDataFormat sourceFormat = SourceDataFormat.JSON; - - @Parameter(names = {"--source-ordering-field"}, description = - "Field within source record to decide how" - + " to break ties between " - + " records with same key in input " - + "data. Default: 'ts' holding unix " - + "timestamp of record") + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are " + + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer" + + "to individual classes, for supported properties.") + public String propsFilePath = + "file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties"; + + @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, " + + "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}") + public String sourceClassName = JsonDFSSource.class.getName(); + + @Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how" + + " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record") public String sourceOrderingField = "ts"; - @Parameter(names = {"--key-generator-class"}, description = - "Subclass of com.uber.hoodie.utilities" - + ".common.KeyExtractor to generate" - + "a HoodieKey from the given avro " - + "record. Built in: SimpleKeyGenerator" - + " (Uses provided field names as " - + "recordkey & partitionpath. " - + "Nested fields specified via dot " - + "notation, e.g: a.b.c)") + @Parameter(names = {"--key-generator-class"}, description = "Subclass of com.uber.hoodie.KeyGenerator " + + "to generate a HoodieKey from the given avro record. Built in: SimpleKeyGenerator (uses " + + "provided field names as recordkey & partitionpath. Nested fields specified via dot notation, e.g: a.b.c)") public String keyGeneratorClass = SimpleKeyGenerator.class.getName(); - @Parameter(names = {"--key-generator-config"}, description = - "Path to properties file on localfs or " - + "dfs, with KeyGenerator configs. " - + "For list of acceptable properites, " - + "refer the KeyGenerator class", - required = true) - public String keyGeneratorProps = null; - - @Parameter(names = {"--payload-class"}, description = - "subclass of HoodieRecordPayload, that works off " - + "a GenericRecord. " - + "Default: SourceWrapperPayload. Implement " - + "your own, if you want to do something " - + "other than overwriting existing value") + @Parameter(names = {"--payload-class"}, description = "subclass of HoodieRecordPayload, that works off " + + "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value") public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); - @Parameter(names = {"--schemaprovider-class"}, description = - "subclass of com.uber.hoodie.utilities" - + ".schema.SchemaProvider " - + "to attach schemas to input & target" - + " table data, built in options: " - + "FilebasedSchemaProvider") + @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") public String schemaProviderClassName = FilebasedSchemaProvider.class.getName(); - @Parameter(names = {"--schemaprovider-config"}, description = - "path to properties file on localfs or dfs, with schema " - + "configs. For list of acceptable properties, refer " - + "the schema provider class", required = true) - public String schemaProviderConfigProps = null; + @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") + public long sourceLimit = Long.MAX_VALUE; - - /** - * Other configs - **/ - @Parameter(names = { - "--max-input-bytes"}, description = "Maximum number of bytes to read from source. Default: 1TB") - public long maxInputBytes = 1L * 1024 * 1024 * 1024 * 1024; - - @Parameter(names = {"--op"}, description = - "Takes one of these values : UPSERT (default), INSERT (use when input " - + "is purely new data/inserts to gain speed)", + @Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input " + + "is purely new data/inserts to gain speed)", converter = OperationConvertor.class) public Operation operation = Operation.UPSERT; + @Parameter(names = {"--spark-master"}, description = "spark master to use.") + public String sparkMaster = "local[2]"; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -408,6 +299,8 @@ public static void main(String[] args) throws Exception { cmd.usage(); System.exit(1); } - new HoodieDeltaStreamer(cfg).sync(); + + JavaSparkContext jssc = UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName, cfg.sparkMaster); + new HoodieDeltaStreamer(cfg, jssc).sync(); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java index a0c8e4a62c845..3228fdeb5e3e4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java @@ -21,6 +21,7 @@ import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.SimpleKeyGenerator; import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import java.io.Serializable; @@ -30,7 +31,6 @@ import java.util.Date; import java.util.TimeZone; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; /** * Key generator, that relies on timestamps for partitioning field. Still picks record key by name. @@ -64,7 +64,7 @@ static class Config { + ".dateformat"; } - public TimestampBasedKeyGenerator(PropertiesConfiguration config) { + public TimestampBasedKeyGenerator(TypedProperties config) { super(config); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP)); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java index aa38bf8b96adc..d4df829c43609 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java @@ -20,14 +20,14 @@ import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Arrays; import org.apache.avro.Schema; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaSparkContext; /** * A simple schema provider, that reads off files on DFS @@ -37,33 +37,30 @@ public class FilebasedSchemaProvider extends SchemaProvider { /** * Configs supported */ - static class Config { - - private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased" - + ".schemaprovider.source.schema" - + ".file"; - private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased" - + ".schemaprovider.target.schema" - + ".file"; + public static class Config { + private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider" + + ".source.schema.file"; + private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider" + + ".target.schema.file"; } private final FileSystem fs; private final Schema sourceSchema; - private final Schema targetSchema; - - public FilebasedSchemaProvider(PropertiesConfiguration config) { - super(config); - this.fs = FSUtils.getFs(config.getBasePath(), new Configuration()); + private Schema targetSchema; - DataSourceUtils.checkRequiredProperties(config, - Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP)); + public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP)); + this.fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), jssc.hadoopConfiguration()); try { this.sourceSchema = new Schema.Parser().parse( - fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); - this.targetSchema = new Schema.Parser().parse( - fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP)))); + fs.open(new Path(props.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); + if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) { + this.targetSchema = new Schema.Parser().parse( + fs.open(new Path(props.getString(Config.TARGET_SCHEMA_FILE_PROP)))); + } } catch (IOException ioe) { throw new HoodieIOException("Error reading schema", ioe); } @@ -76,6 +73,10 @@ public Schema getSourceSchema() { @Override public Schema getTargetSchema() { - return targetSchema; + if (targetSchema != null) { + return targetSchema; + } else { + return super.getTargetSchema(); + } } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaProvider.java index 3a192581c4ede..8202660d7bc53 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaProvider.java @@ -18,22 +18,29 @@ package com.uber.hoodie.utilities.schema; +import com.uber.hoodie.common.util.TypedProperties; import java.io.Serializable; import org.apache.avro.Schema; -import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.spark.api.java.JavaSparkContext; /** * Class to provide schema for reading data and also writing into a Hoodie table */ public abstract class SchemaProvider implements Serializable { - protected PropertiesConfiguration config; + protected TypedProperties config; - protected SchemaProvider(PropertiesConfiguration config) { - this.config = config; + protected JavaSparkContext jssc; + + protected SchemaProvider(TypedProperties props, JavaSparkContext jssc) { + this.config = props; + this.jssc = jssc; } public abstract Schema getSourceSchema(); - public abstract Schema getTargetSchema(); + public Schema getTargetSchema() { + // by default, use source schema as target for hoodie dataset as well + return getSourceSchema(); + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java new file mode 100644 index 0000000000000..e77a67bed3358 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java @@ -0,0 +1,71 @@ +/* + * 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.schema; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.uber.hoodie.DataSourceUtils; +import com.uber.hoodie.common.util.TypedProperties; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.net.URL; +import java.util.Arrays; +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaSparkContext; + +/** + * Obtains latest schema from the Confluent/Kafka schema-registry + * + * https://github.com/confluentinc/schema-registry + */ +public class SchemaRegistryProvider extends SchemaProvider { + + /** + * Configs supported + */ + public static class Config { + + private static final String SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url"; + } + + private final Schema schema; + + private String fetchSchemaFromRegistry(String registryUrl) throws IOException { + URL registry = new URL(registryUrl); + ObjectMapper mapper = new ObjectMapper(); + JsonNode node = mapper.readTree(registry.openStream()); + return node.get("schema").asText(); + } + + public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.SCHEMA_REGISTRY_URL_PROP)); + String registryUrl = props.getString(Config.SCHEMA_REGISTRY_URL_PROP); + try { + this.schema = new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl)); + } catch (IOException ioe) { + throw new HoodieIOException("Error reading schema from registry :" + registryUrl, ioe); + } + } + + @Override + public Schema getSourceSchema() { + return schema; + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java index 083ecb7d3e0ad..e8abfba791802 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java @@ -27,7 +27,7 @@ import org.apache.avro.generic.GenericRecord; /** - * Convert a variety of {@link SourceDataFormat} into Avro GenericRecords. Has a bunch of lazy + * Convert a variety of datum into Avro GenericRecords. Has a bunch of lazy * fields to circumvent issues around serializing these objects from driver to executors */ public class AvroConvertor implements Serializable { @@ -82,6 +82,10 @@ public GenericRecord fromJson(String json) throws IOException { return jsonConverter.convert(json); } + public Schema getSchema() { + return new Schema.Parser().parse(schemaStr); + } + public GenericRecord fromAvroBinary(byte[] avroBinary) throws IOException { initSchema(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java new file mode 100644 index 0000000000000..335d06a9b0675 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.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.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.hadoop.io.NullWritable; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +/** + * DFS Source that reads avro data + */ +public class AvroDFSSource extends DFSSource { + + public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); + } + + @Override + protected JavaRDD fromFiles(AvroConvertor convertor, String pathStr) { + JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, + AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, + sparkContext.hadoopConfiguration()); + return avroRDD.keys().map(r -> ((GenericRecord) r.datum())); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java new file mode 100644 index 0000000000000..a4d01d1b9c75f --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.utilities.sources; + +import com.uber.hoodie.common.util.TypedProperties; +import com.uber.hoodie.utilities.schema.SchemaProvider; +import io.confluent.kafka.serializers.KafkaAvroDecoder; +import kafka.serializer.StringDecoder; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.streaming.kafka.KafkaUtils; +import org.apache.spark.streaming.kafka.OffsetRange; + +/** + * Reads avro serialized Kafka data, based on the confluent schema-registry + */ +public class AvroKafkaSource extends KafkaSource { + + public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); + } + + @Override + protected JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { + JavaRDD recordRDD = KafkaUtils + .createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, kafkaParams, + offsetRanges).values().map(obj -> (GenericRecord) obj); + return recordRDD; + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java index a6bd623f6ee86..2d4d07851b2de 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java @@ -20,8 +20,10 @@ import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.util.FSUtils; +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.HoodieNotSupportedException; import com.uber.hoodie.utilities.schema.SchemaProvider; import java.io.IOException; import java.util.ArrayList; @@ -30,31 +32,23 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.mapred.AvroKey; -import org.apache.avro.mapreduce.AvroKeyInputFormat; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.io.NullWritable; -import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; /** * Source to read data from a given DFS directory structure, incrementally */ -public class DFSSource extends Source { +public abstract class DFSSource extends Source { /** * Configs supported */ static class Config { - private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; } @@ -62,50 +56,23 @@ static class Config { private final transient FileSystem fs; - public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, - SourceDataFormat dataFormat, SchemaProvider schemaProvider) { - super(config, sparkContext, dataFormat, schemaProvider); - this.fs = FSUtils.getFs(config.getBasePath(), sparkContext.hadoopConfiguration()); - DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); - } - - - public static JavaRDD fromAvroFiles(final AvroConvertor convertor, String pathStr, - JavaSparkContext sparkContext) { - JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, - AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, - sparkContext.hadoopConfiguration()); - return avroRDD.keys().map(r -> ((GenericRecord) r.datum())); - } - - public static JavaRDD fromJsonFiles(final AvroConvertor convertor, String pathStr, - JavaSparkContext sparkContext) { - return sparkContext.textFile(pathStr).map((String j) -> { - return convertor.fromJson(j); - }); - } - - public static JavaRDD fromFiles(SourceDataFormat dataFormat, - final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) { - if (dataFormat == SourceDataFormat.AVRO) { - return DFSSource.fromAvroFiles(convertor, pathStr, sparkContext); - } else if (dataFormat == SourceDataFormat.JSON) { - return DFSSource.fromJsonFiles(convertor, pathStr, sparkContext); - } else { - throw new HoodieNotSupportedException("Unsupported data format :" + dataFormat); - } + public DFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); + this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), sparkContext.hadoopConfiguration()); } + protected abstract JavaRDD fromFiles(final AvroConvertor convertor, String pathStr); @Override public Pair>, String> fetchNewData( - Optional lastCheckpointStr, long maxInputBytes) { + Optional lastCheckpointStr, long sourceLimit) { try { // obtain all eligible files under root folder. List eligibleFiles = new ArrayList<>(); RemoteIterator fitr = fs.listFiles( - new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true); + new Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), true); while (fitr.hasNext()) { LocatedFileStatus fileStatus = fitr.next(); if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream().filter( @@ -130,13 +97,14 @@ public Pair>, String> fetchNewData( continue; } - maxModificationTime = f.getModificationTime(); - currentBytes += f.getLen(); - filteredFiles.add(f); - if (currentBytes >= maxInputBytes) { + if (currentBytes + f.getLen() >= sourceLimit) { // we have enough data, we are done break; } + + maxModificationTime = f.getModificationTime(); + currentBytes += f.getLen(); + filteredFiles.add(f); } // no data to read @@ -153,7 +121,7 @@ public Pair>, String> fetchNewData( final AvroConvertor avroConvertor = new AvroConvertor(schemaStr); return new ImmutablePair<>( - Optional.of(DFSSource.fromFiles(dataFormat, avroConvertor, pathStr, sparkContext)), + Optional.of(fromFiles(avroConvertor, pathStr)), String.valueOf(maxModificationTime)); } catch (IOException ioe) { throw new HoodieIOException( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index 2b4de59eee845..faf6d0966785b 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -20,6 +20,9 @@ import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.util.FSUtils; +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.utilities.schema.SchemaProvider; import java.io.IOException; @@ -30,14 +33,15 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.NullWritable; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -69,12 +73,12 @@ static class Config { private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root"; } - public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, - SourceDataFormat dataFormat, SchemaProvider schemaProvider) { - super(config, sparkContext, dataFormat, schemaProvider); - this.fs = FSUtils.getFs(config.getBasePath(), sparkContext.hadoopConfiguration()); - DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); - this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP); + public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext, + SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); + this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP); + this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration()); } /** @@ -110,7 +114,7 @@ private Optional findCommitToPull(Optional latestTargetCommit) @Override public Pair>, String> fetchNewData( - Optional lastCheckpointStr, long maxInputBytes) { + Optional lastCheckpointStr, long sourceLimit) { try { // find the source commit to pull Optional commitToPull = findCommitToPull(lastCheckpointStr); @@ -125,10 +129,10 @@ public Pair>, String> fetchNewData( fs.listStatus(new Path(incrPullRootPath, commitToPull.get()))); String pathStr = commitDeltaFiles.stream().map(f -> f.getPath().toString()) .collect(Collectors.joining(",")); - String schemaStr = schemaProvider.getSourceSchema().toString(); - final AvroConvertor avroConvertor = new AvroConvertor(schemaStr); - return new ImmutablePair<>( - Optional.of(DFSSource.fromFiles(dataFormat, avroConvertor, pathStr, sparkContext)), + JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, + AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, + sparkContext.hadoopConfiguration()); + return new ImmutablePair<>(Optional.of(avroRDD.keys().map(r -> ((GenericRecord) r.datum()))), String.valueOf(commitToPull.get())); } catch (IOException ioe) { throw new HoodieIOException( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/SourceDataFormat.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java similarity index 51% rename from hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/SourceDataFormat.java rename to hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java index 12596a7cd144e..3167dbe592ce1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/SourceDataFormat.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java @@ -18,12 +18,23 @@ 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; + /** - * Format of the data within source. + * DFS Source that reads json data */ -public enum SourceDataFormat { - AVRO, // No conversion needed explicitly to avro - JSON, // we will try to convert to avro - ROW, // Will be added later, so we can plug/play with spark sources. - CUSTOM // the source is responsible for conversion to avro. +public class JsonDFSSource extends DFSSource { + + public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); + } + + @Override + protected JavaRDD fromFiles(AvroConvertor convertor, String pathStr) { + return sparkContext.textFile(pathStr).map((String j) -> convertor.fromJson(j)); + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java new file mode 100644 index 0000000000000..6dd111654b21a --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java @@ -0,0 +1,45 @@ +/* + * 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.sources; + +import com.uber.hoodie.common.util.TypedProperties; +import com.uber.hoodie.utilities.schema.SchemaProvider; +import kafka.serializer.StringDecoder; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.streaming.kafka.KafkaUtils; +import org.apache.spark.streaming.kafka.OffsetRange; + +/** + * Read json kafka data + */ +public class JsonKafkaSource extends KafkaSource { + + public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(properties, sparkContext, schemaProvider); + } + + @Override + protected JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { + return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class, + kafkaParams, offsetRanges) + .values().map(jsonStr -> avroConvertor.fromJson(jsonStr)); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java index f470848f1e0b1..5e0328016fa7d 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java @@ -19,33 +19,26 @@ package com.uber.hoodie.utilities.sources; import com.uber.hoodie.DataSourceUtils; +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.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.schema.SchemaProvider; -import java.nio.charset.Charset; import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Optional; -import java.util.Spliterator; -import java.util.Spliterators; import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; import kafka.common.TopicAndPartition; -import kafka.serializer.DefaultDecoder; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.log4j.LogManager; 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.KafkaUtils; +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset; import org.apache.spark.streaming.kafka.OffsetRange; import scala.Predef; import scala.collection.JavaConverters; @@ -59,10 +52,12 @@ /** * Source to read data from Kafka, incrementally */ -public class KafkaSource extends Source { +public abstract class KafkaSource extends Source { private static volatile Logger log = LogManager.getLogger(KafkaSource.class); + private static long DEFAULT_MAX_EVENTS_TO_READ = 1000000; // 1M events max + static class CheckpointUtils { @@ -72,6 +67,9 @@ static class CheckpointUtils { public static HashMap strToOffsets( String checkpointStr) { HashMap offsetMap = new HashMap<>(); + if (checkpointStr.length() == 0) { + return offsetMap; + } String[] splits = checkpointStr.split(","); String topic = splits[0]; for (int i = 1; i < splits.length; i++) { @@ -83,46 +81,70 @@ public static HashMap strToOffsets } /** - * String representation of checkpoint - *

    - * Format: topic1,0:offset0,1:offset1,2:offset2, ..... + * String representation of checkpoint

    Format: topic1,0:offset0,1:offset1,2:offset2, ..... */ - public static String offsetsToStr( - HashMap offsetMap) { + public static String offsetsToStr(OffsetRange[] ranges) { StringBuilder sb = new StringBuilder(); // atleast 1 partition will be present. - sb.append(offsetMap.entrySet().stream().findFirst().get().getKey().topic() + ","); - sb.append(offsetMap.entrySet().stream() - .map(e -> String.format("%s:%d", e.getKey().partition(), e.getValue().offset())) + sb.append(ranges[0].topic() + ","); + sb.append(Arrays.stream(ranges) + .map(r -> String.format("%s:%d", r.partition(), r.untilOffset())) .collect(Collectors.joining(","))); return sb.toString(); } + + /** + * Compute the offset ranges to read from Kafka, while handling newly added partitions, skews, event limits. + * + * @param fromOffsetMap offsets where we left off last time + * @param toOffsetMap offsets of where each partitions is currently at + * @param numEvents maximum number of events to read. + */ public static OffsetRange[] computeOffsetRanges( - HashMap fromOffsetMap, - HashMap toOffsetMap) { - Comparator byPartition = (OffsetRange o1, OffsetRange o2) -> { - return Integer.valueOf(o1.partition()).compareTo(Integer.valueOf(o2.partition())); - }; - List offsetRanges = toOffsetMap.entrySet().stream().map(e -> { + HashMap fromOffsetMap, + HashMap toOffsetMap, + long numEvents) { + + Comparator byPartition = (OffsetRange o1, OffsetRange o2) -> + Integer.valueOf(o1.partition()).compareTo(Integer.valueOf(o2.partition())); + + // Create initial offset ranges for each 'to' partition, with from = to offsets. + OffsetRange[] ranges = new OffsetRange[toOffsetMap.size()]; + toOffsetMap.entrySet().stream().map(e -> { TopicAndPartition tp = e.getKey(); - long fromOffset = -1; - if (fromOffsetMap.containsKey(tp)) { - fromOffset = fromOffsetMap.get(tp).offset(); + long fromOffset = fromOffsetMap.getOrDefault(tp, new LeaderOffset("", -1, 0)).offset(); + return OffsetRange.create(tp, fromOffset, fromOffset); + }).sorted(byPartition).collect(Collectors.toList()).toArray(ranges); + + long allocedEvents = 0; + java.util.Set exhaustedPartitions = new HashSet<>(); + // keep going until we have events to allocate and partitions still not exhausted. + while (allocedEvents < numEvents && exhaustedPartitions.size() < toOffsetMap.size()) { + long remainingEvents = numEvents - allocedEvents; + long eventsPerPartition = (long) Math + .ceil((1.0 * remainingEvents) / (toOffsetMap.size() - exhaustedPartitions.size())); + + // Allocate the remaining events to non-exhausted partitions, in round robin fashion + for (int i = 0; i < ranges.length; i++) { + OffsetRange range = ranges[i]; + if (!exhaustedPartitions.contains(range.partition())) { + long toOffsetMax = toOffsetMap.get(range.topicAndPartition()).offset(); + long toOffset = Math.min(toOffsetMax, range.untilOffset() + eventsPerPartition); + if (toOffset == toOffsetMax) { + exhaustedPartitions.add(range.partition()); + } + allocedEvents += toOffset - range.untilOffset(); + ranges[i] = OffsetRange.create(range.topicAndPartition(), range.fromOffset(), toOffset); + } } - return OffsetRange.create(tp, fromOffset, e.getValue().offset()); - }).sorted(byPartition).collect(Collectors.toList()); + } - OffsetRange[] ranges = new OffsetRange[offsetRanges.size()]; - return offsetRanges.toArray(ranges); + return ranges; } public static long totalNewMessages(OffsetRange[] ranges) { - long totalMsgs = 0; - for (OffsetRange range : ranges) { - totalMsgs += Math.max(range.untilOffset() - range.fromOffset(), 0); - } - return totalMsgs; + return Arrays.asList(ranges).stream().mapToLong(r -> r.count()).sum(); } } @@ -149,32 +171,31 @@ public static java.util.Map toJavaMap(Map m) { * Configs to be passed for this source. All standard Kafka consumer configs are also respected */ static class Config { - private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; private static final String DEFAULT_AUTO_RESET_OFFSET = "largest"; } - private HashMap kafkaParams; + protected HashMap kafkaParams; - private final String topicName; + protected final String topicName; - public KafkaSource(PropertiesConfiguration config, JavaSparkContext sparkContext, - SourceDataFormat dataFormat, SchemaProvider schemaProvider) { - super(config, sparkContext, dataFormat, schemaProvider); + public KafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); kafkaParams = new HashMap<>(); - Stream keys = StreamSupport.stream( - Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false); - keys.forEach(k -> kafkaParams.put(k, config.getString(k))); - - DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME)); - topicName = config.getString(Config.KAFKA_TOPIC_NAME); + for (Object prop : props.keySet()) { + kafkaParams.put(prop.toString(), props.getString(prop.toString())); + } + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.KAFKA_TOPIC_NAME)); + topicName = props.getString(Config.KAFKA_TOPIC_NAME); } + protected abstract JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor); + @Override public Pair>, String> fetchNewData( - Optional lastCheckpointStr, long maxInputBytes) { + Optional lastCheckpointStr, long sourceLimit) { // Obtain current metadata for the topic KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); @@ -192,7 +213,7 @@ public Pair>, String> fetchNewData( if (lastCheckpointStr.isPresent()) { fromOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get()); } else { - String autoResetValue = config + String autoResetValue = props .getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET); if (autoResetValue.equals("smallest")) { fromOffsets = new HashMap(ScalaHelpers.toJavaMap( @@ -206,40 +227,23 @@ public Pair>, String> fetchNewData( } } - // Always read until the latest offset + // Obtain the latest offsets. HashMap toOffsets = new HashMap( ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); - // Come up with final set of OffsetRanges to read (account for new partitions) - // TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from - // partition size - OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets); + // 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); + OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); if (totalNewMsgs <= 0) { - return new ImmutablePair<>(Optional.empty(), - lastCheckpointStr.isPresent() ? lastCheckpointStr.get() - : CheckpointUtils.offsetsToStr(toOffsets)); + return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); } else { log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName); } - // Perform the actual read from Kafka - JavaRDD kafkaRDD = KafkaUtils.createRDD(sparkContext, byte[].class, byte[].class, - DefaultDecoder.class, DefaultDecoder.class, kafkaParams, offsetRanges).values(); - // Produce a RDD[GenericRecord] - final AvroConvertor avroConvertor = new AvroConvertor( - schemaProvider.getSourceSchema().toString()); - JavaRDD newDataRDD; - if (dataFormat == SourceDataFormat.AVRO) { - newDataRDD = kafkaRDD.map(bytes -> avroConvertor.fromAvroBinary(bytes)); - } else if (dataFormat == SourceDataFormat.JSON) { - newDataRDD = kafkaRDD.map( - bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8")))); - } else { - throw new HoodieNotSupportedException("Unsupport data format :" + dataFormat); - } - - return new ImmutablePair<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(toOffsets)); + final AvroConvertor avroConvertor = new AvroConvertor(schemaProvider.getSourceSchema().toString()); + JavaRDD newDataRDD = toAvroRDD(offsetRanges, avroConvertor); + return new ImmutablePair<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java index fe7a805d4c2c1..06b83e9b1440d 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java @@ -18,12 +18,12 @@ 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.SchemaProvider; import java.io.Serializable; import java.util.Optional; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.commons.lang3.tuple.Pair; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -32,32 +32,23 @@ */ public abstract class Source implements Serializable { - protected transient PropertiesConfiguration config; + protected transient TypedProperties props; protected transient JavaSparkContext sparkContext; - protected transient SourceDataFormat dataFormat; - protected transient SchemaProvider schemaProvider; - protected Source(PropertiesConfiguration config, JavaSparkContext sparkContext, - SourceDataFormat dataFormat, SchemaProvider schemaProvider) { - this.config = config; + protected Source(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + this.props = props; this.sparkContext = sparkContext; - this.dataFormat = dataFormat; this.schemaProvider = schemaProvider; } /** - * Fetches new data upto maxInputBytes, from the provided checkpoint and returns an RDD of the + * Fetches new data upto sourceLimit, from the provided checkpoint and returns an RDD of the * data, as well as the checkpoint to be written as a result of that. */ public abstract Pair>, String> fetchNewData( - Optional lastCheckpointStr, long maxInputBytes); - - - public PropertiesConfiguration getConfig() { - return config; - } + Optional lastCheckpointStr, long sourceLimit); } diff --git a/hoodie-utilities/src/main/resources/delta-streamer-config/key-generator.properties b/hoodie-utilities/src/main/resources/delta-streamer-config/key-generator.properties deleted file mode 100644 index c752017804e2d..0000000000000 --- a/hoodie-utilities/src/main/resources/delta-streamer-config/key-generator.properties +++ /dev/null @@ -1,19 +0,0 @@ -# -# Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -# -# -hoodie.datasource.write.recordkey.field=_row_key -hoodie.datasource.write.partitionpath.field=driver diff --git a/hoodie-utilities/src/main/resources/delta-streamer-config/target.avsc b/hoodie-utilities/src/main/resources/delta-streamer-config/target.avsc deleted file mode 100644 index a2004f535788b..0000000000000 --- a/hoodie-utilities/src/main/resources/delta-streamer-config/target.avsc +++ /dev/null @@ -1,29 +0,0 @@ -{ - "type" : "record", - "name" : "triprec", - "fields" : [ { - "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" - } ] -} diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java new file mode 100644 index 0000000000000..d2ff8ace1c89e --- /dev/null +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java @@ -0,0 +1,162 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import com.uber.hoodie.common.model.HoodieCommitMetadata; +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.DFSPropertiesConfiguration; +import com.uber.hoodie.common.util.TypedProperties; +import com.uber.hoodie.exception.DatasetNotFoundException; +import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer; +import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.Operation; +import com.uber.hoodie.utilities.sources.TestDataSource; +import java.io.IOException; +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.sql.SQLContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Basic tests against {@link com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer}, by issuing bulk_inserts, + * upserts, inserts. Check counts at the end. + */ +public class TestHoodieDeltaStreamer extends UtilitiesTestBase { + + private static volatile Logger log = LogManager.getLogger(TestHoodieDeltaStreamer.class); + + @BeforeClass + public static void initClass() throws Exception { + UtilitiesTestBase.initClass(); + + // prepare the configs. + UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/base.properties"); + UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc"); + TypedProperties props = new TypedProperties(); + props.setProperty("include", "base.properties"); + props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there"); + props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source.properties"); + } + + @AfterClass + public static void cleanupClass() throws Exception { + UtilitiesTestBase.cleanupClass(); + } + + @Before + public void setup() throws Exception { + super.setup(); + TestDataSource.initDataGen(); + } + + @After + public void teardown() throws Exception { + super.teardown(); + TestDataSource.resetDataGen(); + } + + static class TestHelpers { + + static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op) { + HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config(); + cfg.targetBasePath = basePath; + cfg.targetTableName = "hoodie_trips"; + cfg.sourceClassName = TestDataSource.class.getName(); + cfg.operation = op; + cfg.sourceOrderingField = "timestamp"; + cfg.propsFilePath = dfsBasePath + "/test-source.properties"; + cfg.sourceLimit = 1000; + return cfg; + } + + static void assertRecordCount(long expected, String datasetPath, SQLContext sqlContext) { + long recordCount = sqlContext.read().format("com.uber.hoodie").load(datasetPath).count(); + assertEquals(expected, recordCount); + } + + static void assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits) + throws IOException { + HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieInstant lastCommit = timeline.lastInstant().get(); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class); + assertEquals(totalCommits, timeline.countInstants()); + assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY)); + } + } + + @Test + public void testProps() throws IOException { + TypedProperties props = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/test-source.properties")) + .getConfig(); + assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism")); + assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field")); + } + + @Test + public void testDatasetCreation() throws Exception { + try { + dfs.mkdirs(new Path(dfsBasePath + "/not_a_dataset")); + HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer( + TestHelpers.makeConfig(dfsBasePath + "/not_a_dataset", Operation.BULK_INSERT), jsc); + deltaStreamer.sync(); + fail("Should error out when pointed out at a dir thats not a dataset"); + } catch (DatasetNotFoundException e) { + //expected + log.error("Expected error during dataset creation", e); + } + } + + @Test + public void testBulkInsertsAndUpserts() throws Exception { + String datasetBasePath = dfsBasePath + "/test_dataset"; + + // Initial bulk insert + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT); + new HoodieDeltaStreamer(cfg, jsc).sync(); + TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); + + // No new data => no commits. + cfg.sourceLimit = 0; + new HoodieDeltaStreamer(cfg, jsc).sync(); + TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); + + // upsert() #1 + cfg.sourceLimit = 2000; + cfg.operation = Operation.UPSERT; + new HoodieDeltaStreamer(cfg, jsc).sync(); + TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2); + } +} diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java new file mode 100644 index 0000000000000..eb2d00a65d61d --- /dev/null +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java @@ -0,0 +1,137 @@ +/* + * 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.uber.hoodie.common.TestRawTripPayload; +import com.uber.hoodie.common.minicluster.HdfsTestService; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.util.TypedProperties; +import com.uber.hoodie.utilities.sources.TestDataSource; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.util.List; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +/** + * Abstract test that provides a dfs & spark contexts. + * + * TODO(vc): this needs to be done across the board. + */ +public class UtilitiesTestBase { + + protected static String dfsBasePath; + protected static HdfsTestService hdfsTestService; + protected static MiniDFSCluster dfsCluster; + protected static DistributedFileSystem dfs; + protected transient JavaSparkContext jsc = null; + protected transient SQLContext sqlContext; + + @BeforeClass + public static void initClass() throws Exception { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + dfs = dfsCluster.getFileSystem(); + dfsBasePath = dfs.getWorkingDirectory().toString(); + dfs.mkdirs(new Path(dfsBasePath)); + } + + @AfterClass + public static void cleanupClass() throws Exception { + if (hdfsTestService != null) { + hdfsTestService.stop(); + } + } + + @Before + public void setup() throws Exception { + TestDataSource.initDataGen(); + jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]"); + sqlContext = new SQLContext(jsc); + } + + @After + public void teardown() throws Exception { + TestDataSource.resetDataGen(); + if (jsc != null) { + jsc.stop(); + } + } + + public static class Helpers { + + // to get hold of resources bundled with jar + private static ClassLoader classLoader = Helpers.class.getClassLoader(); + + public static void copyToDFS(String testResourcePath, FileSystem fs, String targetPath) throws IOException { + BufferedReader reader = new BufferedReader( + new InputStreamReader(classLoader.getResourceAsStream(testResourcePath))); + PrintStream os = new PrintStream(fs.create(new Path(targetPath), true)); + String line; + while ((line = reader.readLine()) != null) { + os.println(line); + } + os.flush(); + os.close(); + } + + public static void savePropsToDFS(TypedProperties props, FileSystem fs, String targetPath) throws IOException { + String[] lines = props.keySet().stream().map(k -> String.format("%s=%s", k, props.get(k))).toArray(String[]::new); + saveStringsToDFS(lines, fs, targetPath); + } + + public static void saveStringsToDFS(String[] lines, FileSystem fs, String targetPath) throws IOException { + PrintStream os = new PrintStream(fs.create(new Path(targetPath), true)); + for (String l : lines) { + os.println(l); + } + os.flush(); + os.close(); + } + + public static TypedProperties setupSchemaOnDFS() throws IOException { + UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc"); + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); + return props; + } + + public static String toJsonString(HoodieRecord hr) { + try { + return ((TestRawTripPayload) hr.getData()).getJsonData(); + } catch (IOException ioe) { + return null; + } + } + + public static String[] jsonifyRecords(List records) throws IOException { + return records.stream().map(Helpers::toJsonString).toArray(String[]::new); + } + } +} diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java new file mode 100644 index 0000000000000..d460e6ea8527a --- /dev/null +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java @@ -0,0 +1,104 @@ +/* + * 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.sources; + +import static org.junit.Assert.assertEquals; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +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.schema.FilebasedSchemaProvider; +import java.io.IOException; +import java.util.Optional; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Basic tests against all subclasses of {@link DFSSource} + */ +public class TestDFSSource extends UtilitiesTestBase { + + private FilebasedSchemaProvider schemaProvider; + + @BeforeClass + public static void initClass() throws Exception { + UtilitiesTestBase.initClass(); + } + + @AfterClass + public static void cleanupClass() throws Exception { + UtilitiesTestBase.cleanupClass(); + } + + @Before + public void setup() throws Exception { + super.setup(); + schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc); + } + + @After + public void teardown() throws Exception { + super.teardown(); + } + + @Test + public void testJsonDFSSource() throws IOException { + dfs.mkdirs(new Path(dfsBasePath + "/jsonFiles")); + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/jsonFiles"); + JsonDFSSource jsonSource = new JsonDFSSource(props, jsc, schemaProvider); + + // 1. Extract without any checkpoint => get all the data, respecting sourceLimit + assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), Long.MAX_VALUE).getKey()); + UtilitiesTestBase.Helpers.saveStringsToDFS( + Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs, + dfsBasePath + "/jsonFiles/1.json"); + assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), 10).getKey()); + Pair>, String> fetch1 = jsonSource.fetchNewData(Optional.empty(), 1000000); + assertEquals(100, fetch1.getKey().get().count()); + + // 2. Produce new data, extract new data + UtilitiesTestBase.Helpers.saveStringsToDFS( + Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)), + dfs, dfsBasePath + "/jsonFiles/2.json"); + Pair>, String> fetch2 = jsonSource.fetchNewData( + Optional.of(fetch1.getValue()), Long.MAX_VALUE); + assertEquals(10000, fetch2.getKey().get().count()); + + // 3. Extract with previous checkpoint => gives same data back (idempotent) + Pair>, String> fetch3 = jsonSource.fetchNewData( + Optional.of(fetch1.getValue()), Long.MAX_VALUE); + assertEquals(10000, fetch3.getKey().get().count()); + assertEquals(fetch2.getValue(), fetch3.getValue()); + + // 4. Extract with latest checkpoint => no new data returned + Pair>, String> fetch4 = jsonSource.fetchNewData( + Optional.of(fetch2.getValue()), Long.MAX_VALUE); + assertEquals(Optional.empty(), fetch4.getKey()); + } +} diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java new file mode 100644 index 0000000000000..f4c589a00269f --- /dev/null +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java @@ -0,0 +1,99 @@ +/* + * 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.sources; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +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 java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +/** + * An implementation of {@link Source}, that emits test upserts. + */ +public class TestDataSource extends Source { + + private static volatile Logger log = LogManager.getLogger(TestDataSource.class); + + // Static instance, helps with reuse across a test. + private static HoodieTestDataGenerator dataGenerator; + + public static void initDataGen() { + dataGenerator = new HoodieTestDataGenerator(); + } + + public static void resetDataGen() { + dataGenerator = null; + } + + public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { + super(props, sparkContext, schemaProvider); + } + + private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) { + try { + Optional recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.avroSchema); + return (GenericRecord) recordOpt.get(); + } catch (IOException e) { + return null; + } + } + + @Override + public Pair>, String> fetchNewData(Optional lastCheckpointStr, + long sourceLimit) { + + int nextCommitNum = lastCheckpointStr.isPresent() ? Integer.parseInt(lastCheckpointStr.get()) + 1 : 0; + String commitTime = String.format("%05d", nextCommitNum); + // No new data. + if (sourceLimit <= 0) { + return new ImmutablePair<>(Optional.empty(), commitTime); + } + + // generate `sourceLimit` number of upserts each time. + int numExistingKeys = dataGenerator.getExistingKeysList().size(); + int numUpdates = Math.min(numExistingKeys, (int) sourceLimit / 2); + int numInserts = (int) sourceLimit - numUpdates; + + List records = new ArrayList<>(); + try { + records.addAll(dataGenerator.generateUniqueUpdates(commitTime, numUpdates).stream() + .map(this::toGenericRecord).collect(Collectors.toList())); + records.addAll(dataGenerator.generateInserts(commitTime, numInserts).stream() + .map(this::toGenericRecord).collect(Collectors.toList())); + } catch (IOException e) { + log.error("Error generating test data.", e); + } + + JavaRDD avroRDD = sparkContext.parallelize(records, 4); + return new ImmutablePair<>(Optional.of(avroRDD), commitTime); + } +} diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java new file mode 100644 index 0000000000000..785e805694dd9 --- /dev/null +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java @@ -0,0 +1,186 @@ +/* + * 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.sources; + +import static com.uber.hoodie.utilities.sources.KafkaSource.CheckpointUtils; +import static org.junit.Assert.assertEquals; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +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.schema.FilebasedSchemaProvider; +import java.io.IOException; +import java.util.HashMap; +import java.util.Optional; +import kafka.common.TopicAndPartition; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset; +import org.apache.spark.streaming.kafka.KafkaTestUtils; +import org.apache.spark.streaming.kafka.OffsetRange; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests against {@link KafkaSource} + */ +public class TestKafkaSource extends UtilitiesTestBase { + + private static String TEST_TOPIC_NAME = "hoodie_test"; + + private FilebasedSchemaProvider schemaProvider; + private KafkaTestUtils testUtils; + + @BeforeClass + public static void initClass() throws Exception { + UtilitiesTestBase.initClass(); + } + + @AfterClass + public static void cleanupClass() throws Exception { + UtilitiesTestBase.cleanupClass(); + } + + @Before + public void setup() throws Exception { + super.setup(); + schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc); + testUtils = new KafkaTestUtils(); + testUtils.setup(); + } + + @After + public void teardown() throws Exception { + super.teardown(); + testUtils.teardown(); + } + + + @Test + public void testJsonKafkaSource() throws IOException { + + // topic setup. + testUtils.createTopic(TEST_TOPIC_NAME, 2); + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME); + props.setProperty("metadata.broker.list", testUtils.brokerAddress()); + props.setProperty("auto.offset.reset", "smallest"); + props.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + + Source kafkaSource = new JsonKafkaSource(props, jsc, schemaProvider); + + // 1. Extract without any checkpoint => get all the data, respecting sourceLimit + assertEquals(Optional.empty(), kafkaSource.fetchNewData(Optional.empty(), Long.MAX_VALUE).getKey()); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + Pair>, String> fetch1 = kafkaSource.fetchNewData(Optional.empty(), 900); + assertEquals(900, fetch1.getKey().get().count()); + + // 2. Produce new data, extract new data + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); + Pair>, String> fetch2 = kafkaSource.fetchNewData( + Optional.of(fetch1.getValue()), Long.MAX_VALUE); + assertEquals(1100, fetch2.getKey().get().count()); + + // 3. Extract with previous checkpoint => gives same data back (idempotent) + Pair>, String> fetch3 = kafkaSource.fetchNewData( + Optional.of(fetch1.getValue()), Long.MAX_VALUE); + assertEquals(fetch2.getKey().get().count(), fetch3.getKey().get().count()); + assertEquals(fetch2.getValue(), fetch3.getValue()); + + // 4. Extract with latest checkpoint => no new data returned + Pair>, String> fetch4 = kafkaSource.fetchNewData( + Optional.of(fetch2.getValue()), Long.MAX_VALUE); + assertEquals(Optional.empty(), fetch4.getKey()); + } + + + private static HashMap makeOffsetMap(int[] partitions, long[] offsets) { + HashMap map = new HashMap<>(); + for (int i = 0; i < partitions.length; i++) { + map.put(new TopicAndPartition(TEST_TOPIC_NAME, partitions[i]), new LeaderOffset("", -1, offsets[i])); + } + return map; + } + + @Test + public void testComputeOffsetRanges() { + // test totalNewMessages() + long totalMsgs = CheckpointUtils.totalNewMessages(new OffsetRange[]{ + OffsetRange.apply(TEST_TOPIC_NAME, 0, 0, 100), + OffsetRange.apply(TEST_TOPIC_NAME, 0, 100, 200) + }); + assertEquals(200, totalMsgs); + + // should consume all the full data + OffsetRange[] ranges = CheckpointUtils.computeOffsetRanges( + makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}), + makeOffsetMap(new int[]{0, 1}, new long[]{300000, 350000}), + 1000000L + ); + assertEquals(200000, CheckpointUtils.totalNewMessages(ranges)); + + // should only consume upto limit + ranges = CheckpointUtils.computeOffsetRanges( + makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}), + makeOffsetMap(new int[]{0, 1}, new long[]{300000, 350000}), + 10000 + ); + assertEquals(10000, CheckpointUtils.totalNewMessages(ranges)); + assertEquals(200000, ranges[0].fromOffset()); + assertEquals(205000, ranges[0].untilOffset()); + assertEquals(250000, ranges[1].fromOffset()); + assertEquals(255000, ranges[1].untilOffset()); + + // should also consume from new partitions. + ranges = CheckpointUtils.computeOffsetRanges( + makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}), + makeOffsetMap(new int[]{0, 1, 2}, new long[]{300000, 350000, 100000}), + 1000000L + ); + assertEquals(300000, CheckpointUtils.totalNewMessages(ranges)); + assertEquals(3, ranges.length); + + // for skewed offsets, does not starve any partition & can catch up + ranges = CheckpointUtils.computeOffsetRanges( + makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}), + makeOffsetMap(new int[]{0, 1, 2}, new long[]{200010, 350000, 10000}), + 100000 + ); + assertEquals(100000, CheckpointUtils.totalNewMessages(ranges)); + assertEquals(10, ranges[0].count()); + assertEquals(89990, ranges[1].count()); + assertEquals(10000, ranges[2].count()); + + ranges = CheckpointUtils.computeOffsetRanges( + makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}), + makeOffsetMap(new int[]{0, 1, 2}, new long[]{200010, 350000, 10000}), + 1000000 + ); + assertEquals(110010, CheckpointUtils.totalNewMessages(ranges)); + assertEquals(10, ranges[0].count()); + assertEquals(100000, ranges[1].count()); + assertEquals(10000, ranges[2].count()); + } +} diff --git a/hoodie-utilities/src/main/resources/IncrementalPull.sqltemplate b/hoodie-utilities/src/test/resources/IncrementalPull.sqltemplate similarity index 100% rename from hoodie-utilities/src/main/resources/IncrementalPull.sqltemplate rename to hoodie-utilities/src/test/resources/IncrementalPull.sqltemplate diff --git a/hoodie-utilities/src/main/resources/delta-streamer-config/hoodie-client.properties b/hoodie-utilities/src/test/resources/delta-streamer-config/base.properties similarity index 86% rename from hoodie-utilities/src/main/resources/delta-streamer-config/hoodie-client.properties rename to hoodie-utilities/src/test/resources/delta-streamer-config/base.properties index 81f928b0157fa..9075810065aa6 100644 --- a/hoodie-utilities/src/main/resources/delta-streamer-config/hoodie-client.properties +++ b/hoodie-utilities/src/test/resources/delta-streamer-config/base.properties @@ -15,4 +15,7 @@ # # # +# Common hoodie client configs hoodie.upsert.shuffle.parallelism=2 +hoodie.insert.shuffle.parallelism=2 +hoodie.bulkinsert.shuffle.parallelism=2 diff --git a/hoodie-utilities/src/main/resources/delta-streamer-config/schema-provider.properties b/hoodie-utilities/src/test/resources/delta-streamer-config/dfs-source.properties similarity index 60% rename from hoodie-utilities/src/main/resources/delta-streamer-config/schema-provider.properties rename to hoodie-utilities/src/test/resources/delta-streamer-config/dfs-source.properties index 1842069de8762..90c524088d48e 100644 --- a/hoodie-utilities/src/main/resources/delta-streamer-config/schema-provider.properties +++ b/hoodie-utilities/src/test/resources/delta-streamer-config/dfs-source.properties @@ -14,5 +14,14 @@ # limitations under the License. # # -hoodie.deltastreamer.filebased.schemaprovider.source.schema.file=file:///Users/vinoth/bin/hoodie/hoodie-utilities/src/main/resources/delta-streamer-config/source.avsc -hoodie.deltastreamer.filebased.schemaprovider.target.schema.file=file:///Users/vinoth/bin/hoodie/hoodie-utilities/src/main/resources/delta-streamer-config/target.avsc +# +include=base.properties +# Key generator props +hoodie.datasource.write.recordkey.field=_row_key +hoodie.datasource.write.partitionpath.field=driver +# Schema provider props (change to absolute path based on your installation) +hoodie.deltastreamer.filebased.schemaprovider.source.schema.file=file:///path/to/hoodie/hoodie-utilities/src/main/resources/delta-streamer-props/source.avsc +hoodie.deltastreamer.filebased.schemaprovider.target.schema.file=file:///path/to/hoodie/hoodie-utilities/src/main/resources/delta-streamer-props/target.avsc +# DFS Source +hoodie.deltastreamer.source.dfs.root=file:///tmp/hoodie-dfs-input + diff --git a/hoodie-utilities/src/main/resources/delta-streamer-config/source.properties b/hoodie-utilities/src/test/resources/delta-streamer-config/kafka-source.properties similarity index 60% rename from hoodie-utilities/src/main/resources/delta-streamer-config/source.properties rename to hoodie-utilities/src/test/resources/delta-streamer-config/kafka-source.properties index 6e698db1d732e..156890ce07b74 100644 --- a/hoodie-utilities/src/main/resources/delta-streamer-config/source.properties +++ b/hoodie-utilities/src/test/resources/delta-streamer-config/kafka-source.properties @@ -15,9 +15,16 @@ # # # -# DFS Source -hoodie.deltastreamer.source.dfs.root=file:///tmp/hoodie-dfs-input +include=base.properties +# Key fields, for kafka example +hoodie.datasource.write.recordkey.field=impressionid +hoodie.datasource.write.partitionpath.field=userid +# schema provider configs +hoodie.deltastreamer.schemaprovider.registry.url=http://localhost:8081/subjects/impressions-value/versions/latest # Kafka Source -hoodie.deltastreamer.source.kafka.topic=uber_trips +#hoodie.deltastreamer.source.kafka.topic=uber_trips +hoodie.deltastreamer.source.kafka.topic=impressions +#Kafka props metadata.broker.list=localhost:9092 auto.offset.reset=smallest +schema.registry.url=http://localhost:8081 diff --git a/hoodie-utilities/src/main/resources/delta-streamer-config/source.avsc b/hoodie-utilities/src/test/resources/delta-streamer-config/source.avsc similarity index 86% rename from hoodie-utilities/src/main/resources/delta-streamer-config/source.avsc rename to hoodie-utilities/src/test/resources/delta-streamer-config/source.avsc index 11a5649d05ef2..ef7e0273d4470 100644 --- a/hoodie-utilities/src/main/resources/delta-streamer-config/source.avsc +++ b/hoodie-utilities/src/test/resources/delta-streamer-config/source.avsc @@ -1,7 +1,11 @@ { "type" : "record", "name" : "triprec", - "fields" : [ { + "fields" : [ + { + "name" : "timestamp", + "type" : "double" + }, { "name" : "_row_key", "type" : "string" }, { diff --git a/pom.xml b/pom.xml index 9f3394a7272e9..d321ddb15267d 100644 --- a/pom.xml +++ b/pom.xml @@ -116,6 +116,7 @@ 2.10 2.6 2.19.1 + 2.6.5 1.8.1 4.11 1.9.5 @@ -376,6 +377,16 @@ hadoop-client ${hadoop.version}-cdh${cdh.version} provided + + + com.fasterxml.jackson.* + * + + + javax.servlet + servlet-api + + @@ -427,6 +438,12 @@ hive-common ${hive.version}-cdh${cdh.version} provided + + + com.fasterxml.jackson.* + * + + org.apache.hadoop @@ -482,12 +499,28 @@ spark-core_2.11 ${spark.version} provided + + + com.fasterxml.jackson.** + * + + + javax.servlet + servlet-api + + org.apache.spark spark-sql_2.11 ${spark.version} provided + + + com.fasterxml.jackson.** + * + + @@ -561,8 +594,33 @@ com.fasterxml.jackson.core jackson-annotations - 2.6.0 + ${fasterxml.version} + + + + com.fasterxml.jackson.core + jackson-core + ${fasterxml.version} + + + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.version} + + + com.fasterxml.jackson.module + jackson-module-scala_2.11 + ${fasterxml.version} + + + + org.codehaus.jackson + jackson-core-asl + 1.9.13 + + org.codehaus.jackson jackson-mapper-asl @@ -573,6 +631,12 @@ org.apache.hive hive-jdbc ${hive.version}-cdh${cdh.version} + + + com.fasterxml.jackson.* + * + + @@ -596,6 +660,26 @@ hadoop-hdfs tests ${hadoop.version}-cdh${cdh.version} + + + org.codehaus + * + + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + test org.apache.hadoop From 6821fbd79793518d00440a787278f385ff48406c Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 6 Sep 2018 00:49:38 -0700 Subject: [PATCH 091/374] Docs for describing async compaction and how to operate it --- docs/admin_guide.md | 149 ++++++++++++++++++++++++++- docs/concepts.md | 17 ++++ docs/images/async_compac_1.png | Bin 0 -> 60344 bytes docs/images/async_compac_2.png | Bin 0 -> 54164 bytes docs/images/async_compac_3.png | Bin 0 -> 70516 bytes docs/images/async_compac_4.png | Bin 0 -> 66932 bytes docs/implementation.md | 180 +++++++++++++++++++++++++++++++-- 7 files changed, 339 insertions(+), 7 deletions(-) create mode 100644 docs/images/async_compac_1.png create mode 100644 docs/images/async_compac_2.png create mode 100644 docs/images/async_compac_3.png create mode 100644 docs/images/async_compac_4.png diff --git a/docs/admin_guide.md b/docs/admin_guide.md index 42b2185d63ca2..9b8c9981e44a1 100644 --- a/docs/admin_guide.md +++ b/docs/admin_guide.md @@ -21,6 +21,42 @@ Once hoodie has been built via `mvn clean install -DskipTests`, the shell can be A hoodie dataset resides on HDFS, in a location referred to as the **basePath** and we would need this location in order to connect to a Hoodie dataset. Hoodie library effectively manages this HDFS dataset internally, using .hoodie subfolder to track all metadata +To initialize a hoodie table, use the following command. + +``` +18/09/06 15:56:52 INFO annotation.AutowiredAnnotationBeanPostProcessor: JSR-330 'javax.inject.Inject' annotation found and supported for autowiring +============================================ +* * +* _ _ _ _ * +* | | | | | (_) * +* | |__| | ___ ___ __| |_ ___ * +* | __ |/ _ \ / _ \ / _` | |/ _ \ * +* | | | | (_) | (_) | (_| | | __/ * +* |_| |_|\___/ \___/ \__,_|_|\___| * +* * +============================================ + +Welcome to Hoodie CLI. Please type help if you are looking for help. +hoodie->create --path /user/hive/warehouse/table1 --tableName hoodie_table_1 --tableType COPY_ON_WRITE +..... +18/09/06 15:57:15 INFO table.HoodieTableMetaClient: Finished Loading Table of type COPY_ON_WRITE from ... +``` + +To see the description of hoodie table, use the command: +``` +hoodie:hoodie_table_1->desc +18/09/06 15:57:19 INFO timeline.HoodieActiveTimeline: Loaded instants [] + _________________________________________________________ + | Property | Value | + |========================================================| + | basePath | ... | + | metaPath | ... | + | fileSystem | hdfs | + | hoodie.table.name | hoodie_table_1 | + | hoodie.table.type | COPY_ON_WRITE | + | hoodie.archivelog.folder| | +``` + Following is a sample command to connect to a Hoodie dataset contains uber trips. ``` @@ -135,7 +171,7 @@ hoodie:trips->stats filesizes --partitionPath 2016/09/01 --sortBy "95th" --desc ________________________________________________________________________________________________ | CommitTime | Min | 10th | 50th | avg | 95th | Max | NumFiles| StdDev | |===============================================================================================| - | 20161004211210| 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 2 | 2.3 KB | + | | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 2 | 2.3 KB | .... .... ``` @@ -158,6 +194,117 @@ hoodie:trips->stats wa In order to limit the amount of growth of .commit files on HDFS, Hoodie archives older .commit files (with due respect to the cleaner policy) into a commits.archived file. This is a sequence file that contains a mapping from commitNumber => json with raw information about the commit (same that is nicely rolled up above). + +#### Compactions + +To get an idea of the lag between compaction and writer applications, use the below command to list down all +pending compactions. + +``` +hoodie:trips->compactions show all + ___________________________________________________________________ + | Compaction Instant Time| State | Total FileIds to be Compacted| + |==================================================================| + | | REQUESTED| 35 | + | | INFLIGHT | 27 | + +``` + +To inspect a specific compaction plan, use + +``` +hoodie:trips->compaction show --instant + _________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________ + | Partition Path| File Id | Base Instant | Data File Path | Total Delta Files| getMetrics | + |================================================================================================================================================================================================================================================ + | 2018/07/17 | | | viewfs://ns-default/.../../UUID_.parquet | 1 | {TOTAL_LOG_FILES=1.0, TOTAL_IO_READ_MB=1230.0, TOTAL_LOG_FILES_SIZE=2.51255751E8, TOTAL_IO_WRITE_MB=991.0, TOTAL_IO_MB=2221.0}| + +``` + +To manually schedule or run a compaction, use the below command. This command uses spark launcher to perform compaction +operations. NOTE : Make sure no other application is scheduling compaction for this dataset concurrently + +``` +hoodie:trips->help compaction schedule +Keyword: compaction schedule +Description: Schedule Compaction + Keyword: sparkMemory + Help: Spark executor memory + Mandatory: false + Default if specified: '__NULL__' + Default if unspecified: '1G' + +* compaction schedule - Schedule Compaction +``` + +``` +hoodie:trips->help compaction run +Keyword: compaction run +Description: Run Compaction for given instant time + Keyword: tableName + Help: Table name + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: parallelism + Help: Parallelism for hoodie compaction + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: schemaFilePath + Help: Path for Avro schema file + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: sparkMemory + Help: Spark executor memory + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: retry + Help: Number of retries + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: compactionInstant + Help: Base path for the target hoodie dataset + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + +* compaction run - Run Compaction for given instant time +``` + +##### Up-Coming CLI for Compaction + +In the next release, more useful CLI to revert/repair compaction schedules will be added. Here is a preview of them: + +Validating a compaction plan : Check if all the files necessary for compactions are present and are valid + +``` +hoodie:trips->compaction validate --compactionInstant +``` + +The following commands must be executed without any other writer/ingestion application running. + +Sometimes, it becomes necessary to remove a fileId from a compaction-plan inorder to speed-up or unblock compaction +operation. Any new log-files that happened on this file after the compaction got scheduled will be safely renamed +so that are preserved. Hudi provides the following CLI to support it + +``` +hoodie:trips->compaction unscheduleFileId --fileId +``` + +In other cases, an entire compaction plan needs to be reverted. This is supported by the following CLI +``` +hoodie:trips->compaction unschedule --compactionInstant +``` + ## Metrics Once the Hoodie Client is configured with the right datasetname and environment for metrics, it produces the following graphite metrics, that aid in debugging hoodie datasets diff --git a/docs/concepts.md b/docs/concepts.md index 8de1915a545e6..4097efbb21aa0 100644 --- a/docs/concepts.md +++ b/docs/concepts.md @@ -62,6 +62,23 @@ Each record is uniquely identified by a `record key` and mapped to a file id for and file id, never changes once the first version of a record has been written to a file. In short, the `file id` identifies a group of files, that contain all versions of a group of records. +## Terminologies + + * `Hudi Dataset` + A structured hive/spark table managed by Hudi. Hudi supports both partitioned and non-partitioned Hive tables. + * `Commit` + A commit marks a new batch of data applied to a dataset. Hudi maintains monotonically increasing timestamps to track commits and guarantees that a commit is atomically + published. + * `Commit Timeline` + Commit Timeline refers to the sequence of Commits that was applied in order on a dataset over its lifetime. + * `File Slice` + Hudi provides efficient handling of updates by having a fixed mapping between record key to a logical file Id. + Hudi uses MVCC to provide atomicity and isolation of readers from a writer. This means that a logical fileId will + have many physical versions of it. Each of these physical version of a file represents a complete view of the + file as of a commit and is called File Slice + * `File Group` + A file-group is a file-slice timeline. It is a list of file-slices in commit order. It is identified by `file id` + ## Copy On Write diff --git a/docs/images/async_compac_1.png b/docs/images/async_compac_1.png new file mode 100644 index 0000000000000000000000000000000000000000..2bfbff9012405a2f3de6de379b71e30b40a260cf GIT binary patch literal 60344 zcmcG0by!yGx2+;bmvoCLB8`A_mk1&$B_%B&Al=ePW6>=jA>AnDODuYY}8;kGgE9Vro0k=64$`L&9Sf}d-;WD~~yyxDU76O-rrf)Xdn zd%0N`2LjYSIHD-jb6h(Y+=#z_=_8+n`ujJNh4J7|;P*Uf|NH^M;)8?6 z;s5&Yum0Qy-{j8~bUpvmtrl?pT=9Ro>3_ZzIl}+zrvK&n#DxFzlm6$M{^#ejLHTpV z|Kg?p-U{pL-(UT2pRbep=Kr*k|I@8_{(o*to$aGXk8a(%h4n5{PFY#m*myD93WHq8 zJx-`)Qq<6JzR0-yF!3S0-5e+0(&px-!=#(4l$0@+(;Ey@j)xi=$D@{YZUmv8+BME0 zB<#&&V{vugWyB;T=Do26%N@67er7y5`nT)z>duda&S!;Qj+OU{YTwGr%1V_Cah59v$5N&yR3~F7E=@!kz#8JviroKm6Zs zdig<@SHNQ=|MPtR{o~T+#S52}l*}(IOjJ9uGchqSFl48vOOBn&m*wZn4>4c`3b^d& zq@<*jm#gu+yStwU`4*&QW=eBoDG2iMolH0S3puFWGMAB&QBdgpqFE+iUQtm|RHWo; zX=lf|E#46vjPa?rx0iZVw7|;?0nyUZk_bB>-Ey+l?OurxH5U`nv9L(7{qyLwmYKq0yd}lOM%@W(b8~a+>+93gdZwlo z?q|m*2OG1ko{Tf_@ad_ksYS%7sHiAahTjRz>l`)`mWW5oty8nJ8PJm2quBIwzI-VN zWr~iD_Bw$>(%#nQ>*FIlrX#yh?~M%aSXo)QzrXM4>Dh|B@-V%mMCBpDFY5b>XecQ8 z1qHo5Jyup$I9_7yOpJ`QPZC^D53M6q#L|gawE4}4Sm6bWQ4~JDf8RGa*uvxe(96us zEaP$P1FNa8pFh*RN5#gLx*CTnE+v)9GiUkYMPRLNDl1AvM1%l&{6FWRP4jk-y1M#o zOIS;PObrLEnOB0jNsi{Bv`jizFQ*v`_U5~cm;jgpxh@i|A8h+QEprhe! z)-4eAOiD_EZ80`5fJc~R2*xvId{%DN985BlmXhM7C6GItVn@u1akAW>&H8I$!RV9i z`KgnK$Hk8yiYNlrBQneefaPpf)bIJkr6}a>Z!~6 zSVgvKUUS>)kdTm@J#^mm8X6i)q^l$zH#beqBj5>8Ib2B*n6^(oeY(9n-Fet|IUP0nV>f<^ zkB>juUyBV7zbRNpvd%zFZGC=vl&Jiqcr@b))sh`{5GFRZ|H|bX=jLL)ii?X2rj?bI z4d+2vUS3X5OPkLA^5x6T9{2&`1=a*-Wn@G{M@NT>imES$!&K40;P$_S|3!juqsh+B zPG?8Qr?fN%28N)ZAnG0l)m-zJFJHP~A&4@2R0*eUCJ4AdTutSepd>hNPKkW}Qd(Nd z`cvbx7n~S`6a^6#1>E+P#YO+yHm_bOtEea-88Pmn5SYDq!N|zi97^G`F;UapEUE0j z@Ki;m6O!uL@!rz6k4%~xnwkZ-n0w~u=OH`L)6*{zAv7cu6&4y88bUJrn=8-1AXu1~ zpu|2ac&TTw7a%>`-LX z*`V#>auPzub^ZGFhJ=Fo248fO;>=7`1UEQ1yF}LMve5`U%uGza$g#1pLFgRYB7?d^ zH*Va>e)`4vaFg2gZ&`q3yEJ*b2eQ{)xi}PT?681<*XPa6%|-?Wut=!bF(#&_Z$d)_ zt)_&V!ldQoUYeOzIqfW*pPgV+@ZKJApKm7+A&1onrQq{h=00t2Z=XuzH`~v+rAYYVR?@Z#vd?USO>ztmMLM4M^3G36nu2|Rvu$BvflVc7zJ6UtVu!8?y$t>-E>>1m9UL4Ka^8KYEWB&5 z6ca-_t=>23FgX_)6?GUGclY1R^9AZyt=ni%&*oRpYdA9m3{O92W%2UxRFs!T#l~u? zt3$2It*(BSZ-6mATx1NNm+>TVeQHWqUq3Z1t;Bh6xw4X%yzS`dh%Ricwzf7VCMNcQ z1*D|*uC9T>L2OJ+f_1D<|CC^;w8wjdO9KN>2?#zw?h*4USceT;ThoH%rlh2#qJngH ze|I;IO-yn`f-Ud}(-fB8ryT~X6!f})SBUAw*A|g#qO$D`oK?ypl zp7vH&QM`6pNu?}ky9eXg8Xw=jZIBGu&Pa4${`Kn!{TOtrNWrZTx( zBbhV@>O4Grj$N=F{}yiiC85A~@7{TNc}+F3pmBtD*SH*Dk=&o_Nfvvk0!d3+`W_99 z*lBb9*~x*JG+P$yD%1=Ob#<+BtK`zsr>DamkPNO}$M~F^`xo~7S#y)6jixtvkk}-Y zxNYt195yB>NJ%S=I!Uixy9V$QHd!>(I%fh-LPv+JhQ>%&7c)1v1Aeo+@&wB4(W@hFu74E*jX;(h!f*b-px!2=c?@(_i!>c%@-_6;-BHT-S`QPKv z{~2TgI{wq>{J!?TM*IIBp#J-Z1M2-70l%^M{}bH)f#!ef%fF_Yf5Y)kW{8y@7w52r zymN-sG&DPj7e|SIA6Hw|^Qn>&9{?KD?Lrh3l*1OGbCJKP(vHA{kFN^q%Y*GgZu?c@ z3@dmZ!0p?l=p2i}bq3tXf6(`TzY_nqKH*=${r|8H|FRPQsmJ&a*ZzkF;=k<^{-vY& zUpF6fcKDEZW5UD3ApjuTUb5=+mQPJ=HNJT_pwcNc6P!lNxClWK{wzV#f=xmZrt_y>b1D| z_ySy0Y_bQk3JP289~f@c*FcUax1K%T{pGDZIx>PoE_BQ94hswRH7|fSFUi@BI^N!q z@eyh5=DBq^>qw)Wv^R= zWQh^c`qz&%XJ_Yp#a_1S(iUsi3He+PHvyLdEOwQc4h%#?M@Oe);a5{p**iVj5x~c@ zhS#MaC)dpdWQ2lrx2wH9Ffg!TaCLQ+gx%=tU#6O{wY`JttdIunsHEg|(aZoWvO#T!8eo56C|o zXlZFlNwowaFg=G(22NSE%Rz2FIu9Z;vY70SqXgKl}KI%gP3q zJ2!a4I&V5H_hkyY9h39f%eo#TX6b*2iZ(kl6Mh2`(s6xvH}-HtSn>$qOXyGi#Ttu{ zJpHOtso@iMv668Jqo?qmQ;?9{W9~_jrfns06N%P|s`$4AvqoT&vpP{z1x1aAhX)#> zeB&oip5T9X>4NI%zK*urjmi-UT*$Y3O6f`MBPC`~*Wm!7s}NAp5wGRzHBJpT#2G2v z2bf`LVF9JVmo7Ee(E-O5P&xc7BQ5P2Dwf2eQ;Xpbh{k^pnjahZFD)$%k{SSiFX8O$ zY#P1d=a!ZpC;Ju#UeBQ~dq?{?&gKy)sur*^T4 z0yHNj#PDD}&xZ2-{4U`S^hFk2DX}|ENTl@8f2L<=qoex07pn&VJn%*H?%lf%v(1>j zuFp9ZXJ!zbpb3&95r>AQvC`A?>I)SW6&^l52uDh8t1I^CTFr!r0I_`$2LTf+ao?o~^AYiKa-lDOfqy2>U zw$f@^^pKQ=UH z!r=l`i~}3}1+Zd{zn|ao(h^D-ChiTHCr_lLx!L~d`0xO&$xF6AH!?Dklf#n5UkQiL zFq&>)T_z+VLU;M*v^jOvys)tF2C|yHeW?lGGTH>HjRc2CdUBnbl;X8YR&hRjVH=vx@Y;ay8?}^)#{l?>uQC-7%>lOwY zMn<1a6sPeUx;q_@qzA6VQ+zq(3jbXW>(&s)2L|ltS|cd$P7DlPYiiTU80Y( zl_?YqEUX6)9)yO40sDi7fdK`A-(mfkxw*Nc<6$5H!|v`b76mV2h~zZTWvZ&GW@al; zv7{s<;`yCkK>~&Vie%RE5)jM8T^uUVXH^yw5}N+z1Kq~et5<;#gHZH1NdxYWdM=M} zDIKG`>-ojTqif~WdNQ4l%NiX0Pi1V-^UB8`tM@8f?>(-;hu6Sg(z(mR@%oNC|^xjrbJ0+sobqp>uwZXel?1wf+mEouSxK zTE-fa<&VqdNfy4~h44_ykb{Uit38QsXF;OH9`cpDq*0t^ijIw73y=9U)ptV*EZo(LX&%F0^a5SY3|)bN5rGD%e*zwZR*ov1GO z2)?90%q;?D58z%M6T;f!BC_Yp7cZ8nd%6<@01{O@?EsZ?2kka3ttjtwJ*1dFXh?EF zei@ikP0i7O02Ig4vN8wg1KivMBmp03C@O9P+>(DmJ_q$Kp4<9X$H~ST<7GT~E^Hg4 z6)!2;j$|DdMrEvZT7LceuBy4u>Z6_knN(C%1idxne@OCHCgs~H5Vn z-Iu6ncqtI5{{Ah{hF?QKnEL7K>q}>~KE?y5v2S~YmX$UC(cff zBz#T$uA7%u^*o2w$ zJs^A_6`|5uLNS(>Mr}HHf~^g`y@-cIbKE~qB7=1KB$0)EDtm^L#by2O2bEfZzn`P7 zUdNWseXx_^htxku^A5@`c2P>+iC`;+dtdmD^Pj)?J0DIkd%A1P4Js@h(k%>Fs40SXq+Py1_t#S51o%_X?J=L*N{- zHFP~mreST-Xuipzl;lH{pJK5k`O4|#r!Rk5wr+Cc-9Hh}r+jtZ-(dG^{U~ykYVr7V z|1f;2;X77;X0H5@hZcsM4Igd}11Sfi!U;zBl?Uru?5W&(+9M>{{k3RpdXiswCh8o~ zradnFcIF#ttDa_A>ZHGN^=HPfI0XnEDYhyEUorMok*V7$t?IRuaJ;6}W=zp#+U z)8Y;94y`YC(cQdtZhK({f7_$Eegcz#kv1*oLd69Brb z9{-ZsgPrbpcjWqwftWR>XUe&C^9gunk+0hCpLl7>e2mP;WytR+Gz)de>v~6WlIFYK zS~W;6L6l5&o9{{DSob3j+qMr*t(b}?`mB;lrt%%`jg7G-Ggzct<6Ew8H$hMJoZ+>&yF+of3x9s@HQ)Nrb4bnk0nk8)2 z*l(Y`mFwiWD5?JyJ{*Y^YoL|ixbfDem(nd5i!&ILtJ0~9o~7%Yh*=^soiVSB27|KQu!NS0?LBsR&=i1xp;pv z`bQO`p6kaod1#O{*cT>vh}?Iypq~By=SQ2z&F+WiLO09Hx-~7;b5?cf=U?zl$mtRxF(yhvlL;N(Z}y`+Mxzr)!f}IA7_n;SNlw{rG8rYh zUZzdR9f6#?pC5lv6VgGYC7q%hUHpE4Z+J{Ol;q@E6&krF&POAS{H~ytpSSPI-RnAe z(V1)(H=X^pb%{7gM-^%YjW38;EpNI4sOcx$cRfwy)W@xcM_YG zR;qNFe3T9S!C2HZZqcO4SZlX{1?^a1VS@1kLanaQ0trv`EN@IBtj0yzZ=Ta+B|W#A zJ7tZ1-V9ioa}1EW{|cAZ9%FeHXirx;z{yaVwr{`q^8woKBufv@Feqc4i9#OV0tv_- zm|NM|eKU4<7sRjl{JFKW6H`;1#j~*)KtKw=vP!WbXogU;S|6RN>wh*eH(y;^TAG`K z0_1h6_W1dsSx+S)W09-%bwVp7ARsW9qizFji=#Bu7EezRTifFDB%N|Az3B#DKub%Y zPwoM002L0pX*V}F=<&9kpvnI`u-w{W!Fus}%yBI%yg0#^U~R3&nlEmsm?D^`0Rykfb$MgfJgC&3`6&k4q@1 zw@Y-_+s<3{bA3Xji)@9%cw4hBM+`T`eX=`Jm`CE$rBvh^FXk4Dj4^7Q`!HS#7&fSC zM$(xvVE<4e^SWbR;}`f$XtLsdnkt2_ps1$a*F?A7x`$#M-)7X`cqmj~YZ__MZWeV= zp48r`f3`ETs<7pKaE|;j&h`}+zEePxf4E(F=GTyRCUb#*JX?kmfy7iE-abgjF?WE`_ov`lfz;^zGmmC8=50o|Wk zT-jvoT5o4+ySdakI0ij%Pt}Ph(26N8&fOhjaJhHPbYtn=-6x@BT67x?~XTnJXltboM;Xdi+OZ=rfnmr>c>(OWTl>sIsBqUIIf$5){Gl)36m7bFF zTtkDjgPWawXLYy;06HKI;j4750M8p68VZYwdY?Nx9|NEXqu?9(@xv=`1H1+RiX{eP z>p^f!f0CFG8;dUDU}e?Q+Y4-2h5Z_-LtzssNK1hJ?4Q_?FN#SJ`btY}@LeE- zd^oFySyyAJGMaAnC6j=K;$#T=dvnYd*VB5TkCor4MsfsRZQj-Ri9D}e?9b95*n)Ag zdu)x4I9w?1baKDHxajHR!il7LKgokB8nKY+O@8cH)SmK@*q3U9aSvE$t1)d|50k=r z+mo3_tkzRU?dacX;p}9yOnBDkZN{L*`pZ6{95hRxc^zxa#Fy~I=k;K_w1=-Gdx`~= zKt0p8?OK=4-cfC%0Tzm znapnKpd7QR#W6x;D`6wPW~`!0EKj_FLz2UnKZ(pxo!*yL71X@8t*7KDw0bnhED((P zG1mA?doEA?Cx6USy7|7ic+V`zkj!!ftXcUs0W5t~IuY3W?y(y*W5RFft5h?^oPKtz zQ#U-?$o!1;u9+9nt)u5j#PadcPiMDtxkJfL8cn;YNt~_EZpXwSH-&sZ=V2m{-*wyH z5~7f8nIVea;4qE&KIcP|UdI|FW^PSaxv+E7=5A1*ub=y(%BnOM}^#3Z1V)!YYQh)*Q29;XK$7iEcwiA%orUt3zF zw6!T;1>2#bzWKnQIxsMRcEfvWZ*LDg4=BunH@`=+_}9}+3t7i-|HB_w(<5ljV6&6; z99#>~8fQBA$e0(dvfQKdBo_6+k~?ucHShMQs*EYlu7*Wm)0_Ar-N5jXbDf$E3qI@G z*-r{$LcNJ>yW;uljPv!Wey#{*_NC#Zyi;O)G;M%NG0Oz#m7ZlQYEjMGK2E45FcdHv zGVXbV_XQV9cNx*K7SHnsBHJ=T$RzJQaoeR222{vWGC8b1wogT!`GviziDzt1uAc7D z!)b4tT~MSEB;%Y@x*g0f!Nr&`lE(GCr$ydIDIV8^C@8E(cJ(TaR)2j>ony-=mQ#C* zRFN`rs)ETb&KvAVm)6Eo1@j+6+LAGi9}G?^G(!EgQyo}!cTLh~gdY}_)Gq=2GHdDl zKzkro@adh6&k>o`!8I}tzeldc*2#)zOa~bf?piVhLI$dKfrw3949)fh+2q%e6}7EQ za{&reY}H_L`en-f+AtF;;MPl68qAcl(S$m$CCN0GB&Z^4bDY!~U)h`N$1Njr-=S6{ z0Z$71L^*0~)riM^yL^I<-R?Qndw=`XO^#vrRx7`T*QSMTWsC`-gxQt;XJw2B@A%sl zKb8&{x*z1se&BsOdCSw!=UYn)6}Oe{XsLyj60prt>d&9wk|ja_88ZfI-rlkb4NWTa zvq00nK_n%`#l>ZN-W1T<`p{Ltb_qwEpPv3vISQsHC5K55bcr7s)x8337Q5{1?d7DU zMY3TWpKx(@7OkDWldmU|w70wa01Q}1JfKLudi4rtDJqH>W%p=j@!MU; zD?ZDUbso>2#hLK^aZ*|43ybxgF0?OYV(vvV6gl$*!%9r$qHk5zOFK|L z{Cpm)=lnt}xWtdeHlez~EbvS6r_ycRyF1p}rCLSYvc*!2_7OH5@4WU#h=U6_$*-Z> zC$PV46ul#kr{i^sAHv~hMu#arj)`8me=8|sfz^ad);8L|`opq1$aELsEY4L@OgAaghs~qzw|Kph{X(lcNXnX@ zu=UuWp>;f^${IZ*OIG40T0W*iDHE^?+|HKyNzpyS5c-&4P`e~U9B2v5*=nG;pr=(* z!g~rVesb5-r%%C!0CJ9(@Z!RPXdkmqEx*}7&bJ^U%aLMfoTayKZ$K*!B@ko>3&3Sy zumJf0)bR!bLzLp`>eIE+GT?K=moFV*l*_?{xFm*<#(G~WC@45P^X9mX6r0k0OiaY~ z)ULF>R~jjJmz7n%9061Cc!|c9AkiE%wg7||y`6sw+zn5`T7AMe;!-q+XX`v^W?NrX zowk_NbQt#u7*g`A+NmBnpW+~*Km8b(a^|Q$!^}dttkdsdqCq)RHRp?Ph(B_sch*wx zdq3#g$F%iV->cZw=XfiO6S}ieRzyjXPs6rdlc^OxBzB!1wW1LCez+qPz`!3rf1+byLBx5o0eZb^tveAwgR%=WWz(YY#z=__ zj)~(AL?`Xzn7P%};9>=e2iG!M2we$0G{As(X@`Iq8yvG`s9Lu@h@pw(GR=JKfahVr ziF6D-vv@g(3tO`-!1`PUt>O2Nu%SSbV$fpxGTpm33^Eu9Y(5R(n6WYei3Wuk(HZb_ zzJ+zNI7gpWl|9q`@zD|W`*P%<_OE8ZX`#=L=j&b5cvfVDxhp_JbB#AZD}aiOj6qO( z_D7lwi1-i19MO<{$cC9UOEECv=$wH)Lt0uoCBq9Pj@J$?i-|wxH{t6O&Lx{vQ9X~2 zbZkjza`ho~2SFw1=Fgr!-9OyYr@nuhlb(Kkf{`A)T%OPK)mAHA4%UPG;1D`roVy`1 zvat9|UOe;r46Y!C;(%^2V4>adJrrno4%Ce=9p-ye6B9ZnCJ8M)`qrQ?x->r=6()bX zDlgfjOT%QFjMqO(DlkEpjpje=){*B`D>MFm@La?td61_cewca%~HSEBYJ$lqbNDmU!$nY?Ano9Q5X*2MK*{bs| zLU#@TAXlfZ7S3F9@-(oDt*%;Jt?b~2<3Z_6zO8|Zf})nM>)BHU@eJKLnu~%tSiDSD z05ppd0Y6K~G*X~16-M|U)~@Q|qeot?qH5DgKsri4do(9Z5||Ci5+ z;&Lh~jyj^*!MZX8_F4>=qwRTMMsU_4FP6fZ_X30aIG&fl2pqPG003|1TVP{j6BmUO zs8zg5%gtqOBF$o|_BY&|s)w_(w7Tj#(g~b9FrmqNCPX?evWz07;4bmuLbs8Vw{Gx3 z%{Sl(pOMLf^rn+FhHG&2N&EVi*A88s7TO4U4W8>73k*9=0)+F-7hH%?1a)9Ki!P1R?YXygVS z-p|(W&w=s2I9(L_&C>6Ff3|*?NA-4qA<84uv|djuiZ$nzG_|Pa=<*MsI!!IIV7eBFhOR-mDQ!Z)AWv;N6&>I}FGeQMxTuk@M#7Xk^$ zjte}?s|vW%a1@>oPSm)#_=ozBBfq0ZX6yOg%ev2Xuv%YaH98?P_~H_CBn4H z6T#St7Lr0E(2PtG&8FY%aW?3o0EGtMgc*N{OD9R;`o)}>lIX}9nn3x9C zo9_?bfB4|bC9DpXxR(GKW|~k@3X+rcL`1G;p(9{^Z^{J+6v&z8!-YsMok3`!M8GU~ z{ezlxl`uK!OPyv?cRU~R7a}9A)UQzdbibOg@$vC-aS<)EThEA_o4$Nm>u~`B!+DN+ z(Lk;idT)i(4(E)mC#>az87 zcPPk)5~CJ}a8Xx){+6^J(U0?r&@1Hn0u6xTeVncU-3*N%Q;7_;=`EQ^G-CK2nq7C+QC?x?`2x?!=x5Int#SK;IHTnS# zSe>k++VW+v0PBPRSe!mxEyZkuQH&BR8$Iw^H;0hnBFepaEpKBE1y?E@+5r%M9bbrx zBifQ%>lT22#-H`gE5G+S8YK{2%Xg{ns;a1fEhQ{R_p6nI;EG#Sx3~}cr1i{`E zR$L?(VB8{zZOM=HDa3pTBY+U(6t{tp-aG{8sgFd6wI-?N`c2xaCvs3OiOu=Ggz`A^-BJ08bXP0AK}ltntH&Yp`{J6f8zP11?fhMP zo;azeWk|(`s!AID`Qa^`_O>O$f?ue{Rgti(XNL;?0S8f zCZ2CQi(3qq;SpH;D%pvNo}sxE`j+&lqNF}bMG$RekpYMkq6N#o@mG^oMhn0i@X~vy zNqJ238lRF1N#K_|jA!2emd33&Q6ru)U~W0*z3sd#rm)(sz;13?zD6CqJds2K%$P@w z{X(NG%bZgD!v4*+iYjygf;!F?vRW)_squFF!S*>SLSXfh3IqSAUNqY26xa-p9VLfC6)U0J4_Qr=Bm!n$vOLZ?RaEwq~D7m(G zsBciY6;0+Zq{n`b+tQbE{pdV$|G^l_ixqP=EX~u!$L()Qfc)bo$yqVFvt~le>C)VT ziTvuV4cA#mNzRMW;@+T$z*7WPz4XWXE zCH#$WGUKpTs_FwF>g+zCHKzl-@KR^qAx!>m;yaAx$cFL`Tr$LrQeErkEA&=4tIhB7 zMul=l@tDtwre)!)DwA{0;Tf!K+|>WX(=p*_E*phg4Dvx#^{#3pR?HZ;9`0QBlpd3X zC1b@`)9wCLqaHVfN!+X2l$G`i5Xn`lS@qjmOliJ`5i4`R7SwhgBwq@^XBy{&pVH=c zfBgg}TuNq#2O$I=AlDk(Uq9*yqCrEXp0mI)L1DA?bYDs zCQ8;H3Y3TirF~>+$sGB>#frL8#~^c=GTDDCfxwiA57YhywA(HbCzFqD=c~ z@E6_B@psPY?)5I7C0`SKRYH!v)gL2h)v&};y+*sxG-8K;{6Wixqo4Askerb5-a9et z`GDw-A#Ib&-7Nh`DI+K9DMhX$L&|MnYjb!dX1cf@)O)_TcH*Y1&w4$VYKhZOI1&4l z+(1#TlZ0#8Mqn>l`tE`*ksNj~hrigC0(!sM&vA*zz{kapv!B3mKPBhJEVuf)6w4~6 z_a=;0#F#Je6fpLZDk;iKFzs4aw{4W|01$xj7LNlGtGxHZmcxa29z588LPfHTj!!_K zH^c+x%I#)!+n*Fq?0m1XXJGpDt?T4nedmYV=;v@f%e}_l2q^6?W%BZNsrnXequjP3 z&@m~#t8k`cZWbgGFY(O73c-G_$`ywI5%BQEq2yz6JUg@zWX-d=R1apWw;^MhEBB4Z zA`{ONQ!DT>PNZu$ir&8VL1Ay3PW+XHSrNIk`;NWn;7jyTf8<8)zJf{l+SPk4s)x?6 zQyCE+x$f15ZclX(YiiQ8U3Hs~D#d=Q<)!)=Gvd(7ckn$th$Q9_qN!LYBnWDTezVNj zXRl{(BmXRzjxCPaa7252Q|0#Q%{HVL=X8UMn}im0hc_H}^oH|=e?5KAshD=zp_tw4 z5%b3)pv>=_oKM5W!((^coH{&o6!i7R*)~GA-Ol6*yzi!02^uXY*LZOT_S>Z$L5|($ zT24tDU9cR?BRHB=54v-mi=>VHTd?)o_n{lu}`8m#;KZm?&QQxJ2>K(A^@w+=ZsFD zU++%TmqIJ;J37WSNGHfCUPKWY=W)7)2hT+=d4ZCFVk=o3ViIUCQTTlzjUG~N|ECH& z{G&zf;wQX%5yVH0Z|w)Ncsut@aThO_XysmywzhUd5b;q}HOD5=;;&!P@7^_{UqGeM zOxn=4;m5nr$2E{!leCh>+Jvj^V${6~e0HY;|JL3d^LPN$2uLqQEJK1F+mE=PAMcGs zhfl?+RYkEj>W|vxzcO!OG;h(~(%gk&HVI^;TK)b$LLG2~VH(Qg60T6Lq;;fij}(6X7Ey6sXLZz111XA zijoeQ)U{PZW8$%*nfTXf&f@km$DO50e~GX6J?7INj+s4Mb;1MkbcRMYkDLupd)zB= z)-)pVaGQQi{#-24__b?kban;c2LZDFV zIuCxW+DjRrL|9bL9$8Facc%(V6EDAG9^JRxU=H5P?Os? zK|iIOcf1(8t=}gq?O0^;Ubh(A!-V%rC-s_Tl~5HW!4C?OmH6&Nylw6ce>;51@CDiK z+!`Uu%?t@GZRP}ODt(wau?)Sm5<<&89i;q^nhPG9-4*JdC);k7%mf$1mP76ueRfD1 zQM}uc3oIPBJII{guQ0i*#6ovYYndaQ2=*5yrjU2mI<(l7`~ktiIaygVlN8i*DtJrn z_>uA7RY2Lqe)+>Ws(&x0W+Zl9q&AS*ra5uq%U#*Uv!ah(H!fhqRzvPMe3Abi7VGjn zWBs`*k0y<@<;k0RhCW@p2Jz{!pncHvDuxUTV?PvMR7)qXaI8;Sd3`~%Nmo6R#k6`~ z^J-2CTi0DJZ6X50`<9D{_^Wje|TyuVO#b}=K0#7mLHbPE@%=moD6+0*OXzg zekW9>TF2IyTGPYD+h>(lh9Ie*S#=7<$0OhSw-f*PfE8FApc78#WXRie#DQsrK}28( zoHl5|1#-h==*1ZItTm0$#?E{d)QXRjR_=;ZWOLc9u2U@=btgg~NRzSiZ%kT+qO!xe zjR%uR+DfyaN6yzT1qCba^+EgK#9*f65G>VyDp}n5A=L=<%B;Hi3GFFOXS<0^EoXl& zGkupqj9GrF_k+r$+}*Z}{Ej=%=i6qTT}I?ym*?#!ESu0G#pcwlxkI!KFN*;?%VZNd zS>@~p2&@0z4d72DP_d@TIQ9BzkUhKti5f%d2wZNHi&NPL6+1Ah3KTv@UcSd~Tk$LO z?ueIHV=LY*+1Pd48ufhsUY-|;=e3;d*cJni1T5}V-^9s^AvVP=)uvfbvbLxs3l!OE z2|FflEMzfhhG2s7ufwgY95Gku&>38EHIS!ZA6L`vGKKjLD!*@WYE2WPT|)QbnA|$8 zFzv~9#c#&gaF&`~|M23wz#PHdz5YZtd`ayO%azv2H?Fsg=yH>_z6i$C9rple$2$6 z+AS!2<}qwtqgN?a_6z&~mv#f3H(^Q1V3ipq4vpTRt7Rc*MSAd!8g6w36Q%KDP3uP zq;R(I8GJYf1;nt~z2d9WHmF;*!I^Ru|X^4)M}~CFxWmR#AT+vwgP+9GUM}8lP7~_Q`HFc9`O#C!L@ZyVl_?UR-P7|rOt_5qeLoNnyO6q;?9-BvNkINEW-&5{7tZsIv z)IAXPoN+|{G;$K;u%Gj^rL7TOA-0>;=~q6gK_zcL!Bgm#q_;je^_#v_XS26D@a-x* zt8?z2P`mN?BEppZpEwGK88HQg`fF2$+ndv37%paJ4F2&r+mBFf8!fQ)JEw8R?5_T# z4O_w7!8^sipOx`o?$~WOE4hu(wh`n1w}8o$^%PrTyiHlN&mBodK<6bi%M!M7SDZOn!-t7MVz7yHfYGh(f>KFdjLG9~VnCSlXkV5_4nQRl{ zTzawGeA@Nj_jK0QOf>e0{LGL&Y`5z=eA%*gWOI|MVoRsK4xPDE!hv$pId5NK>FkK_ zGM&2D-nf;68THg-R4i%*VRYRc|Ba9z!>h#{9_Liq5~-Yhl~!)c9puU8zoZllzI)UO z6E*V*tLcTPhQ)-CZ?^@AP`RkdB#wOX4-o{>rKJ(sjgh@{(C-MyR1be?R4Li+wV(cM zjUvz%Lb*cjh($ztHpPMZ);NoO=FH)u4ivoZ^?s`RQ#2u_*UA{5?ia=f1h!=?2*=~@ z4~43SkR&YbciBg?GrPVsdj9F#8(pnC!?mgYt_Y}rlsVmA)q%kblyl#_(T{YRos=~l zVzpp&n~*rsR2bNF44}0YV(F?qQFj!wH7Lj)^WQkb{=~DMv{jl$?|tmH#HD(@lE>7X zMU7Jq_4=-p$HNQkFpES2m#-$XqIZaea)`*L+kLlez*!LOz7=eBIY#m3P56eQXFt5kWr3Tkur zhZ`l2S1H{-5jphSUi6B8Xuq|Bm?yt-(SKrNE2Sl9@aibPO~XS1w91eYzf>LL{SHcK zgNoE?*lgMKv@CJ`52z+m;_aL1brTo^>l4aYltem~IR=4hBHlz2r{X*!J4a8D`>bgA9vmRDij-Z{Y-%lQOqJa3FleZnH29PdSmh~c9_d_9np zwMwV~AnVrPY8J2j828PtUPE%n8CK5{(=9{Sq8%j--YSUnbfz3mzm`_YNp52hyRYcL zV=*rRr71Zff6ce+-P+^Zd&U@;S{a@%6UNev&2FRG*+V{Sdd$2-HVP=Y(@sh>fI;zD z&!dAIJuz#(e&I}~j*~IbATdw3=P&0Cb`?i){^)#h%3oAKMo5=@LQJ*x%Uf=3nKX^+ zrF6&6aKSSPiCQ7$AyD@C$vQ36`33fh@3iZl9^@oU8?be?xsLHG-P06nGhE;1Ug56M z#Qs7jx9sdj6Vv?bkx?*{SGPR*Ydze$khj0JjEj=nFziD_B;>Lm4Bw)Wnw(4x^RNX4 z5grac;i`|4U={9nje_gY9{*^Z*YX{i@dfuFa-qv2Su6l26cn2pw+UXT3Xa;pR9c`^ zeM7}6OUa>*9~uRFj_)8o#e6+LEA>uFPVpJB@_YQ{H9gpg0l|S~UFtM9=@YP`1wKzA zgAqm8RnHy#Q#d50A+6yJexsD4=MR(U^e8(htA2E=Ps-Td(0=)9k!Ss$d1 zLf}lG!$=9Lf{abl``{?DGFOpCcbKyU`icmHV;%--PpCukFU+M@hKW{tY#}593VR~J zr;yuX7FQ^4(8?BEx}QF}w(|g|bKlFOGxDpmT#nWN65pK8D?9B2Zre01kI`$%Z>ywG zRd6sLV4O%jdh%`Jo;M7tSHA8pF8cT|b>-#v_3YM$ILx{5vHp6VCC2MGT8`m!an8** zG~Yf^Paz~kIm`XZcSwV(IN{@uIFlz92~W(+x4MK&z+U0vZA3oyNyYIN_H=mB~fX>M@om@H-*k@M?`#d$0IxUT)qjQozywzO>}eZ=5;lE znEr`#AXPCTlzSq^;y$h7E5OT~bxT9HU*7n0#i3@?@0i`y0UP}(*pBY?u)Kpt63#|b zlel1W=w;d4rF!ObpfQ+M1*CajdX>m8cWVF8TSD)1YJTmX0=~r(TZLU=o*5ovz@I3u614*qxxQznUe#O76A)3I%oqmAUYp~nVOo`TVIaQz(|R; zQVvLF7x#dW_CWwg3Wx@Etb!;fb8UD&ZZNYVsGxXyrI|NIG9f@6W@7AmeYHP@DC1W9 zSEQKwyf85l)>>_Fh5>Fzssfk|YynHDJ_XHdZC$mVDk45CcPD2*R$mb1-drAdF6pF; z<}{Z_l7qGZyG2iZBPo_Go%or~2fxkwt7}VaVf|XnfvnPAV{*azQ>HvrJ9Zrm!(o9= zLs9d4+}(HCe$31<*MM=f%y{?DUh8}V9;~{i2By0fyA$hx$lhBSg!vIR@a)SAm)q&; zCZ(q83k!dBWdoGnJ2Bx71V2nUrUWk$qobm#ms?Szb;NRY4-E|s4ic>l!N38!vF}?L zxw};E&Yx%#NqIuY&!kmp+xGU(L;d;o4{9I~-)P3eT$`+$tRq{!@zj1}0`soq<+P6I zIe0Z5%gW-bb3Vl)=YB0)`f?nN#or0O%7BKpL+w2~Q*7Ahi03a67D%Q+>HK*w=lE^dvf5$54)E9-P!&$cGZ0V-#l3LFI5ut&FV^ zk2Kz@sjpk4S=`Jgq@LI&9yjLEweg%!dL6$M1_bx1Z!j=1OY-y8V1Nw1KgHA(?0B>o+hC!)_x!6N zbp*rH9M=2yMM+=~QGpOmINl$V%pShd0KU2c<`}_n{C*OK0*g(5!f>K)0gE=Nlns1k z1F-s{1#qs(D?x7tON`h`{;8ke_0QFnmB%1{!0f!^&cb69=7CEww2sx1_#&8uU7lj` z8c=kik4Q*JhVC1*;;8XqRDJ%8qE77(lNr-oq6ILS`6P-p83ZDr(V@+T@i&-qgn_+q za}1PYFQGIT=%49B@b=ELGMk8$zFf_?wEN%so2@z)6HhIdctd3TWkW3z)3`G?z&;DW z1{_bkHY37};N#T6;1Vn{-hb77@B7$1dtE_WaDlJ=C8&1D?f6|u z0RMGsu6B(^l*pFbK~69bb-TN^<(vkjF3EddRp>ix%PT7&E(I%Fj#v67fK(@z3G;`b z)Pn4ysi6^`3}3}lIyy8YoCGf047vEr$vdzYOcSsZ-*R(115XlY$PJ%BaIAJa*>8Jy zceYy8pv{ET?NWBLtk}drB1+HM+z5q{m-EzjuVt>?pkXp-{y&Vp1z449*Dbt7kPhjT zPC=wW=|)Nf6hx%EL_$D9kS-~aZUGUbOG;XlZjg}f25I=`!tZ^*_nh;cbN$z~_rBOH zV6EqQ?t9*2jydL-o{t_?Sv~m8lRAIriF?F60$C_{ zb6THHIpx6lASW~hXV}vH4wd9lIf?9^$b#|FIJfafYT1RY`D&nU`|Irly_OlDK0!%o zQE#~Z77_=d9R`$|f%KsSNH+TVVD97H+*9qcW9Z?sjgN~ zRBUc-lJ&Am?;{<$C)@AJsg9yaOu$h=GCJ>y(x_7M@_g0&1=P zkf0zgUf#0uaw25%Nr?sso6XJ52BXj*?X83`UFgqJfOY5fEDdaDzNDu|CM5hoG@`Tg zdc9a-*~io=+!-R-Vt4pZSiJ&`4<4trVPBj6f&KU87Efs}m5|nK^Z6PtIgc=uI~wEY zU#*(z6@9K*cl)a51)o}4SwZL%gZ}SXTm7Vqiwh_ceC$C)eLK6$4J7Q0%*-66cy18b!eecG z29+I51E*K7NE*a(%#!aK_A#KWVGrf}5HiG_QXftlH;u(Xs%TKk2%!4yxe`}YXi z7xpt4yk}7O8)<5qh4OY2);csPgUlfk(XIO7{rko_I5^a9q~0|;i~s)P#|0}=81jov z`{P{`1TzZjci+EczC`bN56PoH|V&RiE0Ezt~cxKf8>>~r2E zAz{Ab(?HX^y$078{ZA+ zm-m?r#&mRWDE)KUGvzAx#D4zxB5AT=YOJ!x&WyR>)vjQe2!GoRtQ*lhHjbE~9x@W~f^>fYFud7dn zCQ~gBHhwiUKywB3`Rxl7DL}pcp;q)X0q_1-DD{FuL5sbbO4L2hM_Wsafq@|+2o5d? z5WTiP{I406f%1Z|r@Q-0S{e;AGpe_!>n=m9sPpcg?V zF^SmsTzetSOI1`kivow)ISe$A3KOR@#Z5Sr>HqSz^1k=rmz6(0s7` zB6eMOq~<4cLn2ppE4_e2ou3pEhj*;FmmbvrpA5Yb1m(g&M+Z8plY;}#CU#Cv4%8G( z;2$^%em%z=q(qhRS?Wrxblo!@co|Fj>bdT-XT57+8p7S!1Py@(RrwYTt*xzrTTrFm zemWrO>;_%r((%H@^9Wypa#2xH=xA%3G>0ZRR8-X9vQmH0szjtnmJlB@N`b=(R!R&6 zPeTWKhuWl=xO>h&E81Q!gEas`RY{rI3Vn~zeBQh@cC$vS!Lx41w@i)ghpMi8VGg%1 z9{#d_QZCb%q)mZ=xDq6e$i;CnIr*`gg!-6W zOl?nUS~0s4yZ&A}ui@sqCG0o&=ICX}CoU`O)jb6Vujn3$M2x8M1r zcbPz;r}J7UDC{B@Kte=>KB)uL%xiKl85E%LURGyG4V8f?%yV}hy}Fo~N}7209U}!tt1x3rMcC;Ad*OMkNdH3;qXM@o zJV~UGhf|km5b6v&qzAC0M_Tn#ZA!X;1eBry>wkF}&i%!eXo50YE zZ{Ej%;?@#)6AH#BL+||OA&x@(rrVSh8m;#*13c&uCe~iC?zj{(`4MC1g(>$oG9^XD z?yQ|&s)29nVCR)OxgDAXLlbDE&bBTuwt8NZrQ*Rlz=CGo*G{naHd%+4Aj`Y_Q zFXp3F9~u~g)87lYOvPCCMz=nbXVHu!g?e_w`C}k^0w{y_J5A!Pq@*M?0)Muoxn_C> zXnaxI1&%y8z~;spF3-1Kt&bYi-k}#2^&HGq15sGs`6FRk-w1K5DDl6Kl&Ql91yl5V zUwy7p1`DD2M3@D!N`PQ=Z?OvugJR#mKlrS7ZV3lqj;BptQcJ2 z$W%r`^udc%T8~B5vHX@qLoZ`e{=2B}8`{`-fa%sk*VeslNm&h~oV z;CZg|EMqWUt2%&R0_mUv^{rc;;N1aY zyos+)A!1%|Ga&NP3_}42*&UV@Bo<)f<5~T_&Si(r)C+Jb#J7L1{A}i4gm!;fv4M!3 znAl%@fVSAo`T|r!&{HJ{zycgGa47|gK*RyT-Jr+c9#+^)yL@SK@+25h0Z5h)dJp|B zDEWvU-u*yA)s`Xwee<#&5YmP)LmJP+s=HCo;4Vv`)+ z7U0*h9MRK$#p<%|g01V#$BQm%9|o&-)LSw>3-eccyIbJ#gbP*Wd54#${{$O| z?99a*JyoJU=nH=@GAo?(UZ(rCi^puN$s~M$63S`I@$c_CyArl^rI4LE zBp!d#zI^p+d|&<`S&e9KB;wI->ws9i;V{H&-XEsqx_5WX)Z)rN-J2FXd3J-=iTvYP zU+u141tZgDdna0X>oT!)Vg`5eG*4Xm_wsYS6bDjYo%{JK2d;+#b={%dgsJu_t%v*; zh|PpFInT<%^8NdFmzzCM6PX8n<4PpBrXrtgO@9iDZ*U*Z*{Z!5kQorIF5AL1Yj12m zsozT+xiAWn(YXvUG7GinOCKsMV6M1VRJ}E(`*yf{WAJQJuZXSVSJV6lu1$4YYU*NF zwGXeRj#hFwygaO}yp#sl)^pDjMVl+0=)BVZ9I-}F^o^(HM*Z<=D%Q$5GokZ)r+~fi zw>(IufH$lM&b-9d&Q9Kbo};%_7Oivmo$Z%wL-NZ0zG*B8cTL56|H-7X*G@eYr~xxEnT>s!Hk~}5!CSOU$ZJyr`OQ-hmC!fQ;TNP`S{Uzsrh$b_ibzQ#`Mj^6X zSD&n>$3HhP_e*rT^vhutZC5^18Ah$}=H{nJm%TWNt@ECq%dLJg8!P0KwR7${H)EHO z!^@o0oQ9lvtNiL}h~%&&_DI zw`Xc)VzKhDCoFxQrF=5#OGQY{b1^~Q$*r2{=-8{^K8*q{SZ&)_`|pCmlLS28W8?Nuc%g2Irrq!0tr$W)97t3{e zcM`{Ng32hT@%bvrr!6nrL_KP1`##T6_dOWguB-Iqw|i-II7iLFJ8jWKx7*xF!jR0VPKCzGEkPK_pm$k z6`O(CI=u?pi-se^q$Q10MqRI+QZF5gv8k$R9ZJ!q+Bto8viN2~D>0P_67JV4oz4wM zD-FpLTxl%UIDe|`q*4ver;@Fv$nDP-dM>sweyW~xhHv~@I2iFdjJ~aht+1aUT04E; z%X81Hz0xyWwPEkXD|JHjo2|l#*x6#kjRO}eKa*2ddeNlq1ys@Q%%5~io(>c)p20m> zwFG0WZk2b|WW6Jw7&Tcb+paMkpZeu%$H?D&=Z)`ojID&Znwr9GJbx`jPrId3>7F(&V_j;icZz1sz8*OUcTO#7 z$=K-e-1QortDbzzMzx52C**&JFjh?YB>#HNbg&#u+a4BfkIK6yFzj>bSLqQ!LPtuRZHm`O#3A zlvK~=t-}@x^X+vjRDWCDVUIAU029?MRWF~sDM3N@ z+r1^+i+0Wdwi&PPKlvlSzC9N_!1!7xlDnu3Aa}tA; zU$Ob3rQWNoMrdfdJTIqf{SQHa$6X^OObk42#YfF{Dw2U@Mny@=qkkm3_9(2mJqG2> z&oy&6pFFM5yL&)@HCnxBVDh+&$4=@bV1zmQxLf+rCV!m1i@mzc4(*!bvw;wMAK1dJ zGhQCF=zaP#E6LZ#eCzOT=e4mI{@UIAY3ljaSw(Kv3w94f)dM}c7>H_sy8=JZ^;-L> z%T~ve_;qn5<2q`YJ`bU)y25TI*Kx>^EBIMnocn=ELdt_xfw#VA5o6Lc z#}`;^J_};i_rnUW zib&gZvMiN?-l;D2dLS~Pe-TggvPmOW5PHJRSiyG|F%4{{^;1o>kOyNz>6KG1jc+AD z1CchP?@up5w+?u~jWkzbD?r6wr80Ck<6Yzt^r1nt1xo{ znv4y4dU~FW`u)c{aex^oQh~zNCb@cCi1>Du-=pt`%z$e+G@-gtNlQgVMoNrA#^5yZ z-vq!u@>gh=0blYF9YJ!J-hSxU;pF7>2zvORi4gRpy%!g#73Z;ygj@07+`$PnO(B{O zeKTo0Iy)KGp-1B0=6~LU|K}&8<>uz*+p)~btY^*r;j>e_g zH^{Qei~X~*vhL~rgvP(`!>z3!FZ{uXzHJ;N^iMQ2&bk{*>~+;b5}TR(du2tN`4`w{zJC4sG32s^o)wT`w)q3&;KViy-fx5lpzA-Fo7>v5 zTOS-7tL}cvRsx#i9?{d#AFGPKl7ZAoP5A-#^;QHPBss|Jir9l$D_crIo{ENB9G!~Q zdYam7u(lM_F4c(HwTjI9=uW=zptyl6UX_K%03UKfxy#ZX{I`eDTKj{yMxssKmpT+!LKZH_oUur zy>fDzZ18H>abN%+i^I*y`@s9i%YVnclWZ*S4dio;%`g4{a%eV;NJ+7E^b{o^ASirR zUGi${xn+C$%=-GX*ESj&gTR1+86=ec&fab9LsKZQ5oZ@-_vC$nwsi%%2iT1j;O>BU zL1b*Btf0FL4Xt%tfjXtmvuBi^W7*2UhYhDyFPrM?(`1YRw*fs<6b3OdeGULiw=R+)L9kA#nXYD!F2d#jaQ6IN@ck4Ahvlwa@h zG48;A0mgg0Kt}+axlhEr?_51v1V8x!7sqUrIk*~zOMrbm(7!`}Vbr zV^EAukQ9UIvd!ez1L&}PQ3G9F))Q5c1>hiS-wQ5l6k=&2K=?sX4t%$PM*(`u89Z?2 z-;5i6%$H2l_5|Y^F|meMQ?96T0-iULRDnT8bWKtDiP+B~KX`gcQf>7%hW-;2Tc(SR zP<1w@>L_Ttf!@Z&-5UHf0N$UuC@9@MJ!X2)J_;?7-@CiB)nvoSWj1p8l#TK~y0~y4 zb?Lv+JaPoHT`=Yjt5ZJ*ZV2K~h>w~id;GPxGv`bZ>$qZP5t`;F@G+sV$%>CckJJ)2 z&d9)U*O#Wr*lz#|cv?C->VD(#c)Vv9U>65m7tMDk=fArEajRs#KYSPgB|oATcrEq* zD;y7LM!5fXbz>^w?F3Dflsa|n>ilq^krVd*Q|J!_dfo6VSg}KwkrPmAlM@rY_R->c z_3qWM4@fH-8pJ9@^uPTRTS{vY`m1cTS|KfL=^l(Hd11xv2xjHnCO_aT3#Dl8VLyh( zuU@xMY6I}b1zXM9DAEntM<0N0ucga(c6Qd{EZ5J0hGe^jbAB&`Th;ETbA17e?_d=K zt4SFt?E53@vlR05@s{B8jLGsXY6_%{*;$`Yr5jL&uCvstdNH$<>J^@!3jXN8tg(oi z&^$!D7b$&cH4-?f_GWq6{1IG(Ceqcpf~Z6Y$lm=4*1)Q%qEwSDI6DnxWxUPW+81MH zsk8&^u5)1m(3EA1Zw{WIhtpou;FYVqJpv7|H0^E47SZBRNG19#i@M?jMH?{lY@;$) zXU^y7_4-teehv-M^DklB=kf$;18=`ws{$Lo0(LNz;;3azN1#C1wug@_Oiw=tkMh;# zgoKouJCvXviRRB7jOew$vl5%`WP+2o?fR{@sC!6VW~Qc0ZA!``1v+L123$7dW#IB) zhOrMlSUP3q_P{>QQA9&!OJWS#wLU3AGv6ngVpwG~mp~^D-!#9Dj);_h!$dAh!mNQ4 zSQ-O=iI(y9Z8NkXKGW4LZnnE=e$o#{^1&>nLm-KP4o^SF+5FLTg*3R-A?V_RD8b0Y zK?zrtLa_J7oe8V*Z&Q| z3$=OJXLAGhUu`l7Jd165i1Npe82UnC!9q_UWop}8dlbEe zHV)irF_H84U6^eUp$}UZ*ku^>Li-4uWY@qL%dF=XFId-8a=|If`RfKMYBZ0mmHC?3 z=#$7N7*Z*)UTT0$MtXn)o{SKuO&jKGA)*DBfdc?AaC59?7>Md6l;XjTEz)%It5XbW ze0FCj>Zmc$ov<*CiEODzNaPx!)eL<7<=+YtE&pqUa2cLqi(C_Q+(|k5 zjA2u=Gn0cK=~>WiJ>dT#l6Mvf-LkfRB)1l6#I2Bz8l}55WLhT^KB^%x&B=92j{-bPbbSTO{n#E z^z}u)wMTei@>>H}+PlMo0Mmd)#O`@}rqfhG4G}8to(F*;AyzL775Hz>Zyy|BV`1^8 zFsl*#*IoS|pb72&3(Q=>6~u}KSi{&0?*?B1+$P31M|=kF*z14&xc~8i{{2S( z!zKJ*KJLFh(AQ#2-c%@X)YFp(^V^J{9rt+s&9Wq+22>CI*Kkb1MyZzPr2$a`9}f?h zRS(Qu-JZS{Y@n;C$@44_;?>suJo z19dy7d#$pos;EqrTRnjRLQdF#dBL#)6jrTyUfWS0Vq>9zKM&CWEbQTIfWl^HcQ=3C zIe>6>F0M$)uVA0*;_}cR22ZJnX(`fzSKcT2=rvyHau}b2`tI-Z!#3*#D}*>(X#H*L z=ma4EG3WC$n0K^(Jlh(=Fbb&}z));5zJ4&Unn_^7g?d#Ax5)!&MBiZ+m3vIO0m0hq zux47f`lX)@*DfMtK`aAJmtbn7CRnOzXb@I=1l*rkPp*5o4$ioLT%cSIv8gBoF8h|K z`)GH!Equqg`}ywP-aAU666202xa-mSa5%wxK}}LJlay>#{|gLtiRkp|GR7(#4ThOUlc+&G&mu{G!cD>cu*cj!@`=V z_c-nGy4ce$HKpwZ5G$FiP-{E+6<74=A6=L3<-Mv{p^1QUN@?A(s zA{_6q*p!gMtTNp6JdYW_2C;6Al>&YgwGMxrfK2WIvj%Iyg9A1M!pYlmh)pVOM~rf0 z)D9GZSl<4y#-$^WtN7SDJS%N}`45Zrq({nAI>K$xV(oSVl~>t)X6Bsy{Lo`CI!Nmy z465U~3mJWrT;Go#vCCRm)ev>dOFxzXHfhtHhcg?}hg%|U&|BYuXU;}}kKdFUN*st! ztcXWy1g;wiFZDmeXNeZ7VPZ^|8W;0_i3Hv{Bf(ynf9iW;M*7)X&fx$fP*s zxTr>;R>JE+UhTWviXqOG#}^ zaU=i5=J-1^{jzeD@fYSzO;t!WpaYYvw4EF-)a$`l#~CyPkt|>|-a$(G%k5@dh0d2S ztO30I?{=nbCf{?)_=mmr)z+xkHeNnoUiscUZ`0MjqX z^WV&t9p%C4`W;{m5c56ALxh7?U!BJ0T!m8P6KbSa2y^xnRPiHNxo>x8vq6jCE~=Vl z->wkNbbd0@Ad3Mi(Lkz_Br>c~hF-v7S!I=#oUE)81jQm!X<>&hG&Q7_K+~u{dK70y zJN|$iq(CtwoG`9x8FaqjDUtf*rN(4zq*{v;A+M6D`W(?ak z)8{xn*(I<^y|(HOyHi9GV`Cpby`Bjtr*k(WA$2U4 zYa${f)KI(rI{lu%I5a}S=#?P%+1Xj})5?UKhgSzPDvB@)O>)pSYqshsh=>+eRywxl z;J)~0;*#^41H3kFx<$rwwxa4~h!~ZGi5UVDm>$u`spiJ8Cs1$(@Hq z5Lm%)H7eyy+gw54(V8yQ4VxVrRA$j5(ci4S**QyFBjctAWda^f@hk z{`k)?yDf;1$DlR+6E2<^1X0G{|AM}i3E^w!XDszVg{-#=oOMdT8RY)UHfg}kcR`v7 zQR{aN!ZHzdE4K;8>*)#`3P}Xk^+(l6DmO4N0L}nLV1C2^>mHaINVim5E6nzDF_eYH zc!MNV*kxNEY-e)C7TTjwu}I(aWsD%gx~RLL9~_kows3vS503r|R9Yzg zc7JW!n9tUULZFnDqX`Zmu9%ozIH-QPSiddE{Q&YyH|33O4U88s6Qg+qB)nR!k68q- zU&kQE!xK+|9nw>q6QXKVM_NGJi$y9dB4Tge`3p4U^m3NP#-Z_bMZ$uDaY>(QcEUP< zpQP83Y)|uh$$dtn4I1_D-gIgja}tClA0yy+0*SRo#J7w=;EA%%=7FkIRSF%!*_^6N zY&p=QM}S=bNw(~ytA9N3-7y0Xps)4a$Q&H!PANv7i}L@Vp2zA^ybC|0kw{B|fj8ew z!rGZv*UK)V?9eqHJow>@PC)ukTs_)Rtyw}?<;cpzn~3Au+z{zpvZK4Z9@3sMH_+{n zaz@xPz|jYDOkpah`z7Z#5fRa3l|4PUA--h)`yu`}r1oEQAE4o@NCMg275YN}+^hKe zf4uNtv><{iy!wrQW5ECG3;zucrKKK1gMs#So}Oob3N^i4&;78a0Ma2whGNlheWI3n zn@7Hy@A2G);vZ)~PA4VgI{|q>iPDS!-3xu4fP9*neDNEu-u4JQkW zIo2IXEY-o^;7Yy2g!rmmfEh$|E45_J=Rfwj#Is@Wp1Xc>-kK_R*B_+Szl#&zL zF0c^VtuJHoo?AY-uUf%Kc1ML5YlaC29Dscz83By?(lIjnRY7WS(*~*O1^7Kj#l^`d zk!=@6JpMKcuvp%DM+Ur#jX1m~0dR>1=)p|A$bv{S^_8!KZV=VL{1q&-Y1V!1Aumq7MqnhY8@tAgGB^KkY*GD}(M+LBw0X5%r9`0eBcrgnA>5PAO$&HLX$ zY-eII=M-UAZV=_ci34$UMoX&+*r07(hH^-z@e5uVCBmLRp*sNJwKTOZ%7Hi8Q)6$(>1C&r2|>Q7d0w+Vfx zD{g&fQ4Vw_opFnCqYp~UL~E-Fq=2FBf$or@K{R?HjgQA7*)MU_-{7iZ=)JI;t zv}6NPj3DMZAF>30x|{AN*KhPlemw+yQRB2GyVfhakC7aR*h#br_{%6yUJ+g^ob3-6 zNi&!K=fMcjthu?lOJqbPzX03ZLqhQRZOZZ7C=xrnyL75AX}}%N0qaKMv+9=%Bzo>Q zDJbrnc9O0K7`w=V5c&lUanwsdcJjE7xINEYhUwgaF42X7l=e0Ir9GljZ}{Qtc34!Z z4baRrP*CVl#LWbz*L=Gp&KN|%RpiZB`)l#IHqlsOQkep_6E5(i$BC)|2$pABybyE` zR}0I@VIFEvA8Q#JDhGswWpUTq6t5k0h=lGCkI>>16DeNGE|aF-ZPO=kr+RYho+Hax zKi+o{`x=i^XE>TzW*~9B@3a~NLv6}#brx{+w&5em_u}c@Al;wp?d>%S&o8p^;epZ! zKr8)9v|n-RkV@$z{ zK;uhpEc6FHw7)9f1n+xbjBTx9E5=uW>uk?b-v+8hbW0)#7CdFFq#ig=Nw_Yk>nw-z z0J8xxY2Z}#9df`hGuJRP0jT8|3QMc$r9o)g#*Z@S0VS`*gS*$cTbN@x^}@MGDPUSw z8bo*&Zlo}0LK2ebG~mU)Q~rv&gboTKwtAy~{;_gUke1LW#s&jRI0=i1_b342oXB6m z{*f!c*#e-BzBj*>3Y5rUmw5uWyt6koDA%6Wx$%|wMZnQxpwjlF9l|+Kl%ZRX^j6DZ zzA_=U5zkPQyg#znexF?1+-(Mg)mOIKow=DUIO5Z4k+ChB=Z~V3XB$Tu-cnc4Wl8%t zc0|xhnG13o5UqkF_Z+x}0Cl0UK3HxYNt7`3xJZ01`vAgkXgg^jKdsEl_jqN}&-%w$Mlh{bO+gs<7k@p& zxW?Bzy7mz4j#&6U-h-M0;9Ci2becYKV_=pLv-02!v7W9^UM1jq)SW2sPHZJ3bl;o= zNs<%M-r0;L%RR5qdARXQ?(*=HcOQ9>g&YV9+nlgiVb;1zlAuH`8F9u4mfCWiIj_eX z+SHw{YqH#gZ+EhGB5nkrzlA7vq578FHftWzlq@3hW;Bon0Hiu2pt%NsIgrGsm3bEm zVj=@|j|S&~tdP`F!{{XC`;?a_sTIjlI}fQM8t5t`XW3H z+$9>12M;hRsRNnSpy503ru?TDRMG(+03P-E z0T^)VrUjulW6iHasBPH>v8403UEUY-=Cv~29t~yu!q;Pj`KTH@V7P;UiY5ADUv}& zO8VW?!rqa*{W)4DD>uxILfp2`easM zxI!*I_EX=J{!DYs$6V~ZNIyy6{*IvC$+3#0&>_QJBT)4ZS@$a%pmdbv2GEhdRUEvj z{J~UQ-uIZn>;^66Lr1kaCnxD4LCZjn!~tGj%E?43u6jc`23zYbW5TQOEg*8wnlSB;m77%|-y-}= z^!sCbFJXjPy-K2=Z>2M`rJdn;UiYu^;{o%SC5}&2b>jwHOZvggkz5>(~t?;HpOdzXI?7#@AN}|NlVl z|7K%{8)k(r7?Xd?6=%CzoVR7w%a`ie{Khb6b`>b!nnfO{tRwAtD~BUc@KUP z29rpnyz}bIBqf^)Nuy)t?3`dbXsOPN!k3Y{TCKd4xN>m;$_b1pt`gEpsfo+KH8daP zJv{pzV`hBpbS!Mpp7JAu?(YxJqNc$ZKbY$aJ*v<>1|{Q?VGeO4O&DYfd?BQQLjtOt zFf*Hkh{(j$^zqQ^6k0IpV`mR->6sw;2}Y1n(b1s9--ik!H!cPS8w*QrYLqTaBa6!3 zwX;)rJk&dglEA_^9Fd%?th72q}9KRD%U?C3PU93N>A z?dywtPv|x9JlPu{G3hvqw`tk^_D%er)JA@s-J7X9K)Z)zV}Egmne0IPfvO*dBZKR* zuWD!98Y;;ta0!&cE)lB4QEgpaVf)YQAmN30Rq3?$_>PfNK8X=nRx*a!-$c3-%tQkH zqS@fa*wAnZ_5D9v(>+5w5MtdE@S!)V`7z`pMmf(Mm4)s$qig{_c+WXUeJ++JUZ`E_ zF9@{d8Pt#vthvVtuZpr#fjoB6pdZcD++N!EEb)tYds`bgad$R0 z&cK0jRX(ki=QlaOd2bp} zWi?e*Gzz0c{*f^;z7^dd2mteSQi><`Ut|IG8xcKkeM)Eyx(n#2S5Q_iaM-&c#6(a6 zFq=*CTk9?aZ0qnZwDoX6!V~7C6NNdA(qTU-h;IR^lZo_u1_n9}J(>em;hRu91_xks z>JL&_x1zs!L#r6nIs8I+&pei?AEt>N{l1>(T<4Ey-iOgbCL$?mL%s%kkw z;s=D95h=8cKU<7MPk7_k5`j$W&nh+R?}O)T7*0vxEY;Pw33nbQrW=D3Gt{_TB+s+*jslzxZebst`5`4M($>7g_(ziyJzN04M<#?*=k*ZB_cu5MuNE<+{VE zUu6~qRG}<474G#z{)vVZ4mPh^Fo-xmf$LTzsK7a>qRe}V$kuN<+~7#{JYnUMs8Djn zr#@lEmZOM7H*Bx4mfw=;p7Of%z#(Y*1)85zzK9h5j9(6p5zL7ZVmf`7)2>U&U_wUP z)yUjo?gN#jfk#=FDRt+J7`&6&a1%75zwHK1_Sd1)n~_T1v*>4teP}}Ncc<&yE>Pnd z#M6w}OR1nu4{6Ku_?5DUJ4Y6Ed~#wq#d|y5Q~`Y3J>E}<9?uhyf0zKaqCLoGAQDpa zAKl*HBpQRli@Xkoo=6j!*a1g3M>jVwg_D#GR5J8s3<+EL(M5z-$CGmB+fvmiWG9*7;?Z$d_+MooWF@WPjmS(G!VJ+s zln-B_Qe)pS{{xf+!pP(w`6ql7e?ws&;D8EZ`g1gvt}J2{m3(AXlXBeMYkktcVd?RM z;0I=AGja4%d+hi5V^d&p(7cgO#60#R7ri(#y0gTj3YPmIo=lf{!EPO=oO=p;s%U1M zDF81nF)_Ly1|IVDSQ>GowUumz>o@Hi$WR?$htnv9`>p$_bVpeXWOJbb0iMcZ z7H(ge_@&Z^P1bHOaH=0mK4UNP-Ef*I-E@S;3v{Z8n zr1OOjdb7P{+>A46Mp(^pzTf&S*_Yg#T>a;4fiq5WH5V+Q)A=gW+KY09*L!Zre2d?{ zj)mm~Dp$<#g{7tTAwMguMdPn8qob^zcL3g2fB7BjgTS39 z@}m&0ZLF=ayiGaRNt)-_ezIW|f^NxQyp$GmjCG}YnU4AyRW-(NY+Uvokm9(;#9HxR zC)G%U3<|l0p%)qXGu&_8H>x(5=iX^>_*Ep=0ohiKi@@fTkY&o)l#coWbUsgMRoF=9x# z4Il#hBfRHQX~i}Gv5Dz6LEtf`pH)2;kzem+XD5uL+%vCLQBp#e$J-8j9J~yaf>u03 zwcsR^^`a+}LX&KaYjIJ@t`{G$lnx@ENz)HiPC3(K5`?ODPCFi;JDIqM;Pi z5Gth7ECinh`(-WVN~e*1dvRy>VO~GxZG6Ey_Axy337^Rsu1*>+W5q?_ygGT$2!i3Lh5hReYv$P&`u%d* z=Rlu#r5+KFyiS^fz!Hdbb@m1OpW;8Kd*fQ<8^enir95guWon7ZyN4yzYuI)d!R8Zt z!+6%5_b_k{{XVuazBzJh)p-RFr{335qdbqcXTYoE=%NR|_L(=+`o3xg2`MQ;y7N__ zyzTwUpU@jHA3cI?R>3$>NF+7zW+$SUGY=iFa0Wr&D4uugQs~1tWb!B3tlCXXjuzoK zZU;Ol37OW=EX>Q*9v%>Y%58(j`=AoQfb4_$H})m!{+ib2!gvQFuTOsxEkg(I$?2)O zeV%pP9cF^sc>3fNg%CVS6_ijL(o?SHyHVjw^Ygyg1XAII_%y@$bVp^$3Tf~FNa2J# z_O-vSCX-FZC90kVG)G5KghwR07mO~e@fQch@{r5jpvA?s-{N<>Wt?OmT$gC9 zcmt+hcmBgOH|woWruT3T--ieCW%PjGO{KscL%DvoL>`8bG9s$`P!|7P67a=dyXi*N z+1zZF5{$8yEBf|>95HJEaKRS`798*0-Uy&2s)}J$&00EhSK3W-2e#MN^zLDuDvx@m>_rpU}^(ok) z<(UR2_f1F$rmUn}lP8ROo^8fQQt8D*dm6yV`vbwdD(VzPm>OBufE)52J84EvmojgQaXQ9dLycjLR3MdzsKMqd|0-rCkizBQJ1 zIq>V#eHf%+AXkM>=n;i?6K_`B4LPJ`Ui{30J)C4Qh+${`p`-&kw{F5k=VmBpF&C3| ztqWK31OQAWb9! zN6~2#^+{V|W28G=#HcXGe399W{r$4cI3kU2q|&S-9|PJAO~bnh!eCJgkA$B6@xpxa z?_mqPCwtFPhZ&H^T@ogii^Y|cbk8~Y-u*~3JmNACpzO|N)!8wFNz91ji1+!Ncr=+v zv~LMQ8}7Rf zz|Q?z`U+@80K6+_Mlj=`Dd7b47uuH5p7~o`i=*#E){~f z?HgsBfmR{^v57j&s+7ggs8mR{WO_ z59iw0d&7|FI^TFf0nKM@fT6W7OZqFo50oN!K0Z1a0+ z%`e2xx6>mI!gdD(Y;1=_btD_|+<~FvK@wy{s0S{l5J`;9+tys~148u#ximt#byWoe z#n_b|?n+v9^#Vfw1?91;7p}x-SJ^S*tE>9j)t~#9?4AKE9Mm))J=H$TeDOn;p?7E~ z?xBA{h^tX{ZJ-m2j!<}JcTvc$u;?2;Ly&{JNurl$#Cjzqfy=i zz4Pf-8(}2fl<2jAFHg?G>k1!@QZ$I?B-hRFDZKNEC|do&TNR;x#_X>fIq;h#%7%DQjFHs^)dpO^BHts8}L6 z)y*z?@>n`u4!%B=69yw?Vmi{4LEk%nWPjNPaY>UrMRuLBPB0UJdD@VXSbLa zQ!r<9RdCPHTe?tPcYdE^#vg>t@)Y;&S{MD(%q1!aa0E>ruu^BsC?u zuyMX>&6Mbdo2%DgUyUnjYDFDjI{v%u12rc^_{+i#-@es1*MC_Fpvuix8%XPso1Z$g z(r;E2efgtd?VmrJ0~I+Vs2h>h2~GPrsiHYYYS!wMm05aKtJ4nK1LWmv3qwiU&KZ`S zM*Xg$*M^8V_0oR!_h)`6+*_MSeKdJlb=%<3V&L3j^w*)QVddV6^0KVbZlAhdz+oX( zwwZE3ybkG3E?1~(%4=vA`iOf}v8C?dztWQ4{?D7^H&aq2&uj&vE>*6o#b*v0?_*>+ zpKqKS$Q7EV4yQ=A>uAg`);;3mxD!+q_oN{^vXn%?T{$Ep<845=S=FN&hNEXbZ`J7q zs_ps)dhG|+d_Ra8jf`u~&-44gUX^zr9}BsdM5RkB#|-LNaB<_?MWqd<%aD{+`as9r~*;#oAl-O1sJ8eGsmFhlz@Wc7g`Y7QsF;S*y{f67nV~m)ucZExWrVix9)x%SF z)&?t`UCwK8f9&B}3R8XJZ=^Dd%u?JwtxmPF+bhU!Y?WfFS?cnrkJ8J&#cWW63qx)@IT-XQ2u$bKRDiQkum4Z zv~w98eC89ku(nmXY04&ao;9z~^m(7?Do~s)e`9+oyos_;YCJKqYctKfvEI7+WM`~p z^~aCS+XCk`9_Om=4Jzl{g*A^qG$dV4K6xM0X6N93BEKepi%-}tV$uILHm_uNo<1iq z@8<3K~lCg(j_d9wM`F}(Dbw&t^m!DgL(7JkR3(41+a(_Hoa zU8nuevZIlY!jrgD{I=I7#C~?onS`a)s|TRJzH7WbQ24FifPX8OoyD5lEUs*S`JZ&RRQ--9EMbwE81&AM?rCSNF{k zC-D=%&zhX&*|`)`?hTqFs8Vh7sD8ZF)CVq)U2aF?rfysO+FQxIxtXFFw%ZpN#f&aU zi9^n3=PKiDtBzqO(+WXV)W%Kt$gp!D;Adj@?dH@J1KVoU1A)E&kF&Rq%5wd>M(Hl4 zyF@yrTR=rR73nUKmhO~L0i_W^DW#=5B_st@QaYuS?z5il{=MJ(o$-F-JLCMbhuh)c z;kobYS~2HbbNw3QJ?Z&y_rMecwg72)=1WS5*k4A2RhQB$ZR4%du4EhQJ9_;w8BMi< z%gcdJ!4UDZEw23c6z$t)?4?93-$(YR9p$6u52uoHZ3G9YbQ2SEX=L_HcT*f*`l_xS z?){*>Daq04pmi?X4QDd%o7opP*Z1V5E5n@8-iF2bV8sHGuG=1JgQ1aTpHE+SphnIN zFoQJ-eMifKTO~7J>X&~~&l&l=?6%rrr_ldarqf+f%e0{f;@_NyRQc0-p zwzAdz`y1=H>Jlp6qi3JLIC!cPG<{sS{>CAkNJdnBf%v_}qLRtEIJp6c!uX<`@U-JJk3xkjXBargFs)t{C7vGi54CUx7B&8u1W_+Nhe zE79q!sqlMV9Da~+9CqWsROJ6w8^8qoM~edWUH5FHjsrDGZ9{!OTZ=q7NSAHDu#==N8uZ;PK(cc;&$V^|q7#>bDsf`ZB1= zMisjs+t2uZ;AmS$Uqhq7R*-_cbb3?9-OzNmLurQcuc^d8PYm&}{#Eb)73BY|^8P7b zppf~e*7;A-^3U@`lsSJV1pa!8{4MMLc@6*XAMkIT1Oo_vKLe%#U=|Vmw7zd^c(-SG zmrWJ_mTHxusQczvR-8uoySBWufZ!P!`ew&HkMJf&)_eKPGpB?NelsjL1m(Wkmi3ws zCVYKm<9%`rRtm!2J|_ee(q>yLTg}LXB+nxxI{cp24w1iq!)r83na9Xlh|ZZeWPBqy zfMfWv@RO;!)z(2NOd+BE{pA#P|H#;hmAM=Z!EwtP3-#5Z<_v+y!RB6;P1DF<%SS{k!|z0`2MeA^uNF6|9T_;!)yL$ zF8%MD`+Hgbx3~W9EB$W=196J}{ojB71dRZ^<_nziow@P>16tT6MiPzjzO2Bjd?4bn z$PDRW#8R`<+`GhIVnYvfb>VA8jK;q*@C-~e(a6#eoFl@qBZ0dc&UmPKTzh(VO_x=XW_b@xdt8?yhwb1~tlBnhnp5F5`#&mB&e2$tMM`#$10d5MaJBGyYC_*9`GnrQ?AD0w_FW zWMmM+4eg6v2VM?#3&N!DYkx;dxP_ zXa(y8vE?r4ludxg559H?WX1I-D0e==%!qKK3F?;IgCRT#sSE;@OW+Rk(70}_RHntJ zuPfa*>)~c$S?=aT%O$>h=gzgM>FIhCp_|efQldmEHWY*0U6c_&mH8eGvf5;?HhqhE zfQy5}kNza-Y)En%U;S} zMOoh5RcfO*ao49-4~BsM6V7>V|5n7-DlK6#*TX`Sj?6W}yGnf%jpNp>QPE%x6N0T% zsU!_&4!$>LvBb455!tD+WyBrtR{dOFX%OFSAAF&*)h-97U9i8{X)?M>JySB)xFn}$HF(Jnpab@K`7+rDBa_w|I=hd&Y z1+_Uy-w=Ms<>u)5L8sl4>_F<=yM##)*n%lSyY~>pirK2M#IY-P_YM89_WZc3t zqZfA2+tNIRi4$yz=e8SH4l(K~V9*TcZpM-EHy7N*+(PhHOmb9-oYhC|z+jTkU-s+6 zyY2S$=`xxc>(c%pHW+l0y*Y#>ZJimHL*7x#`%9DW+a_%&^q-FkJqeRf59vjid%C;3 zFX>o2+<7Yln-TNhfP<^+2Ds4hjm3yMqdd|?|;;VLAuB16M0&`-toWCuNwSJDpS6wy#fDr@_*q#a#y4TQ3>j&_SfwgWCTvtk^v8{jv#YDm{;O)YqWHTCG%cLQWy@op zWtk%1QDA@f%$Xz%{1B&44b{_fGvdAB(5==wAd`z02gpX#^Cv#W>CZK zr$*6Q1IIu%4|HN#sg)!I7!YIP6v<+L+P~|^Wr42?)bx8x3xuzxG*d-~|7*DCm+^!Y>l_l9FHOx=en@r9^DCx$o*5&&iGfP5ll5 z`X6h1gA(eIhq;v)H{wdxVeCgzFnJ4Y15FlFHjp>JsYOka)j8r3&b6O?ye*rO?oZt6 z^X%0g3gMD+EG>D~%RR3wKYe_wSS$C-*O)BUOvzw>Rh3 zjfj2u*(49fw2cG^O@q3=FunR2_UftWGE9@j6thjx12wxn$;{{?rRFWB0%a|rN&aIz|i+)I=+Lc-* zXK;m#H#|+wVs@{D$03nJ*Hts}+W8$qQ{2=8m{eNNlT$V9#UWusY~pIzuJ8Mz@MYcMhV=`wrbKJP~%2a?>$Zs=JdjB*y1B= z6ch5l3=hrwra;q*#ND3t;EyDDk%J<75)-y@zx~N~-KN5H0JpSKEc_v21NOx)U0jI< zf;`$T_Bh^dH;1FL_9rGQU(YAaM)EA>@Sl8;NX*Hq7Lcfu1_5_T8tqLaair4*`$5i9 z-QpS3jxN=5hUSJp$nGUP4Vwb_=01zvbu*B2EMACp)%l;U=T8xgq<2>jCa@^WZA}kV zy%{fUI-F$B&p;vSUg~{{o0}^&Mpiln2S&HVa3TPH&lj+Y0~|oi;c76fv|`wn)RnYy zkqa-`za5a?R!1RYedC^B$!zor?F0t%$asRq&v!d2Uf$kHzenQ|#%b<2PTS!A#%m!8 zH0Ti|*Hd1pOH&HwFcMLh#r0>|lWP)NE(Tnltoj~quFoN1*arA#vLcs0D5grPN-)}U zLheHN4$kY<_-Q`RjExPWtmH zSrE5zMhTdgJDf!gnx918U(1^W(}$OYBV3kl91&{x#DaduyEbqA$U>Lcr*o-3hsJ1x$$^QV4y^7UvNpzk+6qB(!~<|9iKKpM11}X23D=b`bhhpI{ z`3E>(=R1;f$W)qbSXrhm#N)u|=ROx`z|=Pm=IGS+Q;AL)w*l$J?uarkwwjMcM<$kE z;*^M6%gt~KQw1<3-)Y#9Haq(9Q@=nX+dUsM2q90`%G6)@FJpGiMut6|r# zEI^+vGm}@QEwo!O$&c+t89tq;b$MUMx2{1wk;AzFFXMYf**MR!ZCl@d&Zz6tsXJdd zI|oLpP&7*q*c;nfZt8y2aWPgb4xg-xYZm2v1v5PU9XYXC;Ea(n zj5eHy-2`49#P{^i*9)W7;!r8FMvt`Pe}N44&UiGb`nB<9e+k3tvd`yRHM7-)Q)XOp zjESpAw*5y7F}yfgpFez}#dwP?p)~04bO-74G&@71U1a%FTY?`Ms(Ql&7WSLmAe~Ci zs|F_@DUV}R>MXbwAWDDADosC1F8BmXYq`l==}1;RLsU}|3c#jY>Vq*_xk?k?wy^Uo zAoYJQK2fq!*nV>A!Hw*$$(LBtNp;mY>@HOs_uK?s@h2qw*x! zZf|a4+@YK3Uc<~m%Xle#TVf?c@;1&jH(laGG%`3-i4XB;M2Dr>q+b{js6NL{=%o^{ z#A0g0ub!OV!CFWFHd;K|2RD19kW<@ZtU*_q zvF~+q8E^buHVPXhWM#X6vNd{i4MUb>)h5$>gUuK~vql?J?&6i`8P^zcO&LpvnAc+FDO6T*aa zDPNDInixx%qvzBqFuizeWuoLF?0Lu}$o#PU&w#Q|@~Nyp=A@cq)l>;`c@;-|1;G}F z2ZdR%;|rE0`0Xo(@aUxtdALa*)w_StmK|&zLHWfdvccWl&3~v1d~HHCW17^w%9of&mdgTzfc#=;lKlL{^@o))hhv* z4mEq-QA@=G&RNr&TZfnuu_u4*4-ZaD|71R4YF{r&sq8qGfca|Hn7y7y4+vqCjZ(8ZBc z#dBKx;iv_E#-VV`G^<07|M#1Yj|Y>7?#CN-IlMnKp>&hOY>3?#-y#>rO7BC?;Ga4+ zhGKMn0KcJ7--jgLD{m(?D)zV*4X#T&jTJeasacGBF$&FD;O(D? zCyJb@52hBE`OQ= zcU1VvQ`AkL++II{cg(6OUcrUB+K!yTnWrGP7B+@q?pBR8YQyD`0_ve$!nCp5PX)+& zqr|;9i;aCtmnZX7sq>@!ep1FalpCD9vx)4jDW5XSxW@rIeA);s?o+pyYcu%Unnve- zIFsqzMY33fi=tm%Hg4w_dH=`>DgeeMPnMIxNvus`r?CQ}5sb0>ZQaWN;}>5DNe_zkt30;!r7Yii`oUa{ zkFvV&w_Xu{)BVrnCg34-e_;Av(8WP%psFXqasWwTBsA?DGG+8XsQ!M|zV=rT1!+u6 zh{?sC1ayDp_X>^i(n8luK~g^fr6#mI`O~w0A}h-qqv|r@F;I*RT4^5OUtFH;tS&G` zOvCInb@TcU`>Qud10gU7C-b|Lm{4-J70#(Zr9Ze>MG(+_Y($t`NqA)NTig}kml9z5 zx>xJ62nrUM6njW|uKLnyxnEmdDz_2DjF+H`zDjR?ktZn?5`_E+oxvD}4e!x;D-^le zC|(KKkAcya*}0nk--k9#ShC+Ve%qT$)S00GFq0WLbn!e7REDPRRzgrki3Szh-Uc{A zm{#Ikf?W0n@(n{87$y4=AS5SK+I;#k0?IXE!-zfzxR1eY2@2g$v!-e<6?b4ty3Dde z#9#Gf7Gs#k6fSD<<@u_N%+a(PjIp~2%8r(ekA)+iKcTUW@of2WHaMDU zli7Q^uUZ9iL3n`VQn<^Yfc*hS?TPRHnhU&5&-D?O*-~G>#G`#|#)kb7W#!jwn(*P9 z#mkj~YIxbrx%3;Nc ztlcB5AZ)P6nAYs1;bX?$iVq8Mv1e5_WBmj%yo8`Mqn{riK0Xs}YlIurw+ zr+)1n{K4U@XpYZ15Ojc`f3G0!`qdbG?Dcsw5o z@K-E%Y3)KZBE+q(asI&6f90VR%Zfd;ay47B?XjlHcvO$cElG2que zTw~0)6bI}1ZzDD#p`(w211MJxz*2{V7{efET>AO`rF7cNbjBB%v&{;UH_#Hk^_aAS zt7c^$Jp=FX0xUyx;N-#XXw}k7>B!z5zVWazJTgQ z5Pn?*74wBJ?EX+s7sj7~pJ@*P@-1W}m`a0j#+d?5X3?~#XY-Nt)gVTOf)~sd?}GSV zV5NIM6}xed51R3yB5&PObAmKVZ>^2IHD^UH889f zFkzK^usPudxff>^tQ+=u{$z-`k?|NF>;5F%gDZ>J3j@{yL3Pl++=C1jwgOSN@o@4n z9LYW(wJ(xc9^2%sLL_VO3zhX6Z#oR#zwWDlFos1F=Il#X7Szu0KQf05Ja{_@O+c*XH+HG3VBF*<08fc?0 z;u%x==`i!B;KUjQUL1fG#l2o^a!t_(U?n7;;0JAS{8E~vki$4ppbR*HU_GE+N)1>z z@+Y9exdu5`X_wXu7!2sH20%Dkg>v)1!u_m zfP*~1+-ykVe-j^6JNp*vd?FWL>^P=jvz%jww7)n5oU|j@A);UH5Ot9zXeZ!2C)*SG zD=u@9j9dRk0*#of<@Qki&QuMdH()t*_N$j1Hwe(a#8DM(c)+*7Qxr`Lm<@nV+L?c5 z<+l`yp!N;ShASs_GiXwMU2$cgdv{-r__G;B`9W-H2J%pIw=iz%@a9Z|$dQQRtC1ap z6;5^4#+?AJ|7M++CfMjQCrhskOZv8h)i!^b#A1@SS@Q63JVXgO z>TchE-cb~(l2+Krt8T1X9Q3Yz)2@$FJmBU3%I_DE|Ml(RM+Oo6&Hp&zC|UCV-uvwT zigp_R^FNjQuY;2Pf2tw=y;b4=hkyG2?{oj#ulWDrb1~B{4{KU3h~G}}fH3Lput?|@D>(eIw{B%B@awQ1l|CxB53a1QZl zG;Iq3wJv0+Uk!M_A#QC=RIt)t93O=eN}J|m@#zss!-?1#GL^jr+GeO@SP5>qOCy|I z!9P_y{DMA2=djeM;rqV7x;0{aS&o$}sSUw#@EDz{->Sqag5LsCEsRtdg2P}mDQD8E z9r!qb+r)Wbk{Csg+<-{N_56-MG*2cz-FX9cMJWRFR$G9BkzR^Yrs+O#5|4S)0+EDh zerIqvO(MHU-32QGaV(J5hJQSUdj>-AV$Jtm7e}+ut9O8^7gFW5);gpXC>}sgT?a7| zMu2`mj|d~j=FfIeL^ON>!%SP#7=o7STRuT>a^W9VZ$!}r=_4&({QS<1IreNzCTeR2 z;imZ=G*0_=RiWE~@4N#`TP{yw7?#QruwwaFI=7mR@;=Ee9o@-{{nL)i8%rZ5kQ_i` z3tqmw7`F}vCGnxh2g};eLMrEtBgvy_6TPeq-I`4A=jyNX+7(_I0!x z@)+<`HtE^$f@8*ADfOkDzdy36a+tV|wElooos`XdoL`9BE6MH z?X&zI-;JC*11P8VcNaCxLa>sE%=)=OctdM}lnBZ0b5XsjG?g-IZ@8tfgtpfxNrhxF z8?S#-er1>5JGnn1wCTLst?xYZb6B5cN8`vWIaXDuGfc^B3k~ zA@y?he{Ggzp{HHyP?f)7(aB1ibL{;5e*fSDp^gz(^8`if(fX^g97-KFluOqz2{P`f;xu-y=m2PNxj zQ4sd(K}xPyh`J8-LO%>PY}YQnfE@SUzQ}JS)yCLsJ{$%(ossDTZ8dV065*t*aHLP2 zJ9&S3_y<4~1UEr$=p2me4Q|GcO$XnhMnR)Qq%6q!h$%xjlTf^A%Cm~Qmtz(YRYR=^ zUsCrIj4(h*CZrbLgtLa41~ukBKpG4v%5d!86Azz2gWu3vs$aFyNvv+U3YUhIfKJj2 z>TmmIc>a2i2)XECUn;`gmo(G=a6(&6EEjS;hmCVqK&pV1n$&?#LD;o}z&`Gp*E~i( z;4p1UayO_Eamm90-5AveCJU5%TE}2V_sa~*ruHeIyr8Tg zeQ%5PnZ)%F3SDuy`v)&g1sv=rE7z~xqhVC|g03MnjJi0^a2X-%WRBoYVpU2O zv{|d}L!*P)tBYAAHNherw{ASCXVg8Y-!GS6d+&b?R4o@5kYc&{;w2D0<{qw{0^f_5 z@ZI$Zr)wmkpWE`63m_ahCI;t$>g#gOjcbt1)&)s6jX+#5t4 zE}K%gCA-I(>17U^L8HbhYQ_f$FpFiMD&&DZ^=Y}6LTuG#)%a`EM@efLZ^T`2ymTKj z+1c+iB;<7r+n+vO=uZ~+tK6-%xRyp6o|Zzp&U3b0vfw}~fFzjC^=gp$7L)niupjVR zAKZLJvnz-6sAPry>iFKZvJLY{ynn-2k5#Q)I8L&$j~%?QGAi*l9Eu*8fxXx<8m#bS zLAGt4y1~aE5#)%(QloB}Hw4h}c-3_7#mVa1hs1J>W3%TgB7nMvDL|o3*3+d|vhBQ) z_P7KQ@ycGKI`55JyrgyQVFZ5HlhkG^JCZIstd-2;Wbf^jA=F7Z2Xmo>0{81;q?`qZ zfu`~sH;bGPf==c&hptj~Ko<7B?fPFllA6W(LPll?k27mUOj_7tHjBsyaZHIYm8Ex& z2#1uT6PL&h>aoYT3avy-RxO#D40?7GpO)nbrQ97dE3GOBG(}OQw<<@0>KTRsDG0`U zihARlDR0WIpV9;z&l^T?O*HhOXhD}I1FS0a$g)r{j)gt$N}S4YAGx1`jq9rFY>B?k zt%CoE&8H{*anW^_(4Wu_hA^ASwzd~~L#->O3w;M^9PwDvY6e^)a6Z1pl%~%3)bHyZ z^73jZs2y%;KnDT_l5?G(;4%p5*#tuWJScTEjQ|h8U+-lxE-nQV8tEcLI(il2W*` zfm@g=!mJ!%zSYir6R{wI3iZ7+F_%E^rb@c7YTTpG%#=D{!!jFi_JnItmE~BX!J?a~ zZu3*$C2ZAumjAR*vj2;kHHBP_?J=w!TuDCf7MFxrRwPcV-*0d2sZ?arl6j#u42j>Q zKfF);+I?r<$?F&;{NtaNk_Q7%`b?mdu7EHOT*!;L6e}-(s4d(yuEfdT!CyYwYq>lq zb%M_C8dBQ^IlER4G(3YsuQ-cC<~Y(x4i}w5zxYyGzi|Idv7A&wdH_8%POZ>-E$~3D zise?BOWN|+h`#N38R+~Qcq?B2%ncMEDfVb z^^AlM-G^6j-Wb^i&p{KTbC(*jc6bYg<`srq)2=N{{XydEu$Q%jRGDq5yx=YBXy~R*OkHP!#J`GYR;uLhe37@_}XQ-kH1}gD-A#w z+V96ZXT^@cBTG5=j`39))}mQlJK#FBH?sN5!soIX&r*>mcMH-IlBPv$PtXr`f5Fq| zJ@&#~+`!q%1ad2Fjs6EmZ=)$4i3MH3bvESgA|SNp<=)+8=!pe&C{IOY^k)hz0WDC79iI)CX+ zw`;JO^&q7oA5(8Z-|*r`0UdZGE=dO+E5kh zOZ=dQ++@dOj#n{hR#Ua*H&68nS{>tkyM1V$toB5~o2c-`#JO%sF}6{oS9#?P zAvZq}Oyp?}i!JIV^`=-6=cA;n7KXEWqYgiKu_CFIKM&cKO2iS*jm1gBz&uNmn=bX^ zL`})@2+-zHVW`+Cl;t`~$vn@q>rCTCO+**Ch&?JVAc5r%d8RdO`DsgZynk4}Jw?oxv9nrhyDGV&~QYWK@hfwKXIS zQ+jg?))90{qGCW$puWa@U#}!WV**IUni&L{tTsK0Waj!ig}{8v6Do#luTS|tBFa;H zgQFGy1)#hLJW3S9phYM~bmU7EjwL z+p9jj1;sJTomB;T(#zdQ8RsvCyScEKmA7#hg|FS?CQ1U6Snxtou%(1tejG0X#kkdp z&m3ZE{7Ys)4)`dc$aPl?N!FS6Qm7{4_88RMSc5QBMGBpX-nD(2rnc^vAghU+t9_07 zf|_Y2En@?rpK_jW-$-%P?!}1vIM{|s&lQ^RJVm-s)*-$;m`d1z&D>sK$hb@sg3OeW8CvrbN z>T+g8e^z~bc%W(%9Ci3nh*`9N_N6#8*TchfX`5FHSQXH;>km{as_>^&zvJ>EX2ez) zv3KnhdzSe|m7P}UsET?l;a2EM-y>7}$BwmJ=Ty+a6v#zT={*Y8O?D z-OLg%K#8630d4!HoMnGnHuu-@8;E0uB>vi@WS^%}me|8EF&}WRy3U)gU80@H3feo%Bn?WQd2Yv zlw_}-whlrWJki*6y5k^sWa1Ye^vV}zJZJHLKk|ao4x}`n3u~i)d$sG%^3#M-tx4u$ zuG4DJ6tx4a;O4Y#`I4*MnIfdZA;^Sd4bxK&f36^dKh6b?ouvJAE&sFNn;O)osJxZP zX{*cF!ohUm3k+CFb5u#BcZnRh!YK$$LgW_PX_whiZ);8?pKbs_WYEPVFg*dvGUy$- zKLn-w^xPBf!5$A5yEyLC5B{uIkJ5~F{UlXsiQ`w0EhepdoWt$mlz5t!^}KY^S{8K( z_p3rd4bXK?MDXHAmQ_^=$I+i1uscFa)^K%sh98-E(1G{iwOB~dv0gYL?}sf(ft0F!Tp#D`GsytuEB_X-C#_5KaI}LVt^9Af290c5!S6gGVQ$@d&}ST zVHfNWEob0h#rnx{QjNnJvp11Q@X|4u(Yb*9Jp6nC8qncSX+NY9mY=AxVB3L%*y2su63w2#;V7iIo>uV9!x)Hv4(u6nyt&*my_+&A}wyL z;v(jC7)xo;T`FKf!_HhTS5XsUDRtD$qx@pl| zHIxHS`+ie@@)BlS#{;f(>D=C}rS!p=>6Ae20lyeA+8z3m{^XXHgqAN2>3#JH?>{{_ zWn9GZ`L{J!zq{@a!R4&$B006}2Smb~>Y0tOFd%UM9)AK9VW z%eazXa<5!)-R9$wuwx6aq3 z;fuYv`a-Dvt6OO}Dy8LBe&D17z7XI=s?F@9dk_RJ2tR8Os`wnucwW_+D2Wuqs*9hX zekkF+tG?p>ZLc&eLxN^I*U{x=T?yx3CXQ^pI={w(P}ZA`;|{*DgVLa@B+XaFSZmR3UCquf`Z z#54k8VRN>@)9H)+^#vQLHOW13?3Xyr5lkyALQQGEO6e-iG4{0fmR__(4A-4LQe}xy zGtMDe4+L|81Y+K%KVLWjY6%2Rues;dL$}3OD&P1%n_UI{LM)AO8qNw_6sVcO=Ar&`oR5Nf3DFSHMm90T>a89)BD>FAWB7$_bj$43#F^mIXniE*p`W{ z(@KdPQoweZr}%p@uqpOkRj$^cwTxGvSK1Ct$`@P*md_qAn@qny_Y*Bvu@Ois(!RPLz#dE3tQo_= zB0=*0>C#OL0xvB19_9(3k{T}=qZheBMlp}AINgD83SN}0r#U%mOXKd0J+?bXS$PxQJDt3 zIV|7<9q5Nk_;A7yp4K2H1_k3(Y*a4uG`m0c(u)n-3=!cAU0diy~%qMULgDLv*%vOV=?e z`GPrIB!HOx%4aQ9B73|38uQ)OHZ%_aQ4`)SH6x1wriAFS=UxuLW^kJd22sB6pcP3O zQ0swJ5SHcUDPd;aFKGSRJ^>gc6g`-@7Ds-$oc8s0W5J&SM6`pY1P4RtGtsiAmI!a` zdQ$q#KRu0XHXnmvYr`-b^tnni))X89B0s$>z|b4vV&%Rz46Sn|@F${P;J6e)jPw4n zW7P@Y>Mm1!bCz zJq*m;Q20}i5dH})K0^YANJOOlMIu&%rXYfx-ctyYnEsKTv!)(+O8l}&WFkC`X6X_r zNljQfkMx8=epk^Chm6a1gR*NG0y$ND3zL(*dx<<|tRPy8N?4kTK;KII#e)XYknE?_ zPICW*N~F_#{9$q<_jue!PVvS9oev{{7e4~=hlr~~t0|cgju=kOCpV>gCYdq;du1QW zHQ1PKxw?%>89+HBwghl#b$eBq^bJOY2_`S2dRU9;`C7n?ROc2Lk}XY26=U%UOAJwg^UP*-CX7qQ@-SjcKVZdKUMc}b-sjLQvNkzs9F36a_1gOX## zhdP1`=;EM=8nSA{L$Zf7^v<5J4YB(X)G7~(&`bK2;K@W8#3a)zR zeiWkxs0NEr&~esO9q?U%Lqwf`AhNz&(y;By@Im?RA4S0%9I>k_&<;PDPqpomgf9iO zf_Xo*P4UHd3sHJL29BzV7Ne<0gGL-GL&_>#3ARH^f!t8+qk3PD?;Sq^X^@{d`ota_ctAj%Qnrj0?9 zGjbwNUN4N6xUk=}y2^1T4Yx=rA8BklTO|ev?cnRE)u~5N8%G7*pH63W!!If?b@|PM zVvUT(S2Nl)l>dxxzgw^lgmHkQO_0BRELOf9N5qeJjn%U%ZlbFG-wS<@x>ik!K&#O9gPldt4`h7o95d zPExe^x1K|~ASQcv#{v{`8BjH7IG4)+_Q=4{+z46P<$irRawrM|8a8r;NV*vd&Fx$; zE*$(>jHXw+o8dMX>_WBh5s-M&vUaW_JmSaSi$0L*?en(V{%jQ64M}kxeFo6s0mwfh z@u@LBLPi#X4SRls=B~5Bcf$&1*nlYz>;yao1RHcYShi(9_`7LB8&lE5>$D-dl#p`i z5&d?6N)Gb#J=&)_kn`iE_9dYVx_$jcq%@v3RyD;$ldtFb^K%Vg*DLTlGa>StGwi<3 zl3Y}dfWUR1suknV89xcMHH4p;goSNGP{8Rg&Xq zY|Vn?OuR~XvQwMM?@YBVd5;$`c9!_^)SbqE6oOKh9&+;Bt;enyei^4Eg%&TRlJaBk zM-4ownaNXcQ+Ov!Sq9ne5MG@?f`;(?$9J+?x3 z)xcm$y~gyk)`>Kg;M=h|cP}4)F(+n{yS~MoUYGfYy&fuy9KVSB`jpBjc2zxXXpK+{ zk%d-UE6{y{tWz7#j9}N1hh@x=aq-+Q4t~I@!x7{W6gHl74MOI3>cdFIW$0-%(tgJ^ z0Z~LM1%d0i$v>St@L7~zvq8Wem$Nukh=$FBEx|CG5}Wv&v$~@>4h4k?&siHXXNoB7 zFag0(tZt94TLgRmg=@+ws1nUc&;O9W54}}=+fpZt;kItM`DZ?QHU-!EJCbqa{Vv|% zUcMH`FEZm!T=carK(>)l@OGS2(*aou4@3@ll@z@gLZdSyDaM3~QVl=I9tZ@PAObon zkLTuie1$Ci=MYmdY@DH^yhSLm(Qt4}U~4I;Nk9=dMo8{u1WfRNChZ@x`@#&$!|d{= zj5nL12Eyxf=}w4)a+unP6^H~uabwGKc;TWfW7c8JyE<<_p%`~GLs5iytqN;c#Pyzn%Au^7XQ!<262R?yxO} z26<}`ovDyo7Ai(M)4%2%>t#tc9I3R^zAd|&)_1C)mCY7ajaq06Q*NMNC(sLifkla2oo%qZ3Vc>j7|n5EH8t_iOCK`{{%pt~&@smL@fH-HaahX5#PRwS9Zh_>zK^1z0GCt| zgD>CbOQR2s0E=JT(+qK}P~Ur65IM#zCuHtp+}Z8zx?2^#E;^A!YQ4B3J;GSnI#zTO z^`>f?PJ_o21ethzvGv1^N>Iw-hIe+h#o@D`;HZpX8DQQ5OzY34-G-h3ieLXySj<-I zkCZN7YP>I@yw8ZDtBgp`2k!^%4oNwy`p(=^cWxYUYw`I5zZs&ekwP~ck6%n>`~3!Y z=6)up8aQq%fCZ){L95`A3(Ew~eRULT1t2TMl!u z5KRL@g>h;RX%~9*d#|phMY_s0)lypfp_VPbJ=w`ddoE5A|g`DGjyoQ9kxPO-&3OOV~^uoh|?K zvTc(-Dr}+Q9`8RYng70i0ul5f&{C`o7p#Mi?^?#Cs0PM`a0bcuOYwuq| zYuyAV1F#V*nMbR0P!r9-E&_mtvf9Cw@p9&@|Hp$GnK{B1%G1)oi^l>%QV<9STmDcF zp5DJUN{U%`(o}Dq1>DsS*>z8%y^F9+3+sZ`*uVk~iF8t!t^ zcZ_ruk;_n+)`R@ z%Ea5#-cxm{d|qQ$W$v^M-~HM#myI^?h&4fo0- zC~f0e5RR-ik+IJ@3qL{?pgdlRZD?KXoZSTzO-UCYfe3XInRb?k=Qa@+zYMyG7*b9T zh%4qkj6ygRq^;093)~e#7R*TQWECB`(R3l;xR(C-d z<||3a2ELIKRdU~qRViU%)-1)b9wyS_72cb1&!NFv>+q51)M;y}Pv|PhBcOJ-b)5he z(hx3hqlR4j8$ir@raP6cP|TC=+?RRRNw?gH`;X`cgjH*i*Kq7W&7u-9dY0wJa zWiFHFU?>7=0$$58$bd#>2$YDf`}I054b#leGxDYRdSa3(%!4dIJVn{bk7*I&Y)CO^ z976^{A)o^Z7ARsq5|V{%(BfTfd$u${g}-YEB|cpr@uu|hYi*+R75J-dyZkpv$Co|K zFCAbtpribFef>(>W5m>z(QU!$?~tH044~4N&FcU_6Cj6U8=12S=&6yH+kOM6;tXXH zz!<0T_vpShf^LNE>Qf)js~fG|Q083%9im~}65y|btuk*E@mR&GgZ^?Cb35l5J0+B9 zcNM&#jQC3Z;|4X1&}qzw-UL*qJh|G+AeeHhO zO_dae1X2ziq`dX#W{Mx(T#^X*gFGRWy@(oCa(YfBWd89KdSvr1m{1SDGq`&;dqrn| zOUtQOG1OI?^HF5>6;RnafNDi^mWV=!r0_GOt%0f6bd>edP(xex#zVL{)}fC4?!ep$ zUL(5oS%`85Iw;1$Mq%D*{vUQiI2M3?EECSAef}HAAqE?xwe!)}qTOG&@Z_p;TUE zfxE&aX%>gI4g%V0f%3 zq(d)d$zE?xxoDZLgQVV{Mw7(F9UTj>t29w_-`UVkletbdi#@t{j#SDfD=6$ z_e_Y^331cZ%7BT!OY}y7JuM0Oo+ZG|iFSR3XesCWIpNB=$*)TsXuQAKb=j!T5r z#|tjI3eit)jVvTyh3jMngdlel&kzb7PI|n+Y)kesN)+O0Rxw(Hr+pHerd(0KBEZg% zo=!cfad8764|?7O^2hN_tuH9}7a-ONw17nEQ&9xv<)6`~K2VSJ4E(*^cr$wnuH zzNH=rd>qQ~tMzty*y+(G(1~gm%z%UDA-W0IOF`$jR~Z0b&^lZ~MVqB-k88Edg9BTr zn8o|ctRc1%Re-V@%MQ>02I@1WhtE8Ie z&5*ULA zk<8DO+uQbgQK}p-|0T@*Ti!Zv0syJ4Ku)+I;xd;7gz79LgH6f$sz0$qG?ZB_$=7Hy z5vm+SfJ*c>zq@Tvy?}JLjy>`#&WhYTK=QxeK@MzBxQVriQ`Cuy9BTb^-1JEb*KiSW zr-LZ*YaDLEYO@8mL9xit0)4(^YEV@aR3ek5GbrB)FegoAKnCdj#q@iqdvH`=Io#V` z(o@Aqxi`uox`CUV!_L1l5#l*iv{KmZjXEE7 z*fDuq&i~Z>42b?lL|8V8Cb4l288`xhXU{v5m2kZ>vVj2DhZ0PC%fTv-L=28+3?FdAYQpGiFG|?Mc~Wq<)O%Vy=Cw9=?X-%Lx2Cb zXBgJ)4H%+^*qzoC`m0q>RLI|eClPKC-Hi`B6xvL}s3+v9lbb~7*vPGD@<2Ip^!>zZHyaoiny}!3YnBs|U187et!TnzDO(?DK+Lj!pBZ7PZ^5v5 zi5+_Fc}$8>zduc;1sTi+$Wmx?es#;BOADUV_8*`%9p1+x>0V!_JhfzfJYV`?QhX4n z<%{ne#w!JeTT$`$(K&LMAJoUCz=6IOqZzToLqy7tuWC7brS4XY?X2J40JJ2zt`_SW zdKA5Q6&80%1R|bXenYKq0vQ)j_#bYZMZEpOU?7j#wP1a^VdFYWv|%$eO@ujI#3qx1YxV1fH&bF6*2UngC+>Oe6pR literal 0 HcmV?d00001 diff --git a/docs/images/async_compac_2.png b/docs/images/async_compac_2.png new file mode 100644 index 0000000000000000000000000000000000000000..368628de9f38956a034dab4cf2279599fdd7a687 GIT binary patch literal 54164 zcmb?@Wmr}1+U+Y+(nv{zl(a}oN{FbGAV`BCA>GXqBveX}P&!mfO1hN}>2B!-NJ{6q z(f8fk{e9N|7~>v~FO?PLaIq+`{`%`LT=_dP5B~b=;+MbvLZQM$ zfxnbUD0BSv*R#LmWp1gtj{c~|P*&HhC^>nUkhR{Q=}>caA1Tz=Y8l6H+n4dBukp*f zox(3KT=%*9>T1bb6&?c_0@+ALI{DAWjCgk#=u%ocj|G41Qh!`cTzgm0Y+aqFwKM2F zq<5s*rFFwAx4O%H=u`qh??)hoN$+=Te2Sm`?AMP3SbV?#NNjfg3W3zM?%(fGPQLW( zg%nAm-!J$q5dZ#!ln?Xox1cEe`UECg?EmUg@hJbhi~hGusY?FuF8W{Z&jIWAC;ok> z|Lvmx_5Ktu{GUGbKbPuYrTag9=zs1{810{*{_9HqbJ72{e*gEIa=PL z_}J0W@%W$%;r;wd9Z50)I)b;iCu&}Ve_g8BI1Dm5cwH1@Z zCx(aj9|t!y$ZSW1httn-ypD{VTUc;j9WD0p@u{j3b#ZaQaj>;j?x*AA>=+w6YIsTH z<>j@rvs3A^emgNUGtpaiPB#P(&9vlX0xGKG1W$w3f`Wnv4<2+YyK-`|a&zP3;_~0PAtiZ76o+OgQP=_9 zB+Q$>IWaMDYikRMlG1HIko7PtD+?DFx6ZP@s>ge4t~E~3W^HxV|2DVgXBjCel<@FN z6p}R^-@Yw$CRkWn;uIpbkv*?S*bRoe62+t`aW1fh-Q&`$dU1b)g&tS$)P4s%ge(@P-2sD1;Yy0!gGs=h_JA*m{y@m35$y22$PbL1wMX-O2I%v zb4kD4{>f;Oxvd>vy+%K_cRcLY+qc&k7@qGd+p3^gSy?eNGhZM5_>ozrp6O0nWo2b? zF-eE@c$triq;F+Kh2Jx?iy_gGk>h1{45L5BO7ZvpijI!Hb?a7HR(iT=do-uGxcG*L zXPuYf!4=;(Z{FnPaXV7o`aYh~Gc**KCY=n6O~&!ac4d$=#pU(u*ZHoSk1-Fwe0g@= zk2Y59?^tGDPnFEmF3qi}(TjWvAy?)0^ZO^;CzwV4{%D~_`VZ99Q1iSVCd(#oZZGw* zv$6R#SDR&~rlw|OTp=X1o32CC`H-2(dJ{V~F0SWGM+ffV(NlNW(0C!c$68vW`?AEV ztE+eK-VF^6<>cffKvsL~eSH5OVxq2R`mmy+;@!J<$R~Xt6?`S@-(zE81&4$fwhDE3 zcbB=WCk6!tZJ@eWNn&r>so%fP!OCi0h5RF%PjVzzmX-zv1}=qpTReSgkox)a=O<5| z6c-m0AisY%z+4gJCckp!OR@|KrzeT)!(=aB^n&Cm4!)WWN!zH)^?^BYKzYAvI=O;nQfyBwkz{DV76SN-3JP0_Qtnu=I z@UHVUDqDi`JN;2iOzgRlr0;Y$L_8WATAd9lk1UjsZXFK~4<{$5?I)(Di>s^D5Q^XY z=dV^euS$_2d)nHtC0T78e6Cm5m{EF)%P%_#Tgrj$-LiQc_lW>{;_Y zOOVcl+>eWm?WUun`!X^@o)vQc`}WJ1FR$t<-n%CqFm+&f^^ek+=IG=^-xbZ1DDH*D z;HQf9G^nejM4(5L5Ne360w%lKL$#ODzRzYjYOt0s>E6CS-CB?#=PCOfYR{&5@&hC` z58=T`p=n*ARlM(u`Y1w-^o^GmDRrrZEhUA8s*f_q#RKw%Pzh~G*uwGg{Lsx_Uckat zX5|YQM6k%C%HgGQ{n-ohF#d^cDA9kn3nuPFhHa2V_Kv-DP z8RS9@P;_;5tt>4Kdh5m>*m}-1T^$vBI#l^1f>sES#~>_8M{A2<#q^eYHITH zL-oYN!y_dnjgE;)`&?e`c5=9bfk!3zWo>OOIhoGE;U`3Z;0_hEQ-Dn8E%DIl^OJT` zT-=bsLG^FaUS49TH+gtew!gR<8#7+NuI$=cl9EE-iN)z(D0Jn@6-i1e99-OKHp4f;rlGVA^H)LEoWp=aZY&@l~gflZU8$Se#asFH!U-It3K~g=`7knF~ z81s#$w{PF}_VzY6XQjZ*=Dx0ylamASWkx_r85bR`uC6{fJUlFM+uq*Z(^Hg6o5o`-;n3cO2OO^sOeO0MX}Wvbf4c8TfgXLlYI z9Uj2KNQYrqT`z_2e7gv3L9px%_1_sugtcv9;_CXywP$Z-K*x;lV`+BwN=N7Qf4|&S z*TsWh>nCCGKh2c?fH3Fi3P8`bztPbD92^0D`UmO&{P|<6{{&Y5EBy4IknI<=`#S>v zbk| z^z5Hog%`CnG)f_!ULK1Qlakh)ogV#z>LzJ?7ATNOB)R|ElRt0Z{{?UUvn=PI{bNc0 zgp2?G);do%Io#OU*%<=Q+S(ebFalAF?EmsbQdZXA)>-t#C^9;_q_i{$eRzC4{>_`8 z3DFqkwV zji&eRVY@G_tcdD_n3IFsoQmF(m7JWMm4#Kky}Nr=P^Wiv^i{NG z-gWP{|5{}yA84R2009vb2m@_!{khPQo14qZ%BpNEQ~)WIkT8)V9auww@uRS$1X5^! zcXdo!T3Rap`1shUIdu9{W#v&@6gz-vf>p?8-^Pz0KLRFWzjp2WUpK_WFWgs=mabo$ zs07sKyfzMv{Usf8sHvGkF~ zzTn~RE?_lUGu3NUT+9lM8ib_LPVqyai=YK~CFx7TK8 z88|sZGBL5S>xY(?mp28J{@(Ao1X3imi1R1iN&yiOO7YR*;YaG~=t@!H;jA1SeoWaJ z8Iq6M+uJSlQAYgxVe9VRxua9%svTF1Pcr}!#A1JvvKYy!ox*7T!vI4*btT!{W zNl&Wm+quq8MPux-+ljs%^^SQuk8yHzBshb*+d`_#!pscFwWY1? zD)}S+rW%8bPqjOS5{_;ekldNco?D`7Ir8{nQd3tq=hG*gBOGjOY{Vr#G72moz{|AU-B)vAO}q}bU%kRIY?X0w5v0-lnUkpXUEJz6ABV1_D}1+h^khD0LsOG==n5Qv9_gfwFpnQ;J(B_bla zclYjPG5s_8f9g!t3JUH^Kd-uHDPgZwGIElVXf4aum|knV-BcnrBcP_{WM#diE#|qu z5zZh7d>Lq#GhoXef5-zNb{3YvrOTYEYfDSzUPsQVszl6}wo_|xYihC%&XZOqby1YH#s6#&P6SDJ9b6_k|ZvXFR)X##y+-HA<9A@(BfIXq%`exvqqkt|Apq}=JfaP=N1=T938({0|ionBqgV&_Hv+7 zHbn9_Y}ff?r>X+*e61Ty=8j7yhf0<9L`_Z2$|}d~0ye8K&eF(Xd)?dL^Be8FTs_Kn3%$7B?5?MA1NuV18sqe z92^_~>C0aR9fI!~7gr6iyrd+3=u=8cT|H@X+`1KhcRl%ecs94TmXXM7?Cc;(JbQ#K zq4)mZqROELcXzr{lKR7k{sdvc!GPbP-b)9iv~%lKU7?`~&=V0B2FS)QC|LgzLOiAA z;>C+~=)JJtAWFP8D)K{Md$_o|yR%)p_NB_-golR*6a)x+iFpjfKYa-#p0i%GQ2^V} z-!20hC7~cd!cB+&sCy8sIJy!?0#*~!u72zH5EmGgP9rNc>oU4<;$d; zT1Do4AN%{&T6p3q5gCe#ikq98Ku!Rx11J3A2};Z8*chxd7EgOy+w~@30fIXgQvZT; z$RT-m_X>nMiQ_gmdOK59mW(LX}ybbEtVUJ+qKOhRA*$F#=nWNV>QiYcn zN_Nrd=_yoN5O!YiB#OGB2-DPVqevYCA%b2AbT>^C8n}&39uk)oC}svg2@qq5^~&wH z=Hx9bR_5l6WxQ1{{z z_?w6)OHTUqJO3l_ZzoSZ{&&9o2}eOe`~@=+mrz15AvB;1fP@tq5~6VTE+`?Sb9IE+r2F3b6q!32*SD^&u$M2(fnL>vDGtQ$1XC_lu4 z_Ix%oqxH9#NKcGhr)OZ0R#p88YuDP^+Kq$)cDTRaKQKUVudECTUSwQcTuYeYhn1l` zvd$UyAA>C|cOa9XO%{bqIXmxu(kx77h~d@`7`-hk3#HIMFc6p_#180oS2=Kmn3(2| z4}JnThTwut2@enFHSP3c3c3t?{6Iq^Fb^Lef3|OY+;g@m1Q@PgUeZk`tY-MUrl#h^ z#KiS=c8%QR2YPxmBLMpd^+9+2XH7|cKqfev3F0oXp1?};|E{i~u{byP<*pqTGp@XU z#OTP#rLYe`g@Ns*ln4dz%(AJf_CXz1)2nn=a3w^aIi5N@LeLj)T*l-%;NreC%}wF0+KhjeDo=MNubL0uUh76RF@ z2exkWC1c8vfPjD$ck=u9#;p-d7cN}5L!vJ`h7Vc!cO9!$^jTY8_B}e67Hh5$6GOPo zudj29UV{DpIa%!i+815hb@OXBkXLlwAYelKN`USd#jXK>^SsHp?=08VNzE}}7FAb! zL1%_W>FMPKvJ4@ubMMqt7$)FK7RC54tbR@|E?u3Sq?ha`*vLl2tXyyt6HG` zl-;qnzoE7dc!5A46#^4+ykC&7R}Q)u=vW}Zg6=UIq9Tb4dWp!*o2WjWZEacJ@;W-> z!Q{MtEZwhOy+TJvza#A$jv+I=R#=Ww`Q; zcS)FIZR?-&TPZ1D5Jw~>ziiGn^+Dy_CoK4U9y^|E=l zm)z}nMUK(?0*ls9TX2sn4sRor>XY7=}^7GpGe#_gt#^AE7 ztgNzL3ihxqbWKM`zHjI0eO-%>^6c?qO^vvYwvNtuCCtdk*kt2CF$B!neGh~K=zve3 zvfZ?T$lic8)M2t5q<^M2Gdt^FJY+iq!Y05G9P*?xDzdtz=Jin@)bDFiQBhNb013~b z!RZMf3AK5P?V8sQah-X7oz;#lc251pn-5vW)WvM`5X9pnk)opA2Qe<*7=~EerC)>s z0|L;SzxZ5RXt^@0jThpBzVGQ27BtO6fNSU7^4N}Z6sJa}|2~v0TR%M++zk@3R}r4Y zd$et@e(~_vf(+gG2}kXsXUE12?Rlj|^THZTW06y(S$B#u<@XL$TQ}G8e#!-GBDg1Y zLq47>0_V4Rt(!xFt6M7bjRJ zNS;1@3VOL;bC=ZP?;c4{v9U$H&6KZJR94bTcpu|)hW(ugz%^*N;i7k)vb(UY^D24A zr#A7#UwLbwQdk-Ibl(Ep4C>uPrHjpE)d6(Bb4WJ=x+Bb?++AIP%3+|R118bZ)=mP9 z?CN$G7&1WksHlOTi(Rr8pu6JW-~a>c@9hQtx(rCu%uMK%`WiIjGOP>!Puk30a@`aW zf%v7Ops28!64zOqn?Uq6W;rJ!Jw1&ilg3i&m{ z+RjE1D@m;{x!Hkz)~2iE5P(rRmFeFDYY7Y>rIE?W#Hgqbe79X)%aY5XX60+^T;=B0he@Bdb~f-udU`@C zDvo2+{Pc8;QP9!F5U0mb?e1Knuo?OM0rVxvVbHtG&CP>IIejxF+_y3H4Krpnq?((W z&bb{u{RKM%fry9*Gk{u`l5TG;b|y$~5m-=VXJ@CW(ys!afB*iybReE#ehZN`H4)Lp z?7{ACg2_0{|R82VLhXiaqkjkA@evEFeU(x^D`J8!q09 z;cArhZd0WGEsC8_8xeBJ{tw>G=puyL~d+s>`ftRGcxY< zU%^N)`jP^`7N|7{h=Qk=%~t#_5z$C^$& zo$(?+#>G+ATK?^8}rSAs}PQWAb~wcTvMn>T%+ zUqCnMj2AAn9Jv8{a9LSdTie}h=4xt@&}e{yd-6fSMZpcq$;fyl5fBvA4x9ZZBBBYK z54AKUHMm^P{bariYfCPqQoTi+UM6`{c;(oC(5ZR=a#) zLZm2!Iu_~))lKY8ko-UeV&zbGhmP@zO`^YBPZ5p*_jP z#SdK_u=uoI8>yvY8}2l=v|v(Tuiw7q((S0?(rwWX11T??EhKfF)srVtNl9VR5X>*0 z7lmKB;g=Uw@%7ASCQ~MSdgk(*%?R z?*Z4a(rwb_sB5R4Gl8XfomD1V3YkX|mOlKGn}qJU3y(F26byW(#O1x_JlXEd{}3M- z)M4?IJ(=KjPx0O=Ra!~CF&}YD>E)ONOCHlE=}~D5bK=mt+c86?I*WF?uE5hSmxGs| z@em!Mdiw4|sd?X|)ik4wy}C6N-!i3$R+&6_XGTK!Db~C3$|*&a{6d5Ehe4|JS|Ycy zHumXHG&B3dTvKbzcGhRH6?`K5F5~9Z+$&FBs3urC2wPemk}&k2rJ;yvUCzm-aTS>J z53a7-IM6N`pw@rCbx+21BR(>6+r5LtI^o?r%?A&*`bvF&H`lo-=I)dEM4IS(OfjC$ zUo_Sf{jB%7(4H){Zp{#Vw*F+(?$Bh~pcnPVBhn#tXiwakHe=&{$c>vd+Dor%kKcJ^ z5vXF?IYeKK-@l*FCZ3=`)|I#`Zs{~PFOAH-s$mqO8T+d5=VOKGG`n<1wI|F+!(2g_ z>OMD2ba=dXTgvxC4-PIqWqGa^!A;>zTsAj8Zp))~VqRjT@L9pQCAK|~zs7q`E!@m& zGg<2^Codz1=;4k__-nmWVaLMs2HkC{(wQ2P+(xm#usn-|+j$nsN>UC#@!PBBQulmP zDy*-*P(;!y9bWlryoPn>PPv(@ed*=Vw9g!0FJpNaWXF%UUCK3iJR&%_7beVCA%bM# zF5l+WXW1Nnb90YjM`gL!ay@b2r<2w@?~(l`m7GQgi2!Lqli?WVM|vlBXXmWv%Ty~k zM?x896Pu@QX66(r==+kJ zIifTfeux({u0eilreeby6`bKXbdaiv<~E+apCsAF9wN>?~e2wUCCD`6M9!mT5;!tn4Nt1Uk=A#^JS zb`@u+vsdkHw0C0$PyHH_t8q2=GMCBYnIMUg6@~;dqo)={Uq%hQW%8HRo1=J{YfEoC zV_8=INPIGF^>Y-X57T=wfI4cj>O{wo7G^_04r@T%b3S) zO!XfFcb#uz-{&(i)#0C9V-q$##ISW4zhcI!(Cjx%$sMULe4&&!!9O>3#GAPAtC~k> zYQGHo$w4P2DWX{VLP-cEI#Z@RWzhm1^T#I%%fTLtgN6|gjhT(^swlstytEypRvh(l zc9bMYyg*R%#X*C_6n-AkHlt(9Ca0oz=;0BxcSBi>U%UfI=+*+7`}@(O;|s;)s*}8f zZ@r#|eJ<^|@qz($&-?vp&YPP-3!XSOleMygQ?gVP7}X+k+zSFl0Sw3UG&dYF|=r0IK4pnBD-6q*X?=LBh_pC zG8>A1TyqcI4;Eh!{iVeE$a3^aT6LVU_}T0kMMj6rb6uJUtOz390aGcSJns)Z4}a`F z=J;7@x=lcI6I+gq)TAn7Gaxi>rPIIsaF{bmU}wnjJyRAG>`t+2pjX*Jvk)M#4 zz{zZNKEK!uol>u0gGgl7pTKO~KBHAZmYZN)wopPU-J|LQ{fQlmQ$^5(UuLdSX;4_V z^g;_N;r7tCFWt7m2wzRrDeMt8N(U*AgE}RhZ>0J+i_{;tniAOI2;Y!d-3T2J41F5A z{O#S%X_tkHp8{if&-^jmAxH#wf1e)D2;z;a=~ak?ZG{D~qbfghrZ>*M3W!=JVB$CJ#J z^MS{6jSJpYkW74P(uNeq6rC8V9HFJKYJ;z>1s_eLmFkrv`^sGZnC`W1(C#9vXhUU$!nDtf5#>#Tfq_x#?UGReJF z+!B&?JLb+^Tk$IHw3W2@Y7qoS9G)>(tuHn-icidxecsOfkT-kRtRlB9=B<1>ZqO_1 zIBe3;Z#?*8I*iHt;w9~3Ink*iK~6tklZa|RT{10TDGEco#+CSUv;4&3`w3nX_7e?H z!(?$)qaC$zS!21psnzPItyL1&a$@}u#K+R(m_wiB3(VckrnbAMHqKvOPH*-WtJF5Z{`P~4k??&re*V25pM)WIUKHQLOUonIl7r6 zGn#cnl>m1WDj8;V{SlOyw983X7}$DTSC=!PsPoVx@#+hB9{pwIzqB^7J(O|_K}hZ} zZ&HIQuSFS*?l_r$MVH}pIR4xC0-Mh!)~ZXH{M!(wXYqFQw?pzzxh_Kje_;Z3j0Yd- z5>xp;XBwrbK>~NiKnRczbVLd)Rg(A2S07^DTp+)z8)eC>ymqj64^hstCboP-aIYlh zG;699B-r0m?kYQ!XcVG^ z;pIVJGb6cAjOW)hi@mIb#u}vPmCiJiMy8UUR{i4h<$0oGu?tEy98)Izxpoa^7o#@C zun7GK6TF2}?Rrj`9p3Yuu4LmjX?QPk;5FedipUCy2xLzUVhSUa;;XY*niWQ;WN5NQ z$v%H%8rw`iuBAwv7akJZUh-1^)6sdbh(__Du)BCQoVPu%cup9<=J*2B-`>%434d;i zzMEtH&Tp&eKjcmG+#PX#6>xr{#rmI*@qa(Qa`La2&T-_qxI}D5`pXOY+olS?BTKW@ zSjLE(wYSBc7u9}wPLcYrC2K+Tnqv~8NxB{?SA*Y^nbiBJf1&XTf`zDvIO0H&VphqE zWns_cY8G+D(Gsq=Bj)+s0_|{0Czj@B|NCJq8>08W?5)H|{a2DxHC|(5V;-RKgX(cd zQ*{WJAk%srmCjKcm4Xa$?zN?p{I$UK*@ac25%x6SquyvQWQW&rqHQ&Cjs98y#43) zwy5{9D-Zx(UFs&i2M>Iw>BM_)XMroKMg5A6rsg}dUoHZ8YE5+ZYG51s{jLsdzicYo z+;@_v(<}x*`B;1eqoj#RU3zs@6)Rg|ihsB`U0{zDf6*){g z`pA52KFwjd)y`e(w+?%zK*ETP_{r>BQO zzD9z$7ex#oj7_S-pMh`$l3zG_giHVu<>f73o+1!6SlQXny0b0F{n{E(RbVIvcNYv+({Us8>!=MnWw!K; z;!I3v`bprKmOcxGoo^ZA>~7p7b-B8YHS^6+mKf3Ic&E~G{H>vc+Wd}Evmfr_mZT2SMA6-G?%YCcsa?5Gdj}U|&Xpid8SZva;g;wsSarJNWazy1LA)(~tX4AAo3gv7XfK65|HH}F(f zb^x-xUxCcr#XleHRyCsFn(vnAG1Kq9OKiirwl3)p1cyXyfLwX@P{ONMrtY0p8?=AQ`B{;Y6aebP zEN6n@tc_r9Fp?d2ls$tRaUDfOr*Q7 zk5^<%J+L6 zOgr~2jE(0aSykz#W_3wN*}wtQV9SY0aq50{>h;1Kh74MDjhk}0DJcc!eazt*d+UYA z&_pe9NL{r!bL0y$6-zGd#NcmhJ+Z0kmT}mf;9a4(@gtbuB(qg<(@Bfm=}z@(_#3)z zAem&Hr)I?sejB>4$nY{!I#_5=BrKkzy_kwvbp`4ypL8~8cXCz8x=%WA?Xlo3?UWxm z!iqre@7#79c>f+79d~J$U|afh>qwxzOq;9cLA*DM)HxnP>5Sh>1cbx*@dMvUHZ|7H zlJoLb-|K_t*QDR_`Obby=#g>>VnRMa&c_u2-^}&=aN&9LqX%g?;gc8ig;V0k({OH` zy<8tZXWgwL^xXtHcZ@-Kb#mgT>i_l)Clvc)0K0TDOt62MgL+}&^-Kuc2Lu{h*qfhM zl;v*;He>T?fn)jBBRE8mk`EDpB4R8u)cX1j9t z7EZoOT7CSs#SJ7Mw2oIennt-IWOvyrkAncD*Sad7G^upB@rwI(HYN2uH^jz|IBFMj zcDti+_(QNvpxYq|@Mwc0KheEv<6-S(y!zY6BYl$+)f)2y#pKt`kvrqu5*sdrh}hYF zjRm_Mc3j|S0EEIPG1gD^KtkwgE+XwZ7?v?|o^d<*i7`kQS^U+LPrBs#DKbq9@N^vS z$wlh5qI3jXFlO}+P*n$OnjZWX^U7+8Aajc-%Zosdn(L+|=#8z>sU(eEEAt)k_*k34%9 zYsIR(wr{Pcj?hZGHD}wdUewOub=++;#Fg1~AU(6+^3WT+N7=h-*af`$)q^+cK}w#k z19K0xSFhRblxPOz8Y!9KtGSBTSQI$#sO<})YNuN|kz9V-*Ic@m`NfN9bN{Y|X44XV3JN15Bg(h;AFHSkR+WG=gFvpu!p+T1QL!_v zb{>Ruf<72tY^EotATehf=b@(&Jdycg2mf4L_V(SrXgH$IlRTXHD?nFO^*NQF@@s>B zOk6*vo{WY}7;V6$BwiM!uE_!1`Lc6!n+o!U-hcjl`Er=i+_Q@*7piD?>W~De zt37|=g?Rdf$oYA`rtA%HI73R%o^zKHEJryXY^pu?G82y)?1s%X@@hKTrRO#F8~5&9 z=R2IEc@o+3nmYJwcX=AQsWNDVqoe|v-=^RFza)-g(^W${F5DrB%!d5 zP_jC)>EPa0V1RU#Uf5#+gQ^g6f+y>1k4r(pjJA|Z3FZps`srF2=kqW8`v!pqPL??< z5@ZTnUMI>>;$;`ZOEJa=u16pO* zmYlZpTqRwd%kBz+x7RhTkzgg6XIv0cSP3O$WsW4}xQMB)oo+&^3WrZ}(^?I*EB(|< z^c+$t=t&byYb~UP;on+@l`h&z(0y{c-JVBF@6fzxWJG9wbFQq?xM;+{pT%W5`F9Wf zbsh|*qt5I1S+VUrx#S}-*^HOJiHv0YQ5GywwaQ#33?9x&&PwfD>K`fH6oa4Dl*6uEB-FrkF?Q5FOF=g0c8fTG9%AUK{(xrv`t{jf1tQuJD^py5@v$8Dt$t(%_|yWfSKT+M#qz zUY$n^BVA3FcaeMm|@8SU?4BQM-C<}|7^(I7~GIPy`S=O$^gnN5{hWp;xA z*(K0k#&nnx-;Z83P#e~2-U*^oxUHm#?BMTv=fUd8DSBh6$|tYyF8>It5Oxo3ZMnVE z=a7OX4g{HON(|cXD2_)UWcrFGfv8Rxl{-^Tb{f5lj0r6zH;}gVAq~&lZ}-R*gWxQEGC!1?80}`YjS+;*7Vb;oACiBuTixg3 z7?bKVdS8&2M}o!9W7Z!t(big5>E%(D`1kCD_xF;Y_)R}=k z1RC=*g+t*W%AADa9_Nuvx~PonP8oo@+Cae38*EQbe7U$oD?`3I#&}V9awfm7-rn4K z`><|oq2xApN(2u}5q(3n`%gcsrCsYtyqz~;;vfCs{g&c>&$RiX`IPJPM zPS@T!Dd4YelH3%T?wfkM^39(o$om~ep=g1&ThrWLQB=+NJU!a<$5vznS<~NoN72|A{dFZdofS1TODPxm z4q^0C=Pi7&$7K>^czmu{q)47&g1KEqRn@}Ux=B>hpS~MRG^y3oU_=qHndG3S?-?F8 z13wB2%U%LG<@wQ`mOU^zgPDnbyxUA(UjA0{*^QE^abS926ZHOlxMxn6z)XmzXkgIR zXDh4q^*z_PzJNvx<8%)FeL11j;;+X{KTng>jz1JMS!RAn0+G;HohyDP{PXtvdx8D4 zic424zCDPHbrhGuy<6eh*?e#DYlCdyBsJvfq@y zl|JY3M}o%{HiaCh6Xx|-8A3EQ@GK%5v?beWDa@1Mheyy7;FMufXg(%GL;wD3j9-7OP*R$QN= z;Iz;gyIZsUxr>LC@vlpRYHyiYcgg2mbB@)5dFCU&f|d>Xd(Kvte3?AvR=9Yz23k$% zcZb^J?~k7K&evNkcI*@ET~-O~pKoAE7TNYK?^TaYM6_GZMd1o|vtGVFka*E6a9e{} zE2RKNPVk+N*zYL%@r2KnUOsFeF0s4n(^T}mFe~e>w)R*)9&rqjYue<==pAv;9%s?zb7rhvw6^Z-f+&VLgP82FJ0C z#(euvjDEVlLkknps;4;9+I)+%oOtXy_sOI`sdAC8jWP0?9v)$M%E|6->SB{DwX&0* z@_438Z7{#%7C+E;m?(cYq%a3+XJzTlT)&A?a?UE+Onlk|iJxg6;fKn4!#QCSTL3e& z_04a_aU}{&yNFCMwXbVimFQpiI*O46!9tLI(6dH4-&M3_)A7gv9N1&pkO8x1h4y}}x0h^$)OBroh+(aYLWfk(DK-|qTfdXMP)AxN0Y+Q@E5Ot+*WkL&P+Z z`IA{2wDez?0}3>)h_mI+@&MQqU+Z35uW(vHK|z57xye;v3;-)9P>rRfrPoV9>O8@} zx4E|`0s9N51a`q6074s}Sg=%>xtN%k0Ja5>BG_)|2J1>XY%MNbdI|OpTCdr2Pvq7h z_BKE&GUZE5w#qsD&6iTkJ7lR6$*MHbvxatC4qTsI z4koG*Ql8dKGASLB8`VO1XXM34*A*=hAzWaH*>tH9XX7>22wM;~JpyDZ%+YINMr|H2 z9S4{b!nHBX0h8D9o&qv`I3Mt0qq7&aok4`e*YiCg(Ov4#vH~#aZXi7La6NjOQBvJ1 z)zBKmPvSkgpK8Jo+}G~T`dRil%)o44Q#^_EZX6ynA&)%SW!3HXc*Ny%#H`ybQJ{U` zntVBuY?d>a*CXO-5bCaF;4#AS@TM%syB`;EGyfGPM@7jQ&tQ< zko9Y-*w19rk?hYK_-i;sGf-`B>IStjTHGnm-{g%a2?>{8nkZOP)?e~R>Eo<6Wo27E z;PSnTWuuaLx5PVk_EFr^Ami9Vs_r4lau)ksiWQ0wyJyO2wXF9`DZPB|rIHZSVg=Dr zqj3$i3bYU7*(E>G7E)X5~5cE|i) z)gmrf(X>a()1J8j#oJG1MQ+n02MTFW8cofdHvD_!JM znEGJ*Zk-0UR*A82|A!F)u_8I%o?@vbZdxxQLy)%?IGRIiy-ucwC!IqvPZgGs7hT60 z0WP05gd`nYCHXz2IQPwCZFgzY6{qiIy*2AB;nhMKmN&FN)qIw&w&3)p<_7+wbtr|v z{mb74xoL9q_a&^Dmoq}mbc)!aN4NnPjW|uu=9#2njOV{j#GUu-bJ5JZgK4wsv=;o3(y1?D?@Q!DR!MKUdT}g6dKspO9-7=cPPd=q z3yg4*dEa>9*+2hrp~>>fVR6pPpVYR>3PntL+O0+Z^ryt>y2%jgQsF45ZPLWpmejiM8Z8y43BEtT2k*z*dHmrg!;#K@k+_`|H7@r{C`Tn_wXV}&Xe zpc#3lr$(K6?&sb~26{0{!CBND7x*mALtxe+$nt3bgViWCKudltwyFWT1^<$THqE3icB0UOxQ=Mz7s~nE=6uF!3Xd=K@{kAZkzPiX2N>Ort~-e5&VP^rdrVTA6biP#l2$1xqup#9X453-e=Q4@vVs!SrtEU+}!+3ZQIy6G*C zj~NF@(*;Jj1s^22f$|+~*&MDvhVthvwP;`K&ik+(S^rjlUC{w(^rq=HR5&Nn z3h(tbk8>{ZcMAUUcH2Ds|1=f-uTxSp(tnRl|D3R%pLVL^ML9_|nUySGRQ@ouzZVg# z=25G45ag~gBk8}`bx2VM3exFTk01wLt>}4dx5804FwcVc!2KpE5YE${E(q41o725j zP$nM!aGuQ=q_J57O8GU;7~p|;xm-Ci)J^}LoN0;$IEIWa+&vO#{V{dbgF~5hv&!3} zradUuCgC$@*Ma@;=xff;fK~x7U9488uLIl%80QMb8>w(|4Mr_1=SPHI_p`HG6dc{G zzXbqjwn^w;;<6)R^myNNq;KoE-t5BJxvKx%2i5g1^_G5uqaLlaBy%yoCD2mBWtK42 zTjIUp^cGnWpFkat{@p1SKK=FsSB@!hw^k)Zaqu&q?cZ>!{ts*P2e4ngs59QsHOuK*}lbZ)x?!wVN+Uv$QsQ1yi8gtU2%V+ z@%i%Wp}?X0-)qe9<<_s+lIMjrx_sjfGzSg{R-9)zUH+d4qel6(!*{$BR*FWv{b2&} zdIS0r<8wkW`C^4?V>8pD?ooqNy`&zGi{Q)Cu1ACXpU!&`V1jMpZJf$e$@kU%xPGD1 zbSIkoSVqYSJ)(iL0j4^P!Cbb5cK?I1w}7g8?Y@OK-HmigD@dmZD2g-)(jgs^(jeUi zh|-M`f}o^;bSg+~Dd`fCQc^nZ+Ir6WkNb^rzcbD_!?WG-t7koH%{Av-{xw>_hNSU@ z{dDHjFWA5YEldC(n-&uve73)=c2D}pH;84g_qh2}4qw84$t~nqQrt!4 zCjI@%Y+X9PeZLziUoXv@?aZ`_AU-4Xuq20R}5@@>Yhtyzt?DA5aq@bzW*>_4QGt%QK6o5On{xnT0WI{yWqIzVa zh5TS+Flcgz(AFp;2QCwmhrZ6#c}`v^|?F*_$&dKIZqD&yV)cuh~vf zEA1DcC*)k3!1-iMmBc+_{{gX>0ZD#f8|GiVv3sM)8>Hvz92`VFL+JiET<;7 zbv|CHl0G5bxW+N+uJvHd*iCC-g-!PTiFAYXN9i3+a+!^g9mSU2h1S3mQ^8!%^noBO zOxdPuw1cPXr$M_CV$tL;HZ6*eZ>d)6RnKJKK?{G@Niz%y(=~`Q?Jii9-}CwueUU4hyPrh>Q2}AbmKcoV&(nRCO3t((d^U9 z@2wj{WaD<9$IJ%pwaOCxJghen%d~HMrPTavD`PkLS`TY5l^j71htdo7qB{twAM+(L zCT9ftyWWhM7X)^8-(AL*e@SaX~nJsX#_sDNFWgv1taKKA9z0BcT zu?4l^Z2L@AZNMkpX35H8amt=M+EGE~;|Q#Cz7Z?lGM zmX-%3*iuQHm+`fjXD7aWNi2_#41~X&EL#d^WgahGL7z5su0N`MI@M6sKJDJ#^NK3m zgn6T(bfbvz8lUX&&{W9Mu=>NJUdDZ_9~0#jLr?Ey|32LNX=&t#NiZK{kKVWIQ>_?2|dGg z>f*+im2;1?quH;04p;MjY#??iCjB(C^Rt9<#I~51y=E!)%+^izQUuPe<#y5#`-aEL&4AUPiMnv$qfunB{4_ zTv1y3sr>q{D4dMR=_oh1FdNxAss7V0L@<3CVWZTqiuPavoTKM4xx6er8oh6ol1Y9) zYZ5II+IvmcRdmmva^ql51?MA*W2NjTJd`?+}P505pohjE4;-}s^B zRPkJKO~U2rkD%;}eYSOLt$|+|aXv8lJdiz3`^J(Y^S!mI_%_b-q4Nbh{3vJncTe-H zX%w*s`(l2+h(1JnmwQjdHmLat&e83ul+BKyf=^qmHET?f)pu(}ESRdN-08BIg`!Nv z)d~9pC7zhp>gip^_r5C4GV}1WMhz;S{PP&+eI-K52`}59&VR7oT8bc-RYpq-T5xh6 z2t3h!Ol^I(bxOY!aUrNH&TRak{m$q{nom+$?a|h&$bsV~iN&q#t*ms_CPq0= zxTi&?8f4!sk?<25oaENrmp-$Mc{2^m*P@W zyRY=0t&cxbV7RpNLl@7et-W3H+dz@P`%9NDy?Rso_VTS4sGr4zqbfQ?6Vb=(1stU{ z0=l>aRJ;-(id%mO3U{z&Aq9d~YD?8ijT91gEGa$#e9bEc6@Zj=Xr+y>&Ci?L+9t=x zYa1H+9q;`FhTC&K5s_gAeN)pckdxlIv-RT!d!W|9$%%hiiX~tQ;!YDDi(O=TdU{}u z1)d)$wX`$`J#8$rY?JSf4GvD!H%|)}IXT2gZZ#;b0R72V1~W0>*Sxe!LW9P1)i16$ z9;?GeU_t@^q^G&E2uQCVct3zJOs81)^tk!_>fAAq^+X*;%fz7t38&B^!n+KV-UT6*UwL?-g%~BWqJ9a6^A*_*?IOxOWYN1 zyx(qjf$o!WdZ|=L{a{NuYlR; z9WE-21QOewhd@CD&myzSIW8mqL` zxRjlg^zlPkD5op_rS=zyGMl27?7QR`Xx?IlA4H8RXL9DjC&>Ec4N(sw<^Q@gB*9*u zo>;DE%C)rUd)wRiL_`sq;FLomM?nuNZ6HZZH9izq6F?B*hXq&#XQ7RPyb&mmh}^I5 zOpZ}brocWsczCe+N0^n2h7`2?_>@L4G?zui?3$?P2=Wb@9l89C8(qF2+y}0A{mr;RnRPxAECVE(_RAYwvFa<8IPVvyx#$= zccrXV#b!ybLy5!%-l5rP6}|xWjkV#Mgw#~~=`idZB`Z%shf&@%c?x0%34sR-H+9(n z2iETh4VNmL27DO$0O^w>Nr{Nq2Ya4CxHPUM9r|Av1KS9UB2oU3iv69PSbk#OMwuBI zGXeW1KH?yrg+~VZMdgU9s3;u8`{1tQkCDkx3ktnu(kT%Hkx0;HD-M`GfQJIh2B^Xb z<>+y^KhfnV_4f3Ddnl6xDC6(TxgTzS7g$zd&d#X<1s+DpW!;kC6-i5FWqjL^YDpL1 zn}J3b?H*eOrC<6Yw=IHc zM};vPbbjqJ=}t*>cJ}PodqFX>py0{_-RU6yrM>>E8;H?Gdms5s&uE0u zyefQNAYd{UMx=vd65j1IOgRvCdM_i*k9JoU^E_Mj;o1RBk=Hv1D5G^=KkF<%5i>J0 zKmIvgzh7qF4w7K*OHxwcFhKumy3&@Uj9ACk*7jbL{~YA$R-MEwV9EFXTLo zyy&i#Va0z2vO%pjm%tr*3?!6p!!d5I5|Wa%(YBs~e!F&~BukWartC`O5SmeWXxdN| zE^7nCKboq7f~EQPsOsu!Xf%Lk8jA{#T%Y18_7x?OXGUO3MWzsbQ}VO)a6RfVh-(K^ z6jX?Tpb5k%U_emRMd(5(m(y>deB}uqF6pos5W2xMY5o^Pnw2xK0`oEZflKTMj!B;v zJ;3s7ZUZ`u5@cpEKS?EjPr+jYwqJ$hW&POo z+Q%&5nehvDCB$21v?_>_)4isaIPi)HNBigX|CD9_``7>Lkp4g4{6EfdnG+Wov?CrQY(JO>G0){W-EA=4Jid3n7WD7e z7^2I)e*GG_;m(jq)aNNs5H^A*GdMk)T>5))`k4L{B&dV>_mJV@{XKTLqAGvi^`Fc0 zBF}~x3HTNm4L}0%xxat!_wTpw-u3tNJlvdXg`FsHAr5HRVD;&@0{by=pO<1BPdGKe zMY7#dRyzSa>=K|COizCd7MS1)wdUP2GCb@$1E~*~5s8%;U9Scwfsch+-&TAxJy`#_ zhJo#>*d8{cc$8l5?nbq)FN=#WBUjefjSUR&6!zD?K9lAQ0)i>KvU+0{m;09#-aZvRf#iej^`Vp2W=w9E>cyZmq8_?1ZKKbN8eVtA z$@|LZ&Y*d^Oue=P=>wcEUe3MvxrW_c57He_{s#-ufJ;e7wtOU^{m7dU*e-w`3#P8Y zC{KYCl$AO?%QrC8o0^`!t`kRT)7R63v9tuHUH69zPo&DJyTaODj-d?Jvjoi1*KxVC z!8aM4e((8j-2%0GYb$6C+)Y3KXmI_h5r6rBhMxW*Nb~W3a2GPx_J01n3cl2kEh*)l zt%DIASTk^a2R931Mkx>2Y;!7~JtO)3vJFBwbcWaS3$nt^GJlFvPrRL-5jX5z4WMtOy?h3{sqokV^SX-Oad4yO1-I+)Z z<2h#-C{-YB&KV0jhzf$FjLz*N-0;vP?3})zGeB84uJd?=&j5XKFk{GJS3MlWmq$5O zP1bo{z7WX^dsE??57bSd6yQ#cZ%Y$-Jd5Pw!it^s{lQo^k|t`t5E_cUBmJmI3qC_6 zXY>a1b=sx*XW%GEtgo!7riR+=f$GBQ>1eJb;4~q^lHx%)Zo*Z_NFu2MR=5!;-?3zb z-Ky}zJ;!PTK4g-H?5PMyG>m@@7g2DeT{hOcQnZ_>7q(Xh=$3+-*^DRM_YD z%)<@1SGXkqL`8VCib1$?`x$x=+*q7zAz<1*23dVu0@$*#H-d2@-P!NbAfjH{U~H$C zbu}}71W&=+nJz)n`ZHy4Vg_~&*nL0}G4{^ns%xCKTwL?fNGYLQdb}Zq$U;kacsRtM zEO@|xDh%aKh>YZF{sr)YGXPDA0Ax0?kd0$vVg?@r*(q?Sw|dk zUeoE`Bsl&h;SZ0FwvD#D`~K2)6pt$S?B%a{ieUPq*9>Ni%n-dIFsKY8!VJW~@o{)6 z)h6akff_wBNgLyn2|E0JIhemeCD45s>Mn32N3$43K&k=`E+RNjnZz9R z;Sts+l=eR$Win~<|KMAI?*F=+Ud)k&D?jy8YXNh>E*W3GvPkNZ+z-nBw&0_9#l)Nu zuoPGSRE@T5P-#>CM9gg!!q$sgx9^{4&-Q8nUayKsY?(csp1o56tJmv#O9pWyc8R&g zvb2Ce+{s464zXv>8I&h^k-#y(%j7&$@6C?|3!YL1T&0M|`dmi2?ta^EKEE~zSjwL( zgtki`xdPRSosg7NT~*aCRB%vLVIJgkV-Zlrp{Mkxi;bei%DK5&y^^&L9fQnp8fihCIqqls{2l6-_YYxo~D*hCd z5BvJMBL*^E%nAMbbtmHHfMc=WXip%HmJ!=3Monx*T|h|>tW@nY$fA@wn0*2uH!k5o z*in&#nVppD9D+l07#NENW^uUw@$rLlzHvt&a@)m?9w_Q&rl&7X|C@H;<)nwVRKRL? z=c2Lb@9kypbWAbIT1upIWv9W%JV(x1yFw8Y%xPz5YG}*UZN~QVQ-!qd9gUoqQBj|M zGp_-Irqr*hJ%cP3r$~|q1u9fTs_cZ4X`8RwOKxPFa%kg5g_T*+OZx~&sH`c@*OW(C z8iZ@2-ER|%u(h^stv?alKnf^aKS_`i?X6tG{>~!}!vHeCN=Sr4Pc1bg_B)!jR)uxU z^|g|L%m3`HXAXKE9=qT&HWkFn&(9xM#dEh5eEnbxT)O2W`P|Sl@ny3ro z`v#SSm^9f{-^?+@XFgh0{EMA=vc>V=iC!jUus7|>#iz!g^hY)}bV6eJsqu#QEwl<* zFPuHn%SBc0e?W3sojDs#P2^y{3aAzdxgUb!WdJ?{w#88|p?-I>$c`#EH_tm6gRQ-#BrVjZ>q5-R_fxoXb!$DjS9yb!yyN0{I?2e0|*grsi&c9V!I4 z_>L86b8Xf%G)!4RO42hhP+C}+0tgKZKmp61z=e)SP$6TqCT#?UCc^0v#m+W9q@Kx? zc(Cy5))U=QQ;?u2Lit+g4I?KQ+D!W)%o~xAJR|b_+}-_h(YhxoQ#Obx3;gXkpElhe zc>{Yg82$@06gAw;wXapL4cM1+a4na)}Sdn}_1PgF-fd^p#nykHHR1mKqaL$R2 z`9V+#3Oa9(B7s61?FDv4U;7Zk11?85SbDw(9Sa%;cJSQh6~dp(%izOzGZk^=Aw=bF zJURJE;IDn>H83!s;8YFv1MO4DH#qRnW&q40Qot6>AFN=(S>KiN5aK!rT<{aXz%eQp zE+F>BAkimP!l0K} z#;Xd|Q&UrbpR~3Lp!|8750~g4FFa)t%J&6|beiZIcvr_AfyZ)RE{j#<3rW)TCfLtu z8X?}^PtVF~GH=J0$a04qAzG?^bT~?Wtdq_RQai8%WgyV1OkTqMubc36duIm>Itl!; z;4UkQW>cQL{N+Rs*8$ztTpL0CM$EeiB0BndTZL8ZlPAR`C181v@{;~8H|Pl7=V=3n zU|k32*#pt2rw1%m3El*PO&R2_0P*2Xd##PexGCChz@i5n&@L;Zpaq;dStj4A!*dTG z$%uAjP*m6AhQS(Ova{RQkAbfRmHOa# zdSaq=DmoxE;Dgk6JJ!@zN75cp$q_ics=M1U(Urhv?-=Ic;OK17Z`oe7hAM<7g_fE5 zSqMNHqWqu1!}%p1$}qZFJ!{S*gL3H2>acMZY5p4@dhkG_r+?xC5}?sUh`U_q@O(Lj z#>@6qVqyyP$ET#HA5Vma6R?7c5#k3vDV~d%#5wOBqxCkcEABVy;DL}5Lc2#kerqlI zqQiOj$4>R>U5BvNN=UOp9&aJ-*Y7Es?ZEqqkCy;0Z&PBuVYz>=W8p8(z$ye(@Cjy*ux35q_PVp02rHztr{I6f1*C zGDNbOghvX79tc6M7ajUcFe9ZSNY&;W6jb5(EMSXu>tfO@%JPb6ZJyC)0i#(>YIGjE zHM004#ZP`ZW!q3w*u{bgcZ{g` zdh2s218y(@VBN~mvyHKYOGwzkY%JxqPUG8qDPn17(%%X58oG$w&t2+>%B7c9CBdD2 z#T^n?cQHPu>OFgo>ukZViPG(_`vLr&b3Kgeig6cuk-^l(hY92ft+#LGyg{&P-^bC_oFyq(e z?rw+G5#|1}W=Xr!$znsvg%#@scYsY2G+Y5^trQ7OSuv)%WlU)-k*C?^JsVI1^{Ar@ zRdE8!o_}GL^U{qQH_9HsW{F82bj7f$s{u0&4b6}vL*OwPB6-sa?m<<;gZ88J^RbA- zrS@nW_K+7J9C@-Ww{RxAU2JWmTzool7fxHc} z=v6ckZedXDa;n={FG;(mj(bCN6~M`<1VS${+D)(_c<($dr4_sc$SuZY)d37_!W?u` z^J3k%2${XMWh9egNAE<9%RSC-kPB+za`^A`i6__}&P0i0A>-l?yJQ)jd2V8=6*Pj2 zT|^<_l%?K-2ZG~eu|#R|>YK?#J&bl#7XX``t#+dKF>mT8I$)A?V`sl@$@&P)Z7ZLq zf`eZ~q5tSsK)xWF${Tug zAN>p_!i7@y8S$oJ$!P}#ri)(qdJHz%b_CIcc?)@+a$1cn0MBH8N%L9Zs z2WcZbUIb4OH)WE00p+jdj`n}WxL#0Qhrx%!-uy5uMy)Ku*yRLU+Mru<(<}>PCAbVm z*Ie`248{}U;D{j7#8BnoBWh$l{GX# z`e%E_Mi$ITa{GDz-TS}H`oQJ@(01dq<-*DtuW?~k#9WgbaqR^q6~Zn$TUjgg z&}Ew*)9y{OAs3xR6B|+7M1!xeTBNK1no+OGwWqJ{^q!)Er!5-6&L+1y3=o5q+M>W$ z8%-Z^ti@7wo1Q0RN#4_k08@TOlMQp~t&zA?EIGF;V9q{+03C!~MbaZWRsb+tCRV-* ziQuOSJYB{BHK!4`tFp$Qam;x!zr*FTiG34}JUFxmu@8G*EzO#`F?KaY0WAB@&D)KBn zk#9|ZibdiM{x;djAY` z7b-lgINCK^R%)*V+eC8iypp0)_n@^6FI!myi^Q}Uagrtnnm-A9BU*5zWXfW1Wdln^ zW+tOq>r>nW$MFg*Wkgp>IjHLA*VhSE**Q1_v&SbULWNi&4&xXSKJwy<2sCDbnRGDX zHz=n7!hET^ya;>1U{aM5P1ip{YM3}Bh4b%SU5Gyv}scPuS5rYBecS;ZD*gLf! ziiwbBl)h#jh-ivCxW8Y9fe&>9MAj2WDt=0~DqQTHQ?FpgK6ds(=_WE+PKif23<4x; zVZne9VhWTiOH;5fUH9E#z99BApCJ@)jxp21_wn?XFVD#ZVz999^zqC&?yzHTDY>?J zWu&Qd<&CRICYUkJ{?!WX)G)MZwj5X{srD`Jq@hxnd+ zJWZ_E+&nxX0jX?!%sa1$Gh*>lD0l8zBA(w=HuEdMRJJkvzP@O?5PrM+qyRBd6)M%y z(P4(=S?zDN8dc{_ZANyPf*Va-^_o3nts@h+cxhq|sbP2sgPlwc6*>8h)B^4u9y$Nq zpG&@ovCeJl9#3X=n%J0|6^FXShL1;V_?wli_@6MuTZG1MV*9J%N42|W zbo##-Y0$#>v`NdQB$#d6*WWKjGN+{!HteH*v3e){j3cPhM>=_lv{-_{gli|n#NA*k zz1lU^Qiw$*W#pk%C}+A}Cd&0^k?J!e*+$<4_bYx z9DIG4<$XEZ>QuA1bmWvH&xPyk``y$m1J`3F)=QV^G^`(N-D(OM`25-GPtBopq^J=eVIUu!h^*+tSMrDY+OJ@H!adF`e1nPq6NS&}(iFo0GF z!qD#njP?WXe7ZmiH?Xd$rwc4~R8v!HFHJAN>`5jUr;GwyUL9TC7f}@DPEBMr=KauO z+4();qWgX85%j+M8vS;Be-EJB5f;6nP;aVgyhCQhEFzF*w3m`qz(_{g2HEX3I?B1Gn#!mo9v-A0eWH+wRrnP903is^a% zg+rZdDv^4J=fa8ZU#^uo?N@wyR7#Q4$GtTe(sr}t@YV0gz%;XWKQNe@J~NP)CpRv# zD)W$X@yzE-;{EoU?L1ZCu|94pWBq2J=nx8xNCe+Vb|92p;H>luGKh_tYhnPG*d3sT zAHbxQJkS`Xn}C!FP{O>ONf@~Vn21%OxIK&}1as(U$&GyW2arX6eiR)o0f{oS#$l=p zj7MuvO`&wd#=$Y#un2{Lasew<)4OO=Vq#SFFc|Z}kAB8%DQgr$bcLIH6Xr*h3{6Xc zMK30PD=qly61U>>zBg5$lV#FvT$j0G?M7+K+722wHwd^E7N+-w>RjjBAb(ory9;Ae zTc0buS}p8$xBS8TmghcYFefK4OrY9TdP-IU%1IIC%~yQk99jCRA6*?kp=_UoC}sd; zdj0Hv$=N{M^XFvL)caZ%Y?FWvOJbd(z|xD7?Cjy%4Ia}zFj^D>y-_+LF~Cj-TuE|j zv^jW+zyMNhy2nKa=epaV4qy|+PYM5e^#d1sHeKO2u6S|dbyYBuBXlR9bI2QjGs+vO zH|+tEfpHHjNI%J~q1M)rH2r&I9{3#RC%0-*T~V~E-OonKy;}gN;Jj`@8IwSrF9h}b zV@L~74Hk_I3A8I^-XN|g#o)%C0JpC7Ntb5p!8hXB&(UiHAGUHJh-h$v;=Atp0e&JA za8v?TReq@Skg)Yb%Cm7`TYVJ4>q|K#A2uzw0CRN4l>`ju8;g!l=|mqZRjrH4JUdW9 z+bz<54f#KzBS7J_!qBFxahinM2OX8baKhw(1%Y}PG=q^mhd0##g%a-sFjHW@;G8gl z3wRF4GsmM{7Yb{LY><3Ryb7(GH#)@`nVBe<>l(~+xkFiic6qT?Zj{wSg!37#uni2n z1t3mh9qj(6koN##!uIG@F7idIiizoi{OJMV=EaYw;kJ-i_+zZQlD9vyQZ4X2(6DPo zTA^lmK7s}a>tc?GioFkdtb472)(d8C_U#W>#zx95Z(qTwdk^K|E?cHuXoe_j$m4E? zHU)4t2MCCY-}Gtsi4yBrAbfcQ0kxatNv~oc8$zvDHB;)sP_$JE%<8=QGCKYyak5xs zydsToV+bk0Yb8nM`#(U!a zDz`BH;KmUEaAn9(V_X2Yz^dZ4t7?HZG^uYh>E85^+A==bVY2pRLpY7LAl~`9MwEwg zU@qQ&XWT3JOv_Ak63my%M4gdVn)LwuL4h+$&9ADe+SUp<3s7NLxXN!I25$p`&66rh z*yDqN1ga0X-LaEg=7_{1v@O5Q{QM9M=&S@-{UizU`d6`v=Uu22OYWm^O2}3e(3SxY zYf0$F*2N&^R846SA(4_dD@*fb3OjruNw_U=zGU-jeYUw7D<_q*@BA~&R1$HWldH8t z=3Eb^V`%pRdm!#O1Fzx2JH0tT0dRC3g!Lp)>2{Gj{6(rC#J5=}!2lygAXW1-=n7b| z{L$=Ye1By6oufO)B8WIF(LEw$%lLymzkj~4GvF*8o?=jF7V%*pZ!#06!%zthJWG2m z(-I9S`W#Kf*!cJtYQWGSp-1Bh4GUWlb%hYQ2;;(82Y@5hO6e5<6faV-@5oLBjxAaa z4^GERRX>WQunD!-Uq$+(IDwLvUKpA3aBm&&!ws7%MoKZqD2$S8az@;w6#9APkJE*$ z@fFIIi{4Ujw6ua?6T1_pI6kDJ3BNdP8=WDq*kL3Us*OO_{(wR`8$$9G5sk{*S+@i3=md74?f;5%ns zYThhbqiCrG+Iag~!PMnx7^9U`@rrFi^6?l7&9B}7+ju~MX}Xl#5hBuRE)_S;Y2%+i zzdjZp|6W`VM5pHMjXI&-qdk|Laezlgiy*n zr+NnNWq}`P_`M0ibEsubcD)X@$n(qlup?3r`LK1-EOBj&=J%1kVpf=mImGNIuj|Li5ZhT4Bn84#Ng(o(KlNMmz(!E zyy#ct>Kv036V0()5fR|17yM$L#+Xjbk)|;$%7Zx=W<;Zn$Jv5h-Gf1e5oU=*PRdbf zMw;xBMQT^P)EHdx&jLVQ2SVwI?AK$vZ;FoVn-2y!BBuIw$%wBQU6H?HG(mvoGFcttkU=@2uF~ge-OD^!_o6jav13=zV>_)>E$6S#s%1(g_Jk9d!(RE6Jx(0i+vc{| z^?a9-!}X`8jHE$SncAHaBbq0h@tXw>(R)MoRAIORzhJg0WHG(x zz9HYBi;V2>r-^nFGjrySDvEV?nr&iCvHWNq8!jy|5xjo9Cln&u5dMUZNP(yy~wcO+q6OOho*H^d0|>7adFbwpyyO zwd~GHs5m-un4w-ERjyO(Mwh=Ic{z&}b)W_3S$tICLtQSv5TAB@-iurWH%6H-B`;Hi z#;XKdbjx)KN{}v=SaJ6yAXF;@3o5(99$En`&ts9(Bo8xf1Y7M^4m^M7qriLldlx72IrOu@;G=^iZ*6&c?RE$^o^8sKoK>5CK` z=Zz2cQ49diFyw_xCYDJ!nS8fI-*m^|BS$%nu4)fht=ZPcs4SU;j7Fpo#aEiX9$v$( zG#})dlD+M3Wia&z1M_UT#QZ5R6A#;Dp2_N!ezBBa`NKBk;| zug~tI9eFCF8is#bZhy#gfAbjlO%!|YPfrM5`TOU8q9VNcKXL}t8&GfiheU{Ox1#a2 z^0jp$twxq?vD5Rn4i^zb4|J`6$do$maf(|9zw7TS?dNw?;1rh)4m@kPboF&XZ60kZ z@84e)ef&6#iDUeykM4HWt?GA=5FRt01G{0L?g!0yzro81GSpDd9B7)`D=8UInOa92 z`FeUzoUvimY>Xu7p}KBxG(jVu-RhRleLua)XrMX_TI57WK3c1Z*rV6c)@KiP?x<7C zV5~8-R=Zv)_+cDvxAO7hI8%9TZ>B8)g@gMHvzqC!bM5P+QH7hL=l-203JqVh4DI~n+>-qahg6)9ejpO@S zsiTRV9avRoS3dppbMlQ)@@3=XdN)lz;ceyVJj1M&Hu7~;y7cXP5%&8F87T%TZ+5oN z&6d*7ZV;{2-BM4vaMcA%rIk~w=3MYng~=5Rq(<(yEWHsEKT);aK}%JE3-5S%h*l&i z+28hGJ_zepqd0x5sy>c?m9|EKZS`0_Bn;!W4NIoyK=hb+8mufXi; z0k^so@1ImbGBE*D=uHBpB0^FF?N64hB(k>7E8GVo89)3IKd9{5n0KhZV4{83c(-LX zg+bKUSu;6s{ED;VkJ;UKib@U&oRzhgHk(}N$R~n4#*mMK?4o~6Q_q?oD$r{sCThF> zXk}g>lq=9xrIq8JAQ;E>3)YvygL}Q-WgDk=Sm8qkU_a`ATD^~XTlaUlO0^_N|mx`Nup&I4Urx8B~))>)Fz(hvHL{k}iy zcG=6m^Cx=Pl-OE-hml3FJu>V%HMfY(9tZCBwW|k=*%Wl|*G-~34=&c;3mUe?>^&-z zcQK?sP}`?|yC%hAJ<$4fkK?|M!T6S#SL-zPgVo^DChJ7<;lqgTc{a`qIFTablQBfU zMWn9y9ktE9!zgMv|G{|LyuAC0)#T`wTIM}EszV!kEvjr9VfQaVUw`)WU%%;=d?r|# z+)!INxqI;t5`u}4Tuup$=EIr&Z`CAMci+_SVlSCC-ZA<@**sqg7S9%ol*UA>Q(rfR zywKh@S+X6}qv2K4Umg-lo69_Vvp&UM&&b*s*;u=4=(%-Va@ro-c+c3co$JEX@1nOQ z!;~zj<$t8I3#nh*!zR{ik%;6b=IX=tf!nh;pJo_mCeNZXMZQz)iKe+ay%FzqK3zvL ziJZAO+wv9LA8AzGW4&3J)xCB3s~$C_C|kTk@f)q5Bp4S*Z9=~kJtS$|qK|v8Gg9=B z7@p`~w>^&oepChAPJ>)>z`=na0OJNu%;_E1z+JU5B;Ey{A~oSlS&7NArVaRdZ`H7s zDl_u0pf}1j$|-AF@kWTP@>bv&kg1^1PP?nUE#lkbe9nLMbpPpC zlWpwe@2^J*uHX7{MnuP{rF1BmtQ_8`+@p#QYo^yE zXP8+3wN=T2$1K>LcGwoB&igO}NtyM$P*E9P8n1IEV|y*~ysk0hHjwU*?w$Pj@$3Sz z<5#wxm4(WRZO;0+D-9dWQm#)st(r9^LWeJOtL-c0t%kOJ>KQeZ58A9b#osurpc|g! z|8uDg{OB&w54sm#S1tHh8zGjMYQVR&r>bM4yFT?jFxyjL$aXBJJb(Uq>G{w@K*@L+ zHXxoY_SHK)R!PfDb=6whawnPGKaiIXoR!?^`0#a4p!Zy}CN1r$r7e@!vVGBHyzmvZ zJm*p^gY|EK4G}%tzUtLqu^paQaPnm^?bX5}-$V4~6&uySv2wkpAJY?`If-(yyqUsx z>Fzts zllmn)7nb$|0w=Skves{xw1#H8`d(8HG|cP-z-p%Ook06!%aPybc}>eC{-+;Dtx7XB zNpqdCv(^B8gAd0PmiH>v@CS{(Tl)paydBS8%DT1jhVE=tyie3fYYhs_8K^qF4cC%t z&{Amis?EK+ z$V=M~X=FTZjQV~RKlpN-_F{1H?Ze#Hy_Z!~okuzKeo)Ij+!CX|G$@=tQBPfHG-WaA zLtp=x^UyRp*~CxBK>gi3tYR!o(Y~6mjhIK;gReLT)TRTE0dNBl*Ys?o>4 zSiz+H@I~xpTYUAj^0<0Hg+#6=v;X7E~z`_Qd@0Z0pu=Opo|neG^q5` zGpnT7gPHK7)g2qe6o0ayQVl#M9_s0LdR0h%E&Gf3cuyKuJZs(@20d5S9-YQH3k+#Z z6#5`o-rl~We)F!Xj_3O+JN65dRK8Q&>gsC6Wb)!^T7yFGrfW0f)A=u>4tj#hAY z#(I*^J(uG?#k_WUND`AKQe0SjQ&%L*hQGq#gO|$lrrN46ckkR0^qAVkxQ%KrevVPl z@^CzfD<692=aZy&w<|_rkf`)3hJyN_cNOZa|D%DJ*?m{{y?=^Zoa`Rb*G%@nuNi!BR{7e* z23NS%>BARAS@)!p)j`{_K^^*5oa&4c%daNoR1qEqQle&@9; zv#g|BU1BHgH128E(P7|wcnBva;y(ZRyMx&GrS8R#T1xgS$nv3mNrnM_Za9&=&BByc z%L0=ozudgMyUrpn-c4_=%35fdx!UO@&b=!4I5-&ZuB0Z&`At}r6_T`z`tPVP(hu8k zKyHoJPH6xok^l4f{*#UVdG7yavZ&keKXTZ=xAt$&{$GFo=i~o%YyVqfTgS1scP*R8 zJqhVl0YP%~60 z{O=Ws3~9>WgHj^iU+Ta%T3S_6(Pw*6sXeMwkP8MK2ns$Y^a$sW2TaY-kdD*cQAw%0 z*dm3vZ4u0bRj|bryFY)fkfEOR5WyPs<{PjYk|GpA8{Z@|@t&`n1T2R%dw3Bul$!0^y zdSdDsV<{s~8eo+fvKA3<@&0}HQk~gd`{yKD6p{zQCROG0Is@6YAzUflWo92%1^b?| zGIRxRzS$IJyt{!uuR)$l%J)NPC)D#0b)}~>01&65@&eEW01;sZ7*v}1GD()U1Nt20 zK-~mJ=&!NzZve2aTc|b%o=QE!A!eA{hmJo8pXg#M6I=nh2T{=abba!8zg8`9OQC51 z-4niVFc|*Vlw`n>hx*qxKm14-=;`+0fh6c3K)bpBbCMdAK60v3>I(>zNZ2E!vGH1< z#Ws*~X>DnZP+zkHY78mB$c9YiDR;4ja zYj&3-5T;$+-rAD%{*?hDmoHNDzMh$cRApqOq-8-cIKlHG3!ZGdIh>X#Cq;+{g}@gi zgFr;C?pwoKW5QR{ZgSq~odT7McCKVN#U0*p?$1(lMkyme9$O#>wRV|38xJO{lkGRa zt~sBU4@2|FltmC2n#&xF=Xfp}8X6~#cEkbY(CmPw7);(YWI2Z3T}m%0W^B5AtyM!q z1MUViSDLCS)85nILpS24h>)?X33O!23v*(=dkS@V9&TG(3VDGffp9i@5u{^;HQ_a- zr7rTk-&~Jk=Zu>JubEiV27*SWD_FHW2`zjrqS*fI1O4C%w&Z%!KPY2kR3say=)&QULx2;$RzCpvtLn6oShJBma`)u zh8!-XgKQ$@LkKJOfddG>X3tM&qhRxh@TjQD$ivm*Qi+C{yyzdmx8a(EX&zC8v^RRwm7cqG4Men@ z+;`?JZ+poG)NvTXs@F4^AMv&(1S7io_A@5nIH)Ts=EQVxJ(36(4*vy0CK~3l2Q9&9 z;ZlOKE;G_X^&dWvjbO6pq8(^$$cSv=6%jH@CFI%+Wc67Lq^S04b4HO)E~&F(xt05a zh+WvG7vW(z)W+L*jO=w>K3W;pFk017v3XZl{1BA+nnU9#d zJ2WytK@%FTzxCNiWmYXsZzHTeqMQ-TZsh$XM6dn6KMh zK~dGlBJ{yH3yUD~WcFE63>INKR_-tGv1)a=nDhlDJ9*l2@QjNly8R{_2JU)#Muzdm zsAzIKD2;6U(mI0)ZPO)VO?GkHv97CRlTW7fUTF2)m|tR(G_ERn(=sYe(12(NB-Ou5)Ffi(fJh?XD%y^JX-;Z;Rk zPdQ;RDG^X?4JP;oQ-3q=q!bO;Dl-fIoQDzU&pqcEh@T5{w_kyFFNSssWw*vBK{ja6 zr_00kQ;66Rl=NO7LrZAhoyF=i9CKh|0__q0m#*+nRZ(~#8Z9}3+ffSs-;=XvNwR9~ zFp~%CK!^a?-%dbO!Y1Q>3G_nMxKYmd>|zWSn^)Gk5T#1Q4ED@~szT=TV5~gV9HM?j z)XyAliidFzh8-0)K(w%5WO`*%QKyXDoC*<7bp}?;xFGt&sHcHAYW8*0!;Nc_8u_sUzcb<}X zXH1huWApvv1vQ`cz7DYhlyngGhMUQ>RQisaTup*P?Lm<#D`ZKZJ(5I|QV-TgimBye zpzlE#usX~`u_~d21hew&-|W<6^Np_aH?W=J(jMhRpo#Ro%%9^Odpw!ag8g=BTNVa) zd}C)5_$vBM4n_ylHl}vsJ{%MmE$d0Ib%uR(3=$!r|$^Cwd$ z;&D8iLmILsQy{L3QOd!z5+OcxKBQ-?8U+j^EbINWs{W zVDxvix|UxdB%lPCa{2?&pu;*k-3UU_VN+qJ%3}a2#I~z~E)Vo(%{Z||1!x&;L9j9D z!F@T2%0`GG?!^mqX3*A?j#-Sm0;7K?BiNp2hz8Mz9_h+~mBlHc6h4x~Yha-tqeE2i zf7<)XsHoSsU0Q^pk**mU>2e6^?o?2e20=s+l#uT37(yBZlTtxYY3Wu$kp@8!k&@=z zv-gShe?Poyowd%#bH{aG*L91TUdz)<0XZl=OR<9rQvfmzwsouW;tdx` zOq<}{P+AfHJ(|4kIPYeQL)Lx++(GPNM*yK}2wr{a{g}pG_xVXkW4aU1Oc-z(xGfm% z#Bm0MY4-Me7}|vWKdL0cQ4)M?D_0cJ=Kv&NK{tdS&hSJZ<9Yi%lXFTPV`J$gDbrgx zxuJMWAXetgZ?Rg5Q%t;HQLe_(UU9%n-HsRk1F0G{i}>;$wqg5iB=sG`L>(_fD4BUA z(02mX$Olv?snGH*Ds=_;#~J4$bEh3L$@_6P+(iAAeh6+|XHfdFw4@Ay+@zx*px{fBqnN4Z97(`-^sB$ z$i`Q8S#(kEg?$lhw)OtmBE=mNlqoO&&gUz;fq^!S(g#MWhNTM^{0ontcHNGl%b$Nj zRK%!**=2k-@L{w6oW8W%M2k&t0wJH03(WN$^Bu=+ZlMDYB_~0Ga_*;?MCq^ajKnaU41|aipuCL6m3ss~MmG19K((9R@L%2)VphTSo_v?CJ}v9t#4m z**&0kgY8O-_9}Yx{3;#MFdBsK35pT*#TP5%#U=yxFDSf+2-_zu>^DCpuZ6Qf9UAcm zN*D$jEoYWl!>QNuZK1~7KZZVDuKw-zXkz8t8<9bp8if9FPp8Un4-R{A3B2z>T2dK> zgGJ*DGBb--^C6GcagBA)Pc+d8VxXl^f&=-!Enc=vQ^T}p0^v4HePc|(wgvb{5=&6d zbZ-^2ZhKJOXeY@plCmefinSJMKw+F2*|j?KROm4k3w#ll$u(iS%{YhUs1UFN$~{s# z46fq5UWVZFdL^-CfDsLyK@=gz)Pz(;9S};QMJ>a9*&?7xyKL3ga)&L=J|mi|=#rDB zM?|Th1Y@+sPW6|2={*S|37yM>V-OE>*!T!C7Mn#;;Y){5t9uE|hVI9*xOBwfLa6DP3OE?dBHw0UHXx`C5xM4pXLQ`Y#3LILCxbxVOJWByYOUO&h$ zUfvGiC2lJpFiG*g66hQ6lE82*&At>-Tth#ml_TdPOnmFa@BG`4{^5ir3F?gVRB18o zR-_@sku;mLF^P#}VtQOq9ffjc8PdIG78!=zr~Z zQN3^bvsLwkb2X9ys^P;-{iZEf4_dpePu23QjQE2V?WZ3xqW~$ zfGpls464BR8(Ayj6n8DcdJ&{*&vrvxre#!7@5VeVs+;-Cr}ytJqmM5+U1PpIKh@ue zM~%C)7IXB0BfP{E3kq^oRS7v=ldgMrCB(@w3l`=vG3A1TleR++2K+vlnVG@Qot5>@ z3A?#&6i@(@NM5EdofHX<>FZcnSio@g>8Y^_wj8&-xn|#CoIiLdgc%nS7B)EZNc{(O zbmj9!#61KOqSYhdHY)RPXI#BpDd9#(MU|SDcYAsR49irV(hEQh;746mP+M0gL*i5y z6c^>?`E2+0G6?>Vssfv%K0|MmLb@8D<_A&O5m4!}vM+#hz&cdcSTXH`U)R9j14j4< zL?oDr;7xseP$wyTRpeM97b2yp^-fsQ#?ucQ!RTz^Dkr^-!Dgpl=Q`ZxE}P)q%2?_ z1+6vchFW`}NeamWqg}E)?bnvc1N9( z_&o>u9CUPb!^6TjTxJ<^LAoDuAi&OWc=Y=Nq?42N$vSuk?gg|VlWuNqL%b8i1m9GBkXp)g3Av-gD85-akNaEwtHIS!|~|QP_hB|uDs07{t`pQ3GxqRYrM7p zH5bShP*eQ2WD?YsiUB3-n)uB zJnRAMAcEkckKKBl%K5*XKfQsZZPI^L5C2bolE8@e zvwJIC20EGWB%DuhHUB-vJ*aK6(&dNO^+}k4HVqXCoM0$E3KfGDY=~yt+<-Uq!RL1Y zNEeG&sTir$@KypBJd{!8@O!==hV{NUt?1cOp zqQjJtBM(B@I_%iB2!05%0Bitvpi(2%2_2!kH_+RQ7h^jEmLd#a2L|pyrwthyuLxJh z%S#rG-UoaEZW(xaICS^*)diJz0k&qNvYikT`2;%a2LPC1HUXW4ZoBA4K!E&lSL{U8 zWQeNC^f*1oO1#;>KMc$8{rA)TKSGFqC^G;2C6@R7&wpS^q5po_|Bdqld<%TzE7!H1 zCpUtXLxO4#S9_E@%>x(`I+cUwx0Q|5d)~;`RziT`@U^h>0UPGe4 zRj_P3KX|YQ!0E~HF-Lfza0&v*3X5-?h~s((2FeR0^Ho4%eoIi_-6ApSl`CfM@n1Y= zFsfv@e&a@>j2Y}v=`s=Lz3=7K6mU>0bxT^l={|my%(O{HXG&CGW|X(^io%Yo#36&vjFw}3IsTi2V{dmi_2Jr1>s)1$anA~0OfFv(?Bke z)%_t)0?luybVy@?55UQu%!~9luwU`StO^juVxHr}k2k7oASzNjcRo3|&&KlT2vCOw zyzplKT>*MXDxiqu*cwpx-CBnxx?~a{gOQg|+qAWyP0MvYy|h+54%q;z@gh$m#2p=x ze$2~yhZYEq-eAE9_@aeYHOaMk8c0t(tt1x&GDNHuAWD@bt0S8v7mRuAC^blj4m=Za zY2@I_}S%Ry>K<+D@Oo95x z+^7Dq2X~a0h%44fFHV%og1EpHyepR>r{}{%a4h3V(7^Fle6Vg2leLi%`Eoolg*eS* zn@>5)yrBTWdOnuwR(sqS5PNaJv5)s7Z0xST{UwQrm|G||^nF4|J7M_(vV_}IgLpUD zEgYP;5?N$wEo$t5c6YMBkPH#gx=0CE=SG9a9Oz=O2AYCo*HO<{zqvZA&AZsa15*o8+hJL{g!+EVIHj+e>JivxRR=ut_F$AA(NeX3)V>z`<6{{DpK| zuhCRVke+MH1E>4hmFGBY?v`o7+XZ~W^HI`lktdV`e-?QhlzdUf$D7TyAj-7JlyIv(Kb29PMTk;JRm^UPm& za`?1zVGlpkp^{P~3c(2Om^zuqKMd61gv}6534_1_q9-JgRe+R<$e21r_JK|PSU>Nz zLr>cA;SV2CaxtF`D=yA0F5wUR2abg)DIu530-Q*h`L5OTqtXE!xy4*WU9Tx8tO1e-#iFjOc1@<6Zhw>M+$~C_X9?6Ocj!C(W$FSt2PKxOY%J)Y$Q*(XQ zr*|N{@bua%fgZa>C{oLe%r2O+T=tiA?X>E^*1}-)F?0XR4MY>%FC)qKqw#85(YZz_ zS(K84@zX0p1Tn8>S5&lC?o!@Lcmjg(#@E$8&lUQW-<=S+z@X)Pz|d@)Br zpZ+jz3=o5NVG$ObGr2?DpEbeC}%SDEk zX82WS8>B!+fWmGcn{M`lY6glqpTWBb6)vl~BC{yz@s$0XHL#UX>q91%h|y{oxaRLL z$Oy{rHiNMjA27h5wx_@H-Gg`J1C0MMj3;PXTU5(hvN!{;DI9OMv)nZ@{K;syPz(;Q znkJ+dN`Rk9~i*1A?F}{}%S4z8gKqNfeAosgjLROaP29Iw7+4Cy1Dr&sA z9aANg+LKNBw)S5!IUF+bYVvGo^-Uv)^?XcEbTr_Um64L~%*Ts>%i^QMB26$lBc>x3 zBxn_-CDEcAlHbM()HiSFF-1H5LcdE9SGU(gsdYAJI76@-5|Ch(?fg2DcF6Gn!+2gC zkEqKETP6<)E1vEzSfCq4SX0k+OAb;Bv7)sw*PK`9${9HKO^P%T5+%t5`?3Xy7=}FWJ7u&q`LwRqo{RfGM#s_|z{S zQ(eW3{6hmtN*2#PbCQBsNq0!BeUv7NU&m zO&AoK!0{%y4S2eIoCI{0$+&+Q$_Tb^!UTJouAmF6!3j!;w7HK+k}0nbMo_%smQ#xm z9cCpUH`Uw#r)y0u-Sk!&oowO6SDP0;mVAMOTS+mmeh5yN_KR>EOZ6uZ=JQM+l)3r4 z!BK>=2x@-l5R!zuX*9_rh{oC~QFw`8QHN%+f`BR*hLJHz+Mr50$5ZQ2>TyNS`K{k2 zny1SdXiynR-fQ-l!=tesH7m$Od45jH^}i3m z7AB|^%J&d9RG!Mq<>+;qHXa*P87D89Ys_{e=^~~WLnZQ<2aW(GS6?sFNc>^xE-ejn zMF@mml0k#D?@gB$s-M=FFnQ3%5=%bm-ETn4k(hH+|J1XWW*hAFd;w)bSuoheBQhB$ zBNBSp7OK60u^>vETrw_5@~srO0JNd5!#KQ7$(|fsB5?^%(!_;g(;bYx;(FpXX_QS8 zdbVmT@@=tF#aHQquk7d|@`&U+hL}c1#q)b7+>W)j!nG?%)8`o4lest}vTcmD=`KR= z`%GljYXIG0m?X$@ZwBl)tlN;-IC!@m(2SwqSL;jRqa(foE;E$5932+48eyGVN6gP# z>l?O9f1k{N1I35kKgKNN(MtRbWZznFHW9RRw(8F*Nm;1k9UTxhFj-#Ng{++-L;%%Y z@lNLuWf4{oDa@y!6KaJO@*F9npttW2!7PDpm4PMI4)x;tD_3)X1;FVNNG2q8Ts8#n zpn?LxDOo<2xmNX#F?Vg(U85d2z*N&PR`)FO?pZDB?Y2;W@|iTFhcJ=Vd2~RnM_xZy*t6l(yWawVVhYO&gHp zw0C+MsU@2Bk_9V;(+8s3xWD`~V+#c2STRB%icZ{=%iMXYvbr>?dwEB{9Qps8Kv zXeGi9XzGmSpuDaSR1h-uAaYnhu3-39y%_Aj*#ym{XiU&Xd~*1O<}kjq+{xq8Q(()S z!cnVLm3plsmq?H-(}9(L7w{A9h*}T#*mb>8=V-M!3WBaOH?9XvlQr)fF-P zD8;x!e$@fc@8vscz&|tiVhitmB~;GV&it9?yZu6F20tjK20H+Dk*{QQXPxC@r#H+Hea22x6$Y7E z1X6Y(rX4!3t6_goU|L2-$IF2C(~H5<`_{eKhz0Ilz{j!PJBPt~upP3YJ#d^!g7Fs> z(>ft}TK3MU31j&~;35ThfBN6AmcIN66hAAU`<~x=c3AoT>|xM1t)LCe2c9T)^4UV} z{oe^j*zZ04y#NRWoZnMmYe3O=Z)>p!1Y{ZtxQp<1KfK8Y`crc$W!8$sV}$S47` zw*)^xeP4jl)z;8X*b%f{jz6(C%MU#3gL-*izzj9h_I#`*(4S84v-9|Xa1BJee=`3Z z=sY(uP-`JD#7Q-bJ2mDLu|2z}ke{MayEE?r94f1)(ADJIN+bgwmy3g4ynon_J{QiOo+ zb9Uq@%=Uvn^M1%z2+YN8=N&pSg#F=s!ykg-;=(ko5n54TUmct{fbEbe0#W)MesdUn zPr#*tk_CR*%xI{C>ex?0E+a6-;GaAKDGJi!?yqk?y&3!rMDDGst=+~YqU5=00N`O6 zG_c_U=6;Og4#aQ4Eop`lXJGU0x=^`^1o4IxP-DQ(2q3l>fS7i`#I%~jVTF@^NA~_t zT7R{NTxq7b=>^(Rsf~uB$|OChd;O9AQA6Yeb_+c8{jcCVv1sDA*JP!+VBEZ2act25 zJTQ8w&;gFpV)&l($S5awk2Kg@X>#z+RZ#c(b6vRyhpXVERP6?{-Em-L#$@yslz>r>bB+_qn5W~Ne zmI_(OFa>C9ykFsS02kL}X*6GAJGz)OoQ5l0!oHzlj&lm0RUQ~3CMz7q;8O=yQQ!(f zZw1E))NY5j#clhI-fhYYtELKGisgE?3Z?l~$ny{EE1h+idHxzzR-1=q&@g;S=uw3n z?flTx!;Z((U7Yw5fEt0bEuqkh)TjdkLn{JqvQywHH4V)em;gZgUd>_~${=6MttBl; z>Ej#ot7BkgWyPp6Z*HL%8wj$3WdPpZZ)ZOZ5wo!Y>`dTh^L{vm0vgJh1Uk9?4E`}7 zBfw#3gdYLTTN7|hd{xz~jYtxZ<)N*B*%kQztfJ=Dd3N(%E-sK2a(VQbScp9*!Qe&q z6QBjK!g`bMJJtK~DpPE0-SAac%N9$s3mYV4#^zQqvM;*c5VW%%rzJUr5>xkd z#L}F{H~pBvkA6QmZWc1@q8ErJG=E`rFc;14K3aDE7h}qWvVT1WM<>1`#ZAV$HL>Am@c>AN^A#s|1r@t7hZM7sE0kV%cPPT*(cUF{1lyAaaO)9AE zSSLxc5cFQu3k$AmppAAFCH^&8_-qS6hxS@1Quh)sP0yNj87%OOVV9Wzs|kTOkduPX zcD)27%4W#{=bwQKb$^>Qk7tnViaLqgwJk?}OH#U|ZfJ~puF*&zQ11aJX5fvRz?n8S z~B`h+gD6&T}T3yGS`+x-&2oKsCU@KG8MK5_{sq1tM_4zb z%JyT$6PJNpDJt8x#A;{3LYvd9V>|r(3xC@9bu$Gb8vwyyXllH-55=<_&6>TYjkiys!RO;RJY7sToS&>5z4PX~#_d_w4>{2&xP@>VC(6KFU1ghzAs=0`2<)3iH#g><=k-9bze1 zWjmNYy|}?lqn;akw)@3eGoTLeLp|NIIWyldC-Py&?w*8A`I{XFPrXkUJ9wla+h*qjsPJB@hhja!y`wO)J8Q@`&=lv zq+Z;vHAkJDYa>J*!8(`OCfevw+eCRUF z7rq9Y{F8{SsczBI9JR8gNbkIX!IXtM#0=w00ZGwD^$>na+|k~mTFsQYs!Y|1aP*_{ zAc0i@<}?V|!_2?tiWXcQ35zBXqpj)+RHv8)>IBt1iNv-#Gs8nfl)z> z3&1JB9t)WnCTxvszDaktyyX@oz?&LE}SwyG*T{?i`Lz^h5O|x{UaA{AdFAlGj?uuOCZ&C!j+L-2?10 z&9CYHTGUI<88zL0%w?h5SeMOsyOgb{#`TW6yX*oNc-jm)2A^G1VGeJsK`2MVXwBY!4HbY?Q0Ow|TWEV}UaH zUYjjdY#c}X#Q+cgC%wgX(2(u4t$MmkW?%b5>D0g8(Ncl(?Xt})em)+jUQ9~YHrK9`>{r8S&kR^a1T_ADe1X6=D^DJtku(PGeM(GnaaP#DEF4P-ofe z@#v)na1*iAltzRX&6=fA8~1>@@h}Gef+qRi`MjLO-U{`#8n4e`E$HP`+K&{2_tupJ zdP+QI{^uzzj_tWqeM8c~`lXi06Zo+s*CpK#P*hH->8@fLTh#4;4TRF9yN zZbDZUtpje*X=(UKR-JA|B&h#|{ZpgMJWEaxl(phi zsE906_&a%t;=0<98>yExMzrxRyx#2OrVAf?E$|TMqs~a&MF)u12#LpQ`!se() z@~zR!`4BZgoAOA5pHJjf$t?%Xh;@f~ULO^^fO!g$GY(U_q*$E{zSbwks5q-KEvmh3 zvX^-D*ybR(um}&fl*r{FDj25uFZk_%=~5>KNCXE=Hy8f2myQs{B1(}J4F-#!=u_W3 z;s$KyTpRy^G+NWQiQ$xKnp#s!a=Caj8(^+h^jwgy^;L&sQM_r@O56TT=gVQ8H?^X= z=2+wOXocJ4M%#3ieI<{ji12LcyVbkb;Mwz^y&&`&iEQN@Nkf0c5u_1n-*)$?+T246 z^a}=%wR)u6u4>@`1NzrU=|?CmjE7(jvEV$nghCD))pvWHSw$LPraJqs<{tkYw5 z%8jPZ|GvG5YL-dX6yZtVdT9%^2#hIimNxS>@gUy}ql2tFl=F{8s1w#vukx#4zVx|V zm!3`kz)9_E^t^`Kbg4*6A>9QQ(UhVz_HP5RsSdC(xJnj!S>PyzoyKcA5k} z$&lpjfT9dN`w?OAUm(V~gUSH%eV*)n*L@nwvr}55E~xggrrVllC}y@EB<@3=uYl4Y zKT|y1YYM%GwL=gP_?i8@0oFy}2>}RyT2vIEy>o}#&4-Ld0g7Qu>CkGIw~lr*M!}flO|P_;^JN8h$YALhwt06KHp+pT5$0$Y z_>79+O+ed(wb%tT;5~F9P^(JKy#je~=nf~g1Oppe!#rs+R;;*ZBa$izsx%An#6S0svE&WtG!3(bS>N_F1KE}X z1G|}^0fThs1s!omOEmz5j!Hz%^)P-Dv}#BZC6<~W?w*302LM?8Lq1>p z%1{&inO#A2juR+(irS3MeFK8xcBXimF(zr@Blw#!6oS0Cky0m#jzQ=#?G|?$bSoyV zwWa#0D0TQhcHO58q>I-q3_4teM@3D2gO4dzq7UY%ZQJvZ;j!C{xPms+)YV~j6|)cN z7~<&<$Of?RFasaQ*cc_yf>u+>@2`crA4o9iRMF+IY6crvMvWQiNGHp$xZz|aTf*E- zQ?x7`mlOM_A2X_^a;A%;zsI_QgR+CpB+NU2)UuSutxGKgKFe^)O-Ka7Eld}|+AGY6 zQKn=0u?H4Q7rMkB(}y*8g#qkB)trl>JR-gQwq_#JeIV8Et&HHSk_e6DGWSzkcGT=M z`3cZdZUE}S%uP0Agu2NV~7pB-;Y!YT`~Is}Ng2<>lc?ld>I7nC!_;s5a41QWe{eX7gWx?bd)fW0yjtUL~i<6foC$B)ba? zfZ-G8!oWRfkGKZFjzJNoTX5a%BQ)6qGkh89)MBZpG;P<$tG1z>uXp`K(nGGRdKvEK z!;h>!MyhWmi0f!BbOszOWjAs(dCbBR4AMR|SF@K;5P_@j93c`0L))3DvCo0=lF;K< z1Q#o#1_D9C%ri>~az#Uh1BXN=DOiAXnaCM6h+ipjzbHgdAF6CiM=p$?WelJ}G(@q% z(1KKYz?guaSXo?)f0D zpcw>SFqpbtW2dg7(grbZB@%Cb38Iaf1@I)gPUhO9IWroTLvZokKb$;pddxkqtI5;m z1N%CI2)rTNRGE~kJo0RUTQbVJb`1#9wUH6;wg=^IiTQ&)eAZR#zzbJ3$YSZNH4lw7 z0+O;o1Pa88xm>F1Pv6{|?f}iK(iIl*z{oH~-8-LC4L)$d*+MwyBOx9kJ`$n*wzy2< z$>9xWcVovnQMbv7#)a-qH}9h~iOUL=yO}i0-0LS{dFhLUD)x;PK{A`7d z!D}|tW$mh|uZn14CT#(5NFzrAgcnMN&N4d!>*v>`dSScPd!U9mcc+B5)e^w4(1GAe z;a9sM!9ai;y%xKbxt<9!dZJtVJa1`WXIZ?I2!?23^C#mVLg~!~kHH8gyV0`Ofb?v} z{Q|r{e%xXb=AH=BAJ+wx^>$ zM`R4gbhz)F2ZOXugf$%$q%(=~QZQ|vvAbG{-N|91VBK(Q1f`+MQloK*YOp{HOZjfj zLhum7^S@TCLdYa4)ka^SagzdwI)Yv&s>h2r~-oKt|Q!n76Pbu3%knP{cq7EmGu+(S|$aXMq zBc$ovCJvFLnv8i(Jp}iy!^o@@n#{Y+e~8*NUB}OncA&+&fl|sj3AmIUWfV7;RRCPy zh)aNv30&c4bQAY(4av#YjE%?wl~sEP)gKDQ;}k8l{ct1=-T^REezRMv+F_(|oC?(n;k8pzShCy?cN>^ro(hNyM+oYuzq5W? zhQJbg@CuF)dm!%4Oq%uAP)gW|DBL_r{)=>Q;3|tJfHis%zObyhII2?MQu|XUDora& z`WeWM-;K9{Aj}aUX;JIY0}`nAwM>^f@RyudqdQOWh4(Zff}qP`5#9oIuy4~ez+BMI z^mj*uo<;iXA|v4_bDCB-5x9e7IQ%A3>aFMqj7?2l`6%X65#n$)VU~)V-a}GZZ^-13(oS$yWv&!t>ceD95Jjs;iRNM8x=l0ft?*veM`^VC<| zoOW{1+b?Wq-hP|HPw$}C?|_9AAh+H#vym}c1Qy+!d8jQM7DX_ z=IL9y@B4Y)_xb(4_x-+q{0w_vuIoJ4xz@3c<5)XfUr(EygoWhPsZ->aFKHN_I)&GC z>J%;;5ib0uLRtU(sZ&o+UDi;&;W@OJPI!ZUAiMF{>0RZRPvy&Gjc5xSJko#wDy{&M z2tzuJ2qM+E?5~mDOWcEPV-vUxrJ4D+v-pVu!c-7fRy0+r=$eC`O&^nPJX-cbJkq_P zao}suVBhoQRsHtj1)mis%Ka|~jhw_>K~!o)TqR$QZKzMafuE4Pe|{B&tNZ6iB7)Sv z{y-x6*K6qzf4z<$eIECp@BVs+e&%17%Tl5Jzg{4Yydir4trE=Yd zQ)5NNMOoPtD{bk7SFgtU`ufa;RHkmCv?zb<@%{U&gd2sOLtl-esv9XvO=OX^~mBwQZ zaUQ?5&g0!)e{M>m^Fsk_4D|Hj+A(FG;_FjWQ+Bd6@^EOnx{=oO*P_qPG@Dd@^pmXxOFQsLq}a*-NYotbJ~eyTI}4pVE2lb z%>?KD)~M9HvsI1{SNxA-k9R1EiHR?NwK!Od^}liB2HlFxIQ{WXhw{;Wo|O|GoN2@=S^T>O9J0D4YSgQ2mONZ=(wZU)zz@bNLu1^hHIOf_0E%O zuC8VL>t>Y?mxe1Tr-bCh`1m^d`)?{d>J;iBcqlj&@TJJA@#v*f1NAQXl)|IwOP4MY z9hklT-rU^W5zEQQ$#wg6&HZn@VI>IC^(a?W8VeGVpBO{^`1trK0(vGU&-w2HYXK4A z;XkWg`6&EE;%dJ=s362%aX>Uy-3jpOdGE7$hkvyGrw(QN!83dt+Djk_4($pFqNEo zRAzuwrB9Qm!}ojxIFtNnII&7HNZ$I>B$9AuPrl&lB6C$5-R0!uTJwk!9%jCKr~Bmh zpWnl&{`1G{nd5whhK66dy9)~nFz8V&ef_xFTKVS-j$h!d*E5N;;EMp74ncx73#FvO z+?(ijnyVIppC!3YzWliiFZ^f78h>9<03D*pN3Wq?(5H&Klg>%e*X3A*Ug0iR|+=|8ct6C zomFckLU^|S@_@JSI=_5*a&#cuxbQG1H(;!p_d#Sbj^x#^x8L@&WqCj~{6C)5b;)rtoYsa;Z7EsI%g%iy|d>p03JJQBSqLXZKm7UQpw(#5dWC zBdBR;P?D2N)xF|jXl|bN{5iIm!zQ%mDGO4Eg;BY|_lDe>V8|w<24bJKX8M4kpr%2g zvDcY%p6~T0owIUDKQdXjn!OMSGv#0l?(8#n`y^aY8pjz#?W9sly1Qa{7%UjO8g{*W z?3^B#?DYJ;UTHuX?b!X}udZ`ngA7lCE5VWXI)O<{q*^>UWSrya{xhClIZw* z8^R%vNq#3MCjo)CZ?ETlj9wGauDdT*)hbu3mxd-KjcsxxQLwto3S);B*t{{VqtVqtkD??F)?gWlZR+ndK= z&VJCB6m?=^BEpX9)H{*_`c(F!!otFi*4E7M{I_qL<#2<7gX_zQHI4c# zQ<9Uvq{}nv;;hHABt3h^O(ab-ROeL&f&ZYmpkThpqzoNpUi(-#S?N_uO1M_b$cVXu za{gQxxomiJw5FJ-sOb0Yr%z8A;00eiD<(!o$SwGqg21ZIa}A!gFF7+y#mHsq*|TS~ zyjSBA5?ZP=m(;=y{%qx#DB7&T0U3_s>FYQ%%*@P;v3dRG&B{o1;KO^puM{zbAIi$C zqVNy*TEl6BZ(BUJ=Gl9_y6W5~qT#nWe={gFSuaImrY*<$MfL`(#NDvQMt>6%=4UO5 z8?(JRj2g2MJ%ep+wh$_Vh9)NA^&dZeOcJ%y%7fBqF;1+OJrFb!Irj5(wRS|>GWwW zByuw_pdJcMm-j$TtWn>@qi5!IynXwr#hW*8jDJ?S@Ce>c!sBWku5i2{A%Q|;+j#E` z6k9MYN8A!B%4{)&D)sGKOD8kSkZsdbty4a5ZlWV%r>QvT9bu1ib8}5hOo(Db7^80@ z5d2$eZzCfkt!wgQiL6HD%Kz*_K2OsO7Q-@pHJ#vP5h^5XgPAMebzzjkd- zwuDMZNOTr>HczeA`)s3PxjwJnCz9OLBPS=vmSB|n-V0kbuvy55%!${Hos(G;M?XQ*Du=zSO@zCGd$aYA1{6;-|7ta}uB9xG%Rmz_C+AB7C`g25*Qzmtu;ve{och-J&09xv^w073}QHFn7*76 z^J;D-Ms7JdQ#G}zk!n}b^drxW-`zNPr_F2J-gaS$-D=<<43t=DhvFmku3b}pyqKJl z!Ys+o+9%J;$EW^Q6E}KUK~$8|jEcU-F74OQM|-D{D&!@8bO~+|krGxY!MKx^u=sSO zFRSfQEdfja6$68{FDVzQtE;IHUS3|19Xc|8Hu@`*(Mv7&6R>z0f4KYU>l>YojScJK z{1;)IOHn&WNKe=G^&w=Mbs|x1Z3MW&lDbpF(vCxL_Fzkc@VL-KPk$_~PPSZXgUA2; znWVd~{Qfr$+@;ys&=69tmGQ<{BPd()aMo=cBuW2<5{LG<>B-51jsE_Wl$2?0tSKmi zn>VHIw1#M@ZrNHOaH}#zR9*o#_NWL=)RsbSz2+T zjVw)x(BgcnM!!ylGmo>u25Mu%Nn^J3lT@y}4PnGSEa9xS!i{{nDzAOnRo4QYSy`MCg5-JLd) ztZ=3a_oC9%*^6$t{YXhkvFV5zg{e5spG$jd>lxv5=gwg}I>|l+QBaOWSu$8GcE6PF zV=TzeKhskAaJ1&h?uY8?T?lS~2eD^^WYMLLMd~uVx~W027$cX7NZe=*7YC@G6r4&? z?$~rX%&pM*5&O>v{RBhbzhhCsq_>LAWt}IU#>7xO*oJbdZ#D7!bb6Y+hsX^wGH-)( zaI_4p4Z1XRH6A_su*$~jg6!$-y```J^!Y%4|KaL|{5IHQRaMn&UR*A&5BcjuLqklg zIRvZg>(3GrTwPpLeF6$15^+MJM>rHb#o$pGS&=tn!9YUF zkVqa)es@M>I|+rV{8LpmH8tEA%?p?QKCH)iHX+G?Smk77nk4$fGhb@>^QLO!&@*hw zMIdiuj5gNSe-vBTJ2{CCHMq|9eEn+Jn)32xWWxm^A=1l?bab~5+x*{Ucc>ln`Gpb- zVchyI=lG~Qz>Y*mN7L`!uom4Dbj!!^rl+MDl~_g>B|d*nble-v$tmYXG z&ieojt|}Ed zPPZ;XB?tGFf)|Yx#m%qX-Q?~UB_wEa^s~mYP$-!`H<=?V05{PE(MS;rY9~c~oUBoq z)vcxB{p}UT0{Q|$LH5+=B9wLNUzjR$P!Y-z6x}^37wo8ny`^6V2#TiCMAzqc4OZRj zW_cZaCAwXg>Fe>}?gcVQ0>lUvnfk)7j#x;=zs{FZD7Paq{MJii_ip$HscztEGtMx` zYTOW;fIJ=&92`#5W}u{`gpdx3`*Y5x1E~<}_vSWn$r+JMYXHH6W|JDDC1xXU)G+@+ z_`n8mW~9@bp1+7fp|+Ms%}h*o^?s|+{_`q2%z)+rLELLr32 zn|dwP0t@V&aWc1KgwsphL!uO1v61mz9%4-7#5?L(pN{QO`G=nsiR3JVKE zvkiSMoJ7XT2hio*hlj_FVMBwmkxI-njJCJMFu7&)#rf8LKxkGWz-R z=gO1hzg7kcQ`WvVV?Llgg{wmy9opBgU$}6gtfoc|x?2SWw!@J$SyuoR!AWITUtT0H zh+cwZM73O6D(3F)&K68Tb0{SDJt``yrluy6QB;thzqO;I$~9 z$p;aHzxc6=c0Fn07cVwI;llWTz+A7b8;Zl#?eyV7cO?p|yF)cY?M31Wlc8Y0e0_bL z#_D9fv50a9(ankEP3j(srPHalW@f`4!x})L0ntT*Eu5N} z+3eHi3?D1m&sKr77R4s}Xmh^5N*{1r-}T%#Z^CC~AzFll3hV0u3vaH^bU_e6T-qfa zBb-KSHh;w`Yn9VsDs`bFopo~K%O`bu`^6_r1?tB*tW$oSD2Zgx=WU~bG7R%$%?j{iwiO`rxk!8!qztUHq`>B{J4F7cJ>GVW4|@l;eUF1w_bV- zL~A!JSzn(4rY5texVV@iJ<~o&rKPk@)W9tP2M5RgN0GRgSZP_Ayc4uTtiOZ`SM#5Axd`kDT$kQq)0up^!Qdo~ST8~1MCo{;xH zY7gZ>r`7e@f3BOYyF`%6%FIk0?f2{Yd3|Qc!?c{70Eh`lV`+G!j?*(UZEbCeK3h?)|Lval>gh0mNGI_H(4*}++OYuv0W7Sn zSoQq!w}cV@TD%(`TO&GYo! zd2iD(zp!vZ;r*MZ((SDMpDDW4Pc%zzJM}-_`xkZi7a_Lyk-&TA^9!I?hA&l~5f)Z_ zvIEK3%L?DSHH^COR>{5YC60>^M+PgMto~egor9)EW!8ht^A{@oLuY=$qLsdkvz%! zeZS(Q%BM%WX3=GpG7X&X(ES-0+WyPmb~@LK6JZj*gB`pXTQK3!$<@YAh}(8HXP8 z2XveMpLn>o!^6X&##ork3JQL1Z-+yoMUPP2R?O@@dVWmDA&kfrEQm&N?mc}PN zE$3zDr%$Shz!Z#WeFD|#WFh`U>Dbv-(@Pi3X-}IQ870QWo#ErF1nMR`o7nJxKcku0b(i#==5c{PLU}yy!1b1SdNNg*1q&H5NR!7EORT1cI>0$H&M1 z!GjnM#i7wr3&~wR35j(8b~`(QsJ%NCz_zzpASnO34}Xibb>8{@DrX{f9)Z z@xT>;4fX0~b()sOv4{*0M@lus51uJ<9)Se)&a6_zo3bo($$jwg*qHtd!*CRXY5XXmz-EFC^k= z8qZa4Z*MQJ>h$zcSOra7--RFV=!J|_)FF`IeyKl1`aZ_l3b)Cs;Naj02?@zZTv`Sw z)X>my!Vq1cU}U}+f{(VpOY8$6`N`Up=34!yPoLm`R)u8skdTnTZb93?PwV!4FtQOk z4}Rv?0cs%1sLM{AZ{6UfIS~XjX6EM$a&iJ)r6nYWDjZoaav!`9x+Qi{`r!kLflNog z2YN_455!$cvsK0!;sof$|D|ngn<7IVq{+FHXtkZBy9W;h@hck|jyAj4tk}w(+M}3z zdU|e^SiUPRW{{$YlO-pQ-r4a?egmYpIrI3(OPSJ6V-B}($EBnYPpT(Kz@7taZ`s91 z_H=g#mtMJa2|yQqD5w)n?1U6-#^&bdHDbOD4-ZF@maFLKkW!3Px#T>1_7whd@lOdoZ#|g%_U(O$9|s!^q_A4qY2uKVYQ}Bdtnp_a;Tyld)An58dg(0%)4s9U z-VA9eDPqFxyZYzDboH$lKlk*sd!8aGe)Gm1gqpSR32r1b@iA#>X~5;L9^J5bo5nUmv-kIh^eWNiP=1%dn-%9`1?^9J_`Zv zY`cLBn~(KY=`Mf*O)ag@U0vgKUbI)DI{~55(bE$kPpc6Vox{UN7QcHpyg9)A9L^S?&LnufT3&(L);mJGLTzxI*sq$YXCjQ7b$Z<|5y&|YYxCar4ID{r} zA_jpMO3oURbHaW;=q~`)i;`W=4RAeRcIa|qITQm%NnB(6ttzszIyq>9qj=AqJf^P-|8wBGHJ$>bz%~dDUm(uNjP0DS~9B7$uzuxwnW9AH(-%5aTozQGJ;;n%^g-( zi;-WEjwpg|ilWS_^tLMWVehGJNnIHIN-qDGMW6?14Olry{?nr$9{vJp*w4>zRBXkZ z?wB$?tb~Z@8%Ge~LpNf9J)+}~?~lVLeZomnStW zy{EG8bD^}c)r=zMRepzZSMcVFE$h8oLda3-hpjpj`^TgUbpO&p-AaN-6}k?DT|lz{ zw3@)!l>9C$Yw~cf0d^5sC14_?q@9=zOYDN6E@_^HJrq?;f?Kef>+-2r zQFmDtzD`s_%QOuV`@b;K=yiOSS_MT#zN^W%@7@LO+!hGooSX|F(1WfsNk_pZ1FE+S z9v!r)lHwRT-rn*%B{uOLz+wsjAU&+;ayB)6a-f>yRB8c+??`^NIm;NkeAdizP^*4uU^ z0%bu*N0;ftaFvdNBoDi}37BuV%#JeZg1r2G+a;)~(k@f2@bGo~&j*PNLV_b7a~1Gt z$MEy>tK?>lLK<6Im*0l|gp@QA@(i@V#-5H$F}l~TZLh9g($RrrBqb^N{KbpzPoJo_ zvV-F7FiI3i>lPEc?o?ZlvUV9pS1E{yR7_!KLXv**-!%pa%S*;<{q-sV7cqb~diob! z>LEn9(M-R;qW!SggY@`QV1e&sUQvtw}m`GHi9mII33t7WRw5%<>1tdSs}O zvQ7fV9Ri*LAXcDC%d<+y;nNvg3&4qT%0)IcUd{v+3_2bF;S{Q%*M`>Acw1V&@@%J} zrOmsV@(4B}Nl9fMRshm@`=rH|WtB6|FwkT`QMe+~zua!mf^hbAV4(NYr#=Wv;_~=-1E4UXqr0J%exLp3O|T*A zEu2ZiOe=yXFPLy$L{vQjBK5ITifTlEGgNfw_nig4rlN%iyjb2{s1ryr~OgK`Bfmn%9tlMVV_2$2Izv1-u%6crUA z+yP+`U6%o@X>Mf|>v_>}2wzsoXX|`6DxyjS@GBfYF0laXTkh^sk93fJn)+_QOb7@H zX7{?YGEIQxCAN2r_pc5F^XeQLRoDiU2^c@1mxwmgRgAk?S*?QmB zR?7eApug}|ODr|t*)b5`eO%5Qu;vy5ds0}qzdEV@8rLa=7HB2vuaxwO&z?z$iGhlZ z#A)P5!ve@GoG5G%D+>k&1|VeMTj58fZjGc*a4)Vqu>G^dD~6eq>OeF}KO7+5 zPkVS}F|G%$hLPVhH!%@7N{~tmTXmCN1pNkh8zUW^NN1S8tUWaMSN^weV^*zJ^SG+oX|ojcxG!g z2>|T@Hz)3fUhPsYD{$>m_FC+Hvn_m$wD`$@`ECm(r#Ar{7ZPk2(%quY^L`)HC+v682Ko0bEQDoLL8J$vS1X!@o`we8aejN) z`$YMbm6aIPM{x$`?I_yx!^1;7T-+d!w}AX6^1%(#T%GXjS?;wAKj_7i8iImx4^Lm# z(Q)?obFfA|N_4a=6~75@?~E92-bAVlyq!WTqbVIH_%WBipG6tlnV6 z{a>>pz&IcvVEu;?tGCD$4mRtXnd>mkxaE)dCVQf|kG@b3rPy2=zV*67lpX==N91<% z=8N)jNd>0%Rh{=z($a(k1l2V)4<0-K#}Mc`#CDa?Ai{wI!uQ0#Nrw@1-Zh`WS9vZ3v{t%dAUXP>14bE|NYuVndkp)+%*?riCserPuF&-eoH$C6zt=5EvDYfIbBvznaqcgh>=H@{ z0b|6Ewy5?$ZU_p}06)^|1=Yb-kVzr`0lu#S=KN=w-L+K7`v7tR2wTdaSCRt0$kuiS zdWPma@CM}N=7wAXIR_Lfp>yXxK3*CE%n4Bj3kQ1m6F7OIdRub)}%mZNf9KCSgmuvccy@%rTzzrXIm z3b$AVkY1madtFy|0IQo_I0f7YAUNvrue6}$ z#@!_?NV{Iya7=z8-^*qJ@KJ@ETD zZ-PHlTt>#&6G_yH1rG{TYQ#4xUTJB@M^P4_6asXIQe#~CkiAZUhbISw$TMe7jR(}% z`$AE4b-lp04X!Vur)lcnV0+>b;B)2fc&t$ce$gORUDYojKB>%GjCBT<85tU??EtN| z;&-|xHg^+{2QY48_v2M~z5soomnMTeQ2+4$D7Eqp7e;3L)oQlx$zdrbFs-7mFW~;R z?`1PdIbLb-4`H+ys6vH-5CBo7>3r5p6zdQd`o)VE`o_e>;qYATZEd@Idne8+@Bk3G z9vyhu+b@KL5e*is(s;_)Lw`R%1W(I_Uj6ViHEbZ%0rziS2S>=J>%*#Gtbv>5?Onj*r3vOE=fi zuU57!V!DJ*FYl7-;m^jHS&sO`sTncDxRL?x##$jJu zI5J`A_d3ZSe=~`kAu5!t7s(oGK_$`A(GeDY2OM5Yh`gK}u(f2g^!KcuCnYT`EI197 zzs<_xf_+wkp8H{n65ZSkSW|THqv?F2UBfo-Sl~B|ioo z%Ht(dz}P$DcL&DCI6lWL!9G9zs2=d1qW6R}jfw0n2vyO!d|5z1fEhyhII6KxnP#3F z9Y*!Fq^PJ4G8$wqXkxCW%bh61PaYD_`5|LSCB_iaqs>u%dhD^U zhpMCW=Ipy!+>he6%Cvh>?TP1;3i;dzhWMvPDja&PqJ^X9VtkI2HYVDC9uK} zJ?4HP{mV>{TFX?etEKL<(PAmVm+W+$n#Sg`#rD}3aF|;>*6ahL1NtdT{Cr1jdIS>$ z^t$fWSN5t?b7igX7bYTN#}RqP&I5ZRIPbc*BF4?7yCN%&EO3PtguHD_s_8*v0VMAx z>K|nR$y!IJ%U*9%g&ir*!|{Z!QK@ZKMg1CAk^UGK3)Sl;G%&CYK#Qo>0&4)7LXZdx z+eGlQ{>iQc8{={$9#=C+FjR=0NuF0j^uI1jmo zPf}o7kp{C>*B{e$m(c&{mf1G}CHe2$MK5KSBd;m#VkrG;e@olSs0NgACuyhs2pw#8 zpMb*G=yDmJ$M$2WIh{OSx{TWOeN2da#XJ0Qrn>*^Pd@yYO`}4bMFAR#?Il<1 z?hM8K9(+ET=I#4))Kx1wMGxNp%_)U1G?|NoYrTV-;lUN2Ts^ip_WJ^UKdxP0Y^-ir zod2ZRht#s&zQwgM9+YY#8qe|lSag=+!7-gVzu4JeUo>NY+xeQRdVG1;Mhk0^fAKfQ=;JpZsnXUos{n^_tXUL2 zQBBI&YXRCl?N{QSua*oq(F}=EC{>3kh8P0k%hCIbLe-j1< z1}!iZ5Fx5FWi*046zJKL)b?C;x2i9*OwHTAuaR*wl=+wd_`|n4_|7j(qs=q1J4<$BN~(YuDUlfeYB(-BnYLSb`=%D~WgB9o_8d z<1_pFcZ3{tx2FF7{`j;j#OMwOFoj&_15M;a@$KxS>fTb{4iN$8;u28bJ3f;r0=GwM zLBKZ3yZ(;yf^e_xlBFOTZ2}0X^L=bAWh$w!?;#Xba4KZGJRJa<#Df}vI8isi z(#ArIX-G|FsgsqF@q})y>|VDTPvC(~mlF-*#yqR^L${Q0WRPVV>OGLXfX2Aio0?Qe zNl7tE*ll-wIW8`o?sPH6w3|`5hY`5s*DNwG{*JWf4BcHw8ZnD^>%O6Te`ow?eLS=> ze+=9I?WOdxP&L;c3k_}ilbTW8>b)%XCyc1QZ;yX+S&{8``p0Vdm4ED}xj(md$e17f zDo66pbevKgityG|&6@C;@0PmB3wqKsd7rlOwh$_p2TX;U!e^gI-HG|?oXn|k^XB~> zg_-!z-$FWd_Y}6W*IqRilIdBPv@-v%flJ5G=d%qUf;P+lmIOu_7Fq)(CFJU z^@gKj*GuSw!JlIe^G!$oYd^&)XDNjJo+0)vVuaQ&-XF1pG?>*@mPdIjo-;LK!&I47 zX#RqhUJ~=`Gm4myl{@i!HpFX|_`Mw4npV@~73uWv4@`1t^&TXQVn5Lw>aFxg$=Hf{ z?{o+y9o_Bk!typ3*a=Z+&rJ$dIU|~xMZ3KOttO)G36iyFk@~*%VT?ZR9>vK2NUzKu z|3&DAeS{pVabEzKk=2Gkx1dmHaUjHU{As*1LBH=w7XJnCNUbCA)n!XdB}vH}2y9dTOskgSbxq#6H z=moME72l%(rIL3`R>u`p;E1Vm`GwBoUQbSD0w{#d#T4|p05yLb@+{cnFHszNxVcGA zX*>c8%)&s)IH-fYkJnQh51|p&o7}0)^aeWFqTc%^){o5v<|kTOTEv4G?wQ|ae@9*p zJt>{d?Q*XC3H?uG26;zVE0`7}CT5In65xe|dAz?VnFWqVRTqmC zV!X53{&yOTuh?(pQ>*W#jTPc_9~P}}#yY;t$(0$o;ejjRbuS}#c|<(K^cmX7xXXnEmfxuIu!gK@P!=fdxzNahMihbW&&`ehaye)cC|{G}2%Fla*pt zV=P6`K-`AnwCDnw8uF#;cJ&{BX( zUS3{gjZkv=>(_U|(zYkY4}?3*9v}AOC5ro8nyRVPpTLX{)&Nt#(~1j&?`_r?EheC= zo$W~BsiKU+#d=)`cWttCkWet#aKewFn? z+DLlCUeQX%a+(x&{GE9IvDHAaewFVJs>|jZmN(Mx&eqyn9$z~qYdJFYxbp;UxXBD7 z0&GtSV)$#qNjuV|e3QRd^#;T)C|oWNZnp7Cb#U6>q+`R>JQ*_TC=t6@YRR4MI!yEF za?IY@R-Zn|xkA@i<*r*Z{ZQ8vk@MG=7rNuMsV<*(CWO%5k17M_tZw&cES}g+}jaj+g~S&{Q&* zk0BviH*fYsS^_41W^wUg@#lqj74RDH6u{UIr-CQdvb~MX1~?TKfx35r7HH?BIJZF? zqMl2@9;8CaB&MRRZP(We(??qi1C;S+`S^4$T}lIsU8D?Z@uUd>m>c3pnLP&>zQTiYihsx=mQESY1R8`dXEG&pxmX7ghSXm`N18 zIV@#mz1}2hdpd?`MBC;!ql>DJf9k&chn45Vct;It)-#4F@#=eLULP;-etmY>9=mrJ z4!2n6eZ!2Fr?m(Cr9A#0U-p-CWgOF=esaj^EjICC=S3I8wSYFI7-LI18}hE8_^vCy z6xPAPxA=altHj4WZ?#)XO8k*K$c(rW#k{r#y~OcubRho_ipf5RdG%)oU(})7d=SnD zTEb4fy@TA6SG!Y}1NV+K?RuCz_})}s&}x#%D#fuheU_t~o+CMBP(`3e{w!i-YUJa+ z*eeZN?u!IA?lbX*G>6Y;Le(;tzT_shzw4~BZ9ps=I}mRSFcuxwvJE7>auug(qEb7s zlsHYppa1dV`hDr{TPar!;+XL-zq@KY^FWr|FZMdC`0Lo0Z`ia1vehTLS(kEkVvg{= zjGnOvXov@O*4xZ{^keu;wSIVJ@NUCr#(Ulg`bJErk}+4CZUwH~`C8Q*Q&Xt=lUhq} z-t~#bwACl_>Y0*Q1AO!2n~8a!9utWFuzkDVXkUD=R-cOP?;@zFueYGmty+O@83-*f ze*1%8!kUB-pCvn_Vr+1b$?~MP;p5{Ya&vH4oXzm3cxMi3`sWuypuUPD(3zmm2QlKa zbSw{7_I7o34Gc&N3+qxTvGHGdpOu9(>~b>XK}$muB)~?s0MNEQqZ)82OyhvipYY;^ zl*^O`WW_aNjtQ6{kx`J5N%lAitZn22#QpmM3f}{)Jhq*8*KJ|oqul*G2?dVJ5|6lOW)H&!|m|Y-4&bS} zK?mnpUU8!|soVAIFrEDhTKGGGaiG-SwX=(5m;ce+jKc|&K5@=JVcu6zK!BV{>=dqr zfq}TsmV>M72H3Mh^uT-x1`4oZM#@;Fp}26@TH1h26}ciVE>8Ws-gQ>5==d`m!oj@k z9=SR5X_K2bgPW+gwtyK4j0j6E)4lP>I_fb`JE!qh~5PX5=I!0d_o^tuWC9Q$>={R3Wxw z)33|;UCwM=!G92Q3iu&##A!}|7h`$oV`0}fW3X%D9_q<>WBE@Hf(~#Z#--vyt^^DR zW@u*AP|mv7(vXy-JYugkqY=5{0S>^<4HzBTjnn`74L`Muut7i52L{LoW#3Kth__aH zxaf1eEM_`iL!rbvh+v>zpEpCeaHu4jlc=S z%>j-iE-(e!kBO-YP6PP$!6QLB;Oylk8!`fXq>TU36A+H{yY!pIabbXq2^S3Tz=fYd zLFccm+z8X+?7(!<#)PT>GPcIFo z)Srp~#nh|~x@ttUkWo?7IdbWb@81)XlWYD0WDn|`7j&-XWvU^eG6zp>Ne12FAwYmA_%ft_(Z$Ld%&an~DFfg*WuNJw9#q#Q< z@r$7Ihc1z_f+8FeL`AHHi@3isT3ZS3Y4EO1s`@~zfwSdfSy^O61Sm^>=~E~)s=Hek zOY{WDOjPi{Sh^Kj=zzg)SmLj#nh7hY0H$$}^;P{LJpw^S_RaDBeWdS+=MBVbu;UCv zaZVIAgD!>8qWv*==3$fxW`*Do+_SY+4TFCNo>0oXahTP_eVLNti|x|}7gPZ4uSlxb z)xg!lxMtiv8HBZU795)Ibwx#AT3a>1M*Ey3Xc&d!mXLTfRDK^NfBj4m@CJYsEpnZS zoK%9WZ{4^tL1p}5^@=kcg&v_7zlcbP3P?EW9afEB#Isf2mol8V7tQ}Y39gEV1Yz@v zu5K_3sxzw*IAV|m;@V(bDW7lgqPY0lT5pqnR$jgl2nI0U6|g?#tU<%dN{ZZCU2Sp~ zA%WrYxU@7fN@(bF&ot=I0>j(%5@>ngDsh@M72ufxp1^uhT_bp;;JfE`mWxR8D*NXn z9b9td$6yI@cBaB-$pxcTa1dC-|4b(QdZ7Y@JwKW=LWRfq%9YQsbZA)ueEdvG+7(&w z2!k@;_m&Ojp!^tUXt)n_Th0-&F!6e7UJ)MTp+xX~{grM|%6wpXuyaiY;b@%s&71#yLyg7MB( z0Ogu2#KI}5im-&~&!{UAHFaPR1s{frt3fxku3XJv7>o)FgYo1I7y`F1b(=E;OOB*b zejhkFVc;{r+5(>?bB0;bTapHt?enI>-`C&@Tr{tPA3MN+CrA^hTH@fBQwTg5uAshv zf0%^NvUYm_tW=NzgI>UUR2KT`z}k5tuB4I@?5R^DE(yEQo)+_1Hi!HId4pa4F=$S} za1-zOzz8zraiEU@9J&cpP*8xniFt!QR51i%8fG+)uzAYxu$t2ypd16cLu2_O;MLdh zariKUNg^XqrLWMjhLSOJkHny~-7I){c|kwCw5-Ml)M?x^IA_*}Fj}(%*i`a>s?^fe?#``4Y@Zx16dfEiHwW1U11-qmh*E zA^8OVSuXD|DM0~K%BbC}#{45S!NC{?EvjuiLW+@EkKT?B!W3#J9GG?g0ca_nN&`+R z+0s{_*KV&4^4DFUPtgwC&^p==%_UPOQ1W>RH~L!*xtxUy!&nNopXoY<_VD`J;fP~w zVC7+z!AM3)6q9-#RfUnO9NAh&Tf}!&n73g^|y4kIbks zMmiLkRZ@7-uE$1WmOvwXG>zE@JY;efQc@qm@_1{p9`XeS-0S%jaXlg> z{M=6TlG+G&t_1*J5b>l0HGCv)ett)M8BeAb7WNIty;mo2@b{oEwRlAfGeKDR3>ES7 z=ZjvyZtd*E!09~C@TZ^PW%i9w@HMgy9bv*@^A;Kl;$D<~RnEkwiuu@MmgwZ|05 zcCG|oU79i79h=z_N3d8)LGVtiUc2T5M*xy$L1AG_F@!o9lUOD=2yQ7b{it>MrR9QT zc=!TNen?PIU3Q2eWW9q9MbI4&E0U<3>H}ZifX39Ajt=8-9t{+_zBQo9H8XZ}cVZ%g zg8)DOPM6iu)ZCmv1LM8Ni-Y1}_pe?hqX>mQNE^1m3MPbvgzURhrP|xuq2xd=U}QvK zf)f(H0qqJOAMkzw&@c$gFy3*}r%N~7z!C>wfK8fP*3aptq`NX*!v@5IH z$DlW|WD{3I|7Tu$yBWOI?-@Uk%C}lh5>%{yhhR4``3d%~=4YC?$7e}|K|%rVc&@3?Bh)5AS4VgbJ`6V{Wn{o- z`cS;Pb?a6K7i2IL`6Kw;3m6@<4%rqZmgLTmybq8)$eYW3U44qE&?Te+dQ~nT4w4*E(F~vEU;y*K>tG_L3y_DA!b$A~ z<;M+~tkkIF$ts=1^UY&0T;woXb2Iq@pj~Twdsa@)SU=onFtd!AoJ3kqf}AY22r-%2 zhzyaFm34Rf0JB3?zd`P$Rsy}v1xmK{D2z}1{Aprty}kGoFTCX4J74h@u@~^p+S*#u z!#gll@ZlX4u|=*6R{n=lLBGLfs+JMd^Qpc4bgqcZ$t)V^n&7Soj#4e5Cf?${m|Xwn z%?xTHiS6 z(FLHs_q+S08eZ7Fv;6!RDgXY^9}d8`%cxGyh6Cq`py?Zej}N)Sb7cbZpxSV7?jjhJ z!M6h*8*!~UlqD#q&*>inT-3OBt#@e1+m;WqpSry(q+u94gYjGU`-)-+RW-$|%MfqY zpXzPg*H!~2e$`w{SZWkCK6Ff6NG}WNn~j?P#!2hA={;dHPCQQ#Fs8h6J<=5CM(e%qO{n^E(|yR7O?^ zDE*+Z=bPl$RZX7>rqW(iyZ-d+rEAyVqoY6_Z)V7P#e4QFR0c|}60qh%LA`PF#>woo zjf$C}A@@jTo1Ea8A!fG7IJIB`rovywgU%Wn_=a?&A2ic7$q*zklm0Z97baL++@G+6 zxlb((o*KarCx_x;MMzUQK6p^DdYsQGZ?zM84<`_2nO320W_Qp1AHM!FuF7uv9)~v# z(hW)@Akqzz3KEI}A|>4@-KfN-8zdy8Q2_;!kd$sD1w}wox&);AKezXNzUTa&7th%* zZa+t4?|ohCT5HTP#~4!(fs}R}E5ll1t9wKj3d&W98#jK*ePZCBf!hv5&gM*Wpfdg1 zDA>_sRn4f;O(FpYB_~r~!dx(SaNs=^zmom*W{`LVSToHCV&&ViUMVTZk}Yq#Oo&Qe z+dr9CJCivZRvD-PHtel47suWGD&bcK-j28Ri+>JNoV3uS)jHDM_~RF^b4^4_lJCoP z$)U2<9Q)^)S44l21ikY%(ZAeC#=@0OZ6;HzMJS+_b;B#}SXbmN>6z@=aF9UO@n)vq z@9fIY!#awO;{8r~{M3@xmuz@HH+wvD%hu?b%eq=E+CF97Sy#bDL6J{%-ga9( z#4W;CaNCiRJ)eirayv*%i9#pz-_!UJ}Kq zDQ8q_PFB`5n4h46i)vE7JVg)=>V#2fZh#7%gh@t*LI3z*GxmkL(VKvMl2^VnO=u4v zRPTXan46oMfRGR*uQOl3ta{T67iEMotyL!CL3&O4YxQ(>p?t;p>{1uj;0AbMvTcR4 z8YQ1SuBfOeKE^w6(yGxSy8vP|qR{<=gU*>#*wJZ9@!$S{Sf_FbP?(MUtP)GPTy|2Z z>ds!!Mf@^;9jDKS&eZv&D5tGJ&EtcH3Z96oV6tY0jCaQ;X513L)}mU+^*KJC@2QfC zH=oT=%gY|$ap=lGw05|4ju{g%O3m=yDGMxR6Y9OCzVXW9^ILp&f-AdKwo{R-57zI_ zX>%L$p98DEY{84KztQE1FO@A)GHvWptX-_;?j>L@?)|bm5u^~U^^qMLPYX>~r+$84 z@)a?A^=*k2`#?ZnxRk%+#ztPv?&10CP(Ql@W&w}3&S7@q-Ral(qt|)$+Qp!)?>zS%mZg0c%}KoHKl-@CDjzv81^3F21-Gkf{yDcy z*mN-fJJj+)5p}0T!?uzw^Q~?Db(ECe!Oo5V5|Zk<@!T!o!~NX~K<&v$S=NkL9OwVpC-Q|dU zT{;eiJHaSBI!O-#b8LtL0cOujBEB_@J~$ycnd7?J8%mJINRG4I^7p-Rw-?L2`)ih6 zrH+)R`rSLZ(|7|l{ZuaMvsW$#g^amHw#sGK`svHRdfYhG1%g+iiiZ(1?bLj!&K2iV z5-**si*lCvZ`WS|NiAjzZ)6%5J{o5&j&AQ1#TIof%rz{Ze4ReWF}tx;NU94f<|F7E zc5;iD+F(Zl+AM_VbnUJAjbG(_ZTY6&oI=XCeU*RH_iW;8%l5d{lG}I!`CXCIgZBeJ-4jea(3ewcbri% zUvPclhXvdC^S#7a;7jqE*^j=KpAQ%lsYCMLmUVO+oV{JYJ1b8~YVM1!^{#MF;b`4QkZC8qCV(AxpAd1wiu$v2hY(=i1Q zChW1U+Xm7vQNYQ--!!|;CPR`Hs=oaJ7OM|?vVD7*jQUHL1F{I1FsJy^l`7LBJqW_y}i9lG)Qc^ ziow7Ms9TWxy|J!}s;UB680PO7f~EQSu!|9XSe_7B5*^5IFTeZKvibzVjAnk_x(>u` z%RT2=If5(2KoF{CT2a21Yt)xm>+%eY1rDFXsZm>_h>%;;b)?`X#<`$pN7qkl0xgP= z7>4t0?4K(Pyh4>Xu4B>b(X5s$`~H!`E&aTsQ}!S$mVQj^Pt@!=VR=W?^4f|Z2GMnf zr(4eS;)O5ccF*pzpMPHR6WC>)tTB&^a2lZ`*1q{!i!ziW)O#{%ZG_F7e^H&WzB%i{ zJv!RkJ)R!(V9-j-qC?e@jDqf0ak11N=j!0Mv-3F^7mJP7=ff1wr!#i0HY|Uh)Oxz~ z_RV1CM1H$joi32M0e|}C%NIDTLl~#)pX;Ph z5oiLOSOsVFQcm;iUxmD$%}LSkZ`(+xf^7C|A=RS+53)8vnllUDM; z*w`58Nr0+QK|l5K!WD|j*~A4EG#0)RnGLZiO1+0vRZ>uxUs{rHfRlrOQ;GEm!mTDa zJ0DMd_EZ}U>t!nJnuQXKQN|m$X1dZo4H(y~EJhX(Icq+AsBpDQ)ry(2n#^Ne5j5okc~tSz-1x3ibHvo^-|ci=9+2a;|A9O4gb@1rX{mq*k4O55v##q`IZ_$zD3S!|%coMFhG zXyHP6erzIXYUvAWx{JDPyqs6RJzD6<{fzjni|WtGN;cnY%SzDleSeUGAKyTqH)h=& zSV45*{ab)p=1&3VrE$wDmzGh+`pP0J-%(FK7uJ3keubObXJ=c*ZI+>X5wvuyw|qt# zc%6y{K8X(LiaYEQ+iLeAT{~J*zW0ddyAmvAh$vcc;@Dhy{Kt{@@?&n@E+NsC z0PRT(uA}b;J65rJTr#d5-)3LgKE30R#@}h5yJ-w4($`O4&CL}_V!0a zXQi2lSBO^%h|Q%#mmc<;mB5xlaH@KvxfU8c``|i1FqM zC1raM`wZXs^>U9~@RWj+ZB$c72j`PC>>xHa1<}#>?z*R32fHu#o$FYB28$_0XZ)%L7Z6&LSW~Y z!{U$U{75~;mMKs5T_8A}S!~C!?KzuG(FU`ZZc;WNs)4lvcR`WXjIlb@Di9U{7S7dj zmt*K2EVu8{NTS<3BNDd99Q@Bq{s9~spz?hWQv32Bb5N85csy2bhxI zv%1lwvWw^X_3OZi)`$5xv!bfGH|T&t#v&t#+THGVqF+#F1hgVYRzTM(v7@_h0*Y<%2JOtmVaq(4EZhUm-(rxY5b)ubq^AcT? z;2TvNX0dtZyNRvA#W(p;aUJNKvn!)^d&zGU?3S;&dF!mFdpv#}`R+y0Xzy8?g0WPY z<$%ON$n335=YICPYYe)6ap9OGV+yW0VKd*f(CK>fxJ4v)BBu7HmbND)s?Hq_u@je{ z)T~z}(>Om=(J-Sbi`NsGd}u51UhkzfrvFi)p^1HSBjh+t?X%JZl=oQ0C)y@8Zy5MF?jemZ`s4 z8pF8JA5Ee?z30D6S{CNM3_R!^W)Z{o>PnZ*RVT4_^B#B=q<$~kNi&IuYT99`>P1=Y&CKBcd;8nwOVH|Pkk%Fd-cmq0zPfIU8!e#8=m*<{Z4yO-k)B&4_755j{>Nc??3V!DIthi4x}TMgoOJXb!m&jUWs5lh`G1w5xK zofp{a4V-T4KE69RpV~+Ceoe$%8^5%A{HD}tJkfOQ$KDuoS>du+B;Eh<1d#}?+C+%3rzBHC5PesmYa7z04e`RgJP<0eF7PV?SFj=VVBSkiM(@IJ;K z)>pLay>~zKkli!os4fxc5#5Q%yl{BDy^u~v?t5dEJ(b!P_kE_MtDE8{-W~qxs@sf4 zE1k~*TprCU=Cmh#vsgl+kJ)N?|HPd6{A}zS@j=(!bj#b;+^Mp&+9irxHzn%bcTA(2 zPVvY)&zCo9FV}38=rs>WX1f02|0ev++J1*s|HW=lsq7_uC!6bA%0($5nu57*T5oDP zb1vP|{BBV#GQTKJb4e=dE56LNfr(qb-aRiuZKGcGrsI6AWw|TCu6C$m6ovnik}9Qa z^4U}OTFZ;HwjcLDV4$TH8$V2N2>w_>%b0mv`st~fGxIACZ1gQ5-Snr}jOPtsj;uf6 z&qZqp*FAY(AufGqX`|DqO!@zA2yE-h7?^{IigKKVj@TpSHvRLWDybBg-hxfOEcI8AA!|y-Ws-H|0Fs|=n z4e_m&G%iRE%7>W$;-AucE$=hWcX<7NmLMJWx+f`CNkD&W&0U_QG%24iC33MM+sv0u zLq}LzGb*|ARZpL?T_pC>NXzeGxmGJOx+ZrpHwnB?a@S(iCD0vcD4y0>4|;`eJS5RF=e~8Z3hzafCXC;XVAYVn`|ffA{H3qyRYOoE-m0-{dZ#(eXrh-$$G=BHWa6 z$jeK_mqKnnON@!wwJTx2He%cJMDmQyGCc3)0r$Rm@dLwFp2lZ+tFOj1I39jU=p*>H zr%(5eST8*(n;5y_eJ8%WBl_Glv@c@O?sJQ{IoEpJD>hca)+emSzX=2D zbX(!5QPy-HAQOJtt%kR7UD@f~7pIR28FLG@uz@^12>m%Ent#w!oRG@OF6<7Eq(!{< z&BDFXz-eAi!Xt^HmSG({CvRQ*+#X_O%ij+$?1>$d<@A0QC9M_oCXi(|bU(yxA{6Y< zx!tW-K+#3hMRWt_Mp(!;l!3C1t;tQ^uw)-C_Mr7TK$r?f!ka_)j~~ zhi4fVCz8iJ_|;q%K8Y0OeWYB9r_3<5W_m$V+rYNj8nB`xu}|(^l7 z8lRX@mEl01*CV2LGv4-Z-bsJ{MYiajm;U-U!%94<(2VQ{+76D9>1Y82#k0#l&6fJ7 zUeCxLemt!B>Z|Y;ndS7!fi(_+9G+R{M!zsUeuE&`d zQ^1|Zx51YWtmrn7O#1@}Z%7x+Mm!4+zvy%aXCo zh)+uDptJk|LID8E(6l~1I~yaj-ufmO>6Qna%#^Fw*%MH^nqftGQdBlE{ zj(Aprw)>eZc}OyEC0T5$v96SVs^GX`;U^16Qt+fDuRHBXu(Kj@uKSK7?e{!=fvUID z6cSo_k1ULT_<0YPocoKc9bGxw^p=hK`q@(jAhE+XeZq05Xoel!;qAzBB#_TF^1yF~{^f@?n_Ho0V^e2clia_>YK zITXz%Q0#~WUlcuDGY|b^QC_Dj|M1eHh~-%3X!+tDJT4G>*u?N zFnsv%6SPr*)%)5gf~~sj0HxZhkkD09xr+t;ZhB49n=x^%jcJ0jOSGCwy}$?2PS0+}(USo_md?1n=n|dTE@9T&wTVT2UzO;htB|C* zS_l2dz09qhWzy+;8B%tO-$=dd<_vj~b*3-2G7|&D;7Gv?fHxIpBLW~mpJbVZ!}WX?3LEP&Y2n2m{y}ZnRAPooe)wC=UoWf z)$ui)!R3`*W8;Hue0+WuP*}{BvsQpEDUP~P-b1h5gn#_WZcEh+b-m9}>6l>iH2z&+ z+>H0{Qeas&v4`U+(QV+h?2lo?#`YG{A4ay)fz0ae#!+lk0E!cH2vRkyOnF(i{2?{%kcOgX9FOdts%p{u_vqe%AL6p> z>oeRb7WH&>RZ|7u*45E*#_@=o1N~HzRk{O&7|5RFJfIW2O-)BCiz4J%#Rcb7PSzo$~lW7+5P7jz4TWoFmSA;oaY zVJfFs!UHWkZ<|OLJ_gNwH`01n2Orl;`>?9IrhstlCp*oZk{jA(vZjFYz%7n=sTkt0JxwLoL>lH_5Q@IZ>YyHXBhcVG@)B@0I! zS2>zYgq!WGB=W-`V)YbJMH){x1=;sZcnv_duF_Rm?&MYUKay6v>5rJYep9Q+jnTVC?t}+#oMk7YM+f)T81JBG zTdUFw0Q1Ln#4)#NHsuUO$Bb{uNg}asV;;}6|394FSuEtGKnhw@d;8n^`pKRie3Msk zasC+}F1SL>{3PWtgdjz5|jACF!LIt{s{k_j*?=jB zUj&+h6mb4txm|S&N7rWQlSv5bv~g2w&jU&ZlS)+?*YF~5@EzNaDbM_?)Jw|Q_Pcmm z&B5fB3Ra06Y3|96&tBU4vB~Y-$$l~?pGG}axtBV^=eo|zsFNx+%fWMdYDhsdz5m%M zUAc6_V{zA2TzgBS`9ZzaQy}3KEvI0oz|cPskBDO2c|FbhF1cpk@GUQ8w-vL|A5Yd# zVySr1j61CX66O^YOD+R6BBtxHR#&JFtbUK~tyzEHa3@P`7HD~%<^`t=;3ECz%jozV z%6MuRMUD?M#h$hdtR%aqRt^H9oOkcS<+~V-VdpzW2)jaM zS!M~s$Q0tM?CfN67mC3BLE6R8X(up=BP?Lp#Ct)o)~ zXRVFJx^unEv-jCGbH)j5wKq;Z@6J&SmK3j^x|dC)CxrGbTy{O*3YGjqNNg`WFEjs+ zTSWh6fbDMArnO59?v}2rkghbl6TR}JXMWo*XRBqqRY#}|HvJ1MTAX!5jE0M)kHnD~ zx076=Xpb^>->-yR#@~86QLyUR*^@pJZh!^+vODaN8g3mv{j$ey#BGbqI5zkM=qDpV1;B5FvD~L z!#ec@bh+&u9f>oo`qAY;MoFjM)x!?yG!|fupM-cS(3K)(QqSbRgPdwk!2`+5$vHbW zC+W4tM3Q=z_X>naiHQ#(d>J2qMtD$Y(4!>?ugkSPf)ex{pp!9A54ZA$T+Txe=!|xe zC=j8NJyIDZ%Mh1M2xTBg3sV@9SXz@iEh#s_h)YweQ25T%uZ&<#o?o=+(uUAC?2Ib6vidGaP6$ zIrsVP>-z-{r^=VFyT7ZoW-blbnV~98mbiEnzr7XV0FNyDP9cX@h3lI}m!TuV zmz-1?Y!@xlz;|zkIEt5A-ro8{(lRp^MYQUpo%LuD>Ob2tvEuJ{ zzXr3Q(BGSnblJ35%_eoMCT~H%L8kkd(TuX~bU3v;;>IG>RA3Evz**G|q6cr(C>je;lEBAsvp;psBFJkG6%F4VqZ`74(m8GF^Uhl;p zL_vo34=;ZeeHl#)a4YDzLGKINBj}$~4;6w&2J*YuPd-#Y5E1+Zc&{+we^^+PweFEe zsZ7ruMnNndJ)tgw+A_2a4W(&*4}2vfLVe|XS5Q?G9C!HFSXt%e5=EW(&_Fgr{J3wU6P6Rxi09P6>`!VZZTK!xb0W$Dc2?+@mZMnp9 zLJ!o{2Zx5PvFMD6c|!c(%!~OC|e;?=iJ4G4apv-j5+vsAwmP`q5vu%5EIFD05(3n%)rp# zvT%pL6z-&x=HJ)}i2DtHswt2y0d8ryPP)3fnwpw`d_lxHcnW2ej4k$6Ha18zZ{?l^ zU6FpH?-|6kKvbtZ;w&E;N~rk|TS7duH2g4H0<6jpXgJ``{QE_Zub|t!&~|bv2@69d z5FR{4kXCFUlNd3O>}}v=uq@Q5-X^K&-I`(*8|tZ zb1$!5zY?aE0Zq`~Ln*$zAcEQ~{(ai~i$kxL22(7D)9rtp#IMTmeVp38 z{)53(kx-x*qJ2RBa+Vrb7?WIe)}W(KV3UUayuBcL|7g%*Mo`%-s^?F9_m4#^usV<7~|}g z^+RXeJbW9;zCCb-kDI?;(NSg3=WdhbBBdc~bnqx^wG+$v^*c`V(>#RZ=A=&cAurn% z8(!Yc>s?~#5pi!j9W#IN0Qzlvnrq|u=~FMOnaDdt2WN@WD9x38vgnGx>;3ATN6PzW z3PR6&e~g`|jFJT>|?8ZoHh1Ca31TuZ)fP# zD$5lzxw`r}%5ms^+RN{clQyu3s#-_Vij2E@M8+S5HnNNrh1y?2OVUc7)9aV5nUBFc z@y+8FFFE-kX)0SdV#jHyyJW`jYvVc*MpADY(a*L zwW-)>FkV4{9m}w7_~^}jD8#LvMED;Ri6Um zn671wRX*Z2SDJzr7PP~lxR2b`=~#lYER&Z7x+};)Hl6!vlD5JGJyBophL0l5lBhC0 z=Ay=5BD9oPZ=SjDC`BLkw?8b984j7e{X?y=FID1>d`#!0(Wt6z=`?S5h-#W=d;h58 z>qd(}vX7bLr-vkjhi`uSi^N=h&nNf6P&GLqFs_Js`1sDu4gGqy`-%Yq{_g>-XXT|h zs?TRgxGOwMB=Qdaz%!jwKe&BLdYjAeyv6JF73*(Di_uE;&<4Ln8B?or*_L;FqvX;~ zYE}FF|Ey`uLE~7+i0$j`J-*0G*j?g;NJv0cKA9WJAWSg)IhPhPoJWB38maq-)`_95X8(vT~%uRO`QFlfEUv&K)W;gy!lZmV_h+~w7#y>9jmC~ zdVhqtd)oI1X`(`8qsao$wnZ$LenlC4NXi=L*g0*mgpQ6NW}~3KcbJuHCH-37;b$^x z5{@mmKS5d0mGc-cD!NnNrlIfSuQKO5?offg6BJvE-c@yE^D@*`1gBRa&nmsg-{> zE_3O&7|bbLD1(OJLbuhEGFQdeL*9!d(u6Uq-(6}V-;tDWlu;|z^wQfL<`jOzY9rq2t1(5U zOR%32>~hzbn`JkcurJ{z;x_nQn5u-&4S!y|mYbhk_YpB%9YceZ21zCTKI z*P7OK{c2Dw;Mp=_{=gqHqh0R1*GP!_oQ>D0{eJZyfCy!W^3@?@>_MPrzdR^k8en+7 zk@yHmAm@PYpu?kJ2X3cnWw}l z_jBv@d0EGOn(QrJ$=Me1xwZW7byeYh)#QyP0T&J{e-wm9V)1tzF1t9#9A-71F#-X} z(4TvEQO%%VcBQQGON+!@-aUqp>$aS|XGfhEXcmEDP(vkX8Ikb5S0nMVKnj z9Vrhy4DItO4m%!KfD-KA;{y(mk9R&Ho+0;o_!5 zWUW4^WN+pN)JkYivTFnM6&oDlldc>^iBLO~8*MW^N^bYQIPNoehyTJkvc2q7UyHPKbVTa$LYZoP)9hYXK#Sp9hxdwv^_JnpU_zQMG#!tyB5B+5=G z*Ztr|L^jjl#s+`i54$&`K==+@-OkP@Sgsjpl|FU^#mB&U@@*bCO@6N68;Iu~UO&k4 zEW7(Vg>ITBdR2FAt2JdpHxTyU$H`(XqOKe5O$M%ZLP<1M)_LW zwWnKJz(C&3#Mt`Ltd4LcFmXVJM!ha&dg6Q*HO~9oRB7&Q0hlMX5`ThL%6rVvdb&Mq}S~`rv1)6z%J!^3m~=H(~b} z!RFj{{4>9;;jNC{IR(>x)#~*3wZO4W+2fQL-{5_#^N$;^wah=F_q>VK&U)(5VST#u zwYDUG#T>)?&us=PrN5vaB_oht{`OascJr#(i`C!16$v;%i!&Vg(B_}Y;;$AYPWdXj z%{Zzrr-J&-(+IiYXA@{6Fsu5OVG0V$|E+)iuAAkY_jPqrFqo#JL-COYzX{C1-rnDb z94>bFwf|R5{C(qHfb@Ern)0Y(9#F8ulP4_yzdwU&sQ(wlqY54s&|Lkec7%#c_%0~F zfER^^24=o^^s0_~{5uU~Kec-9oOatF+GCb*pc^S$7Y z|E(YY?Y;fK)e5R-_UuIgw6TPRcY%p@`+%bgvBQuMx3RG?{}*=sy-5GQWhU6>U^orr zcrD}DW8{H{`pp}7lmGc-&@BUf8P3FG4#F@|yHUBj;;<8Dp$*!{(%1Hg?L8b+{JB&a zO(Up=Kp+t+3O65K|K}RZg0U@|(f4rnA`V*{{C$2<(z#>80oIPFy-_38{%$^o*8bC3 z(#8=h<3Ddm_L$@yL^Z-#_E!gTtI(Z;RQLHEN4>_nP2cIWeIHL8`1!mKOOu1S)~HHW zc!f*n_U+)%e}o%` zdchaThWZl9m;V0yF73#7;IsSmDbf2r>WBZ{$v4dJVD7P}=K&Zk;L#H_&i_;g>o?K2I@A!jl4kb1MU)BdSFmoBoTzG zss*<4tkOjCrvXBs2m~eq+GEF6PYp;P65>IA?C~iBz72Y-$;PH&QCZm^ATP3 zd1^i*Zy25ct=A7lMKTdxQ7?4C&u}_!ElXmct1Avt@gt}(9YDPdob+Lj3sKietdu?i zyY)H~cN7ds)c}7!Jaqfed;mpsNrb@4LQgWxX@R;Hz6E>*5SEKS+gZHVL4c?NAw*bM z7_5zTQGip_7|+zLp@cxj;veQ8ROl_Lwz?0?qc zO@Zryd=Tgf<~<1224W&BA;H|V3tl)rTh0Sssli0QmflKy|j-hM?j!?+o z**!Q2rImYv6d@9L%f!{of>_{>N`2&wosA|2#W$qWE<1qq7aolg5)YOt7?uL&QV1%e zO!58IhUaZ*e2OGDrThbq#8~L2NskBoceMxVfB}DNcNfxvv@|s(A?^b4>B9%YB03V1 zrtWT9Iy!{MTH!Wk5}`FkK3p;dqFse^Saa6|fas~1f87Y}z4_VM94Lvs>+0)GMH6VYHs~=!0*Up#BtIOQ<9v%K-d_3k$p@` zw4!;-Rw0@Bzl)(!7nT8>U)Yx*C7~qk*}-o^X!JAjsgw0~5)FclcX4qMtQgSu#c*W= zbEL8|E^bUt2G$_ZTaZqp2((do`T;DAEo_WI)pH-jD-AIIElV*o!{zW#OFKb5VzrbX za&uuq4;{42QYQ1YUQ+?r%*@OThs-ssWP}I$-is^CRJbQc)Af63Hc6%hG6WDR4xS2a zNcxa$<3|{3YGNosY&XV2>KPcZO@4!nzbOQ?3;ZvG)k26FiSm6u2@x?PJHEFWyoES# zFNY>K@CQK?{B`UL*#2PWgV(O{J4kPT@fN5pNTL8gn2)zNs^!{tN+2ieb4+1?`=HKk zg{&V-+vu|gkBA6mVmJCStSKPkz~7w3Y~ts40qt%d>}(J;!2AzKnBZ{hHfc^y&b0+0 zdY#)-9bpqfM5|lB@Ny!7#5nP zrOk-!&;v3I+JitYMwwMaT2NudaVh`jdz#GVz|IH8aWKfDwkGWCZM0=>7^q-vO?~CcXZxR` z;9oJ}Dv8G$G||iE$=-qchjcMDon^b5#3c5qX@1_KJ(B{MWr#oyYFso$(g0& zu6J0r&>=dSAjSDB@Px8nTL`3AwGJ4&e2UFsS|H@QoA9TERA67uH_4n?T%@9=#&KF& zSh!8f#Bg{|U7f@imw*7X+3#S|6E;StbYcWizf_@Q@sVq1tgm;f8uwb+P zOYx%4esxfZ!!Y7%xJQMB_+9u6So3HkXdX7Vvp3-2fJ|&u_yi^;>627XcJmmtUQnpFtU#LS7NzDTCtBW?6{P#fN(vq3+G=Q8sk#-{5t+`j(5~9WXCh}^kUlC+1U{8%B_NHF(nS{ zawtUb&m$qcGKV;WOachjy?d4z7boxuaT#3zOm#*L-Zj8Uu2nnt)a_d;*X=A6l0Lc* zWc&Y~^#8zz{})O81!ZsH`Gkw>-OPxLKmPz(IB*^A+WrFtQIBG{@YQyymzG^Tii|_( zH(->orZV$?j@W-O3F`O!<(^S@6V)DgrC|0Bz8M_2yP})^4`J_ser0`b{Qm+EU}xb$ zLf8O$Spjn(`cKN2WL z{U4Yf@&-sH&VK1|De+$M=c2ZNlVk$)FekWtsr3q z^h^-O6k!0&_O0Q|MyM)`Em{D$dRvK1(*mE zA;O^wS_=S5b>6|SMmVDOt;yiHh4vNLM$`A?t6;zu03pDJrP}l}@@8hgpbdl)QhkAp zDOqS%RaCsJ$Oo4Z1Wm4|&XvP()D!c3<3?Yf-xCt)Jzwfzh!G%Mn8Up8c?r@}A*u%| z4Bh7d>r6PFFT!z9fd70^nUR9xp|*C%Gb_43QQ_e*M?t1g>Gdnfe*%027IkaO7){h} z@@L_$wzd<+YD3Msq}z@>z^N7d`kdb4Bw4afkVg2oTfy*6JHj=G75hbU2L84UGS zy?@_|<2I8O#B7qJR^Vi5&`C-L*f>+!q`Z{KCMOuTwiFE$h3)^^z&Hl}+^=6Rm?e23 zzzIXdY~$uJL@7eDMkx*TQyH%<$RUA$Lq<(Wi8eU)w7#;EN(j^AO-LW=%b(F>T?>s{ zUrWjN~E^;oCljn(ya3%@9eaE0R}7(exWd_1DyfuUja3vitW5<||}QT@PG zC{bX(rlW&{l{9k<>W$tsl4T3D*iTf$M4=;)T!I`0GQ65bE6qEJ;0RMAbQxuY1R)4_ zfsUd~DBJ{b2OUnD12L4zNx zRiK>!RZ;GN&vmA3dp?NWf_lRzQ{O;G#~OqgD=XonEm9k!&CNFq; z=l#@wZ_fEk{!>`HFh&Fu8=J*U@%HU|nNmlCIr32IAA-gVDGY}#^g#*a;ClhigudMD zHFAC0mX5&(T3YSi?KLx-P=`PQ_F3ckTH_W#WDNN{$QFRh2qO(!CI&jPX&{vfst#D8 zBdr&HuV0)4Dl4H1i`AfF1@U54Xs8kx@<5f!{}AIfWO7CL)3Q?aw(=90mo%1?Y{4%A zn3BQ)d2A-QoxTKi@C5-#_MW+Dk-IzM|}asp!o zV8;huAoabeFqT*o4NX&BF~N%IJ{0lWXjXF+x{+iz_j`Rr7AZy$*gxS*zTh~G8lJ6( zCLXsz^%megC-KP$R>JGdCE0qA+|&x27$hX2p@nsKc8014-gjOb;rZn1h6#yxZ*KC* zV<;LoU8O=pn+?Vzg*prpqT1)Fq8agC$mHlO^FdbHa*b*}DxDmTDVUi8O0@3XWJt7u z_?aN;-Rx=i(6k!Pe$~CWEB%BpS!{RovnQVonaC|-us+iKV*V05294Tg4NTbKgz0YE z1+j@=TEt#}fr|m>q|;BHVV!8;&xmk00uJ8yyIwbA(0c?0uYVI6bnm;dx!C$^A;Am6 zv0yE)@$fkDt>2WB8@(yzuWzyeL4@kmx@6SUp&8>+3n!V6-!TF&pF7S3@V+T~U}Wg`DQ5H13MlT(4{71TTy??afEN5Y!)_AMJW1CR+2 z2Xoa5P1plQYhegO*39dioKlC$T0mMJNJXZ~k`mo=tLVVMR;V>VP)397D8YP)wF3N6 zpl#|`QpuYM@Sv8U)oe^RGPWW*q6Q{bL&j)mV2me>1A7a&&(3uE)pm`j z2VJzck+IHHPEWria?%u!l9E4$hlJnDq`I9c=q8pqza60S{MSZ+L${h>8`ZdkEwrEP zD}YJNfy(nER`-Fk!rRIVK504hxeJIoW}uumm%AMTup2bG1$mW~`!L>v{~Lfb7=hdj zd7u!9V}1ckS_6|Vh7(OlH}?r-aR=&bQ&KBK9juCAb;dIx%afjohuaRb>}MXFPtoFr z?^@$ZohCp`jftmt> zU#yEp18c_{o?_KX(#XLR{v{Qd<$&S{7JffPYXuyOTZH({&WQqersUGphF9wF zS1qUd`>Jv2856u8qV@z_zOF3kx+V^IfN=a+X zE*^Ln8)N3_w=+;DtbdHS3Y)g$WmLV^+uJLbE>{mV%KlzoQy*v;tgNl)${}W~gN#ZJ zLrmZrEKcU*Kne(9eSr~zLqJKxPK&&M_imbmE4$!LRi)AP<#vQ{g2W{=9Gj$+&#cYO z%{15~4Yb!b9rr=GEx>P*-gFJW{Nz+e48$XF_h@b*>u3ZuE)6wV5io;O)q?N}BW3)o z0hL^N(EDcoC@q>G{({#dMXHFRrZFJ+FvbKdT0~(_Ca@F7VioJSdPp+I!VMfpvvO27 zo|u>zJEiI*TAvf2t5Sq%Iu2-qk8)K z_$)P^n+%h_?1W^KV4{zv?PKtsz_}D06f_uKLcWbdA3>ijfT7T+7>aHSn|HQQuBo}K zaw67X*zKl!Zo>0;m^iAew1us?f#n#&&wnV=$)()V#o!af{pn{FV-xZlPjqAFhJ%Ul zF;m_&qhU7^PxhG^Of6T6&3(>tiJ8UU$A2-!9OH>80sgPAeWd&lVwQJ5BSO8Gfzq@C zEZL>xk4*^Htrv&3V2~I^jG~efD^b(;Yzu{3B%&q<5Cftd1rvJXO9Ypbn?om2)TwnS89RIbf{^n4m&6)3M&S9&SHHT2b*BGia8~A ziP|I^ddeeRT@zJwB+Eo{Vj{oQ9Ld_E0E-L(;)!|JO>M+5NE0D)EILE5v=IiXS1jks z%9*Rdlqy9|0KQ&$ffEodJK9Nxoee_Al$-b>Ez77V6b`^#rrbEHSyE3Q+`1KzkWdbp zWeCFfke{?dGb$y5uKBc0^=9o65YL32Ex;=n^7hResMB$d%pYC>#a}V{+|xvSTbMNp zh$!$L;v*E*pR230e=xNGA9w>Ix+X9A%nKo+C9YTZwYFkl(oGhE-$TYTo0M))GG#$6 zW#mhDH*>uArR-ETqb{%t^x<-Y@)@$kaH+G7l?ZD^%f~UTV6Q=qXim_wFfukKPV-NZ zd+Kqma~Bo*??bM1Gpi)Rog(A%HV!FeJXKxruj{A%0n;M$%EeULX zU@C%i(!GGkic;x)W-vlVL|_izZ3n_6wc*(UhJ`@RoZ==Ewvw4sSw%(rXov=8bB10c zYW@aL9#LA2+l;|9Z_ttv_kd|mA-pW({0mc8l#>&0A(tsijYd$QmjAU#JX&60KmdcR zZ&WTsVe;F+ZY0nX6NfEd0l60M-l2fV7G+>(sd?uNIgQ6sKfTlS9)5xd8cEaZ_c6O# zbW(gxoIurY07KTX+hJ19`nQo0MN5qrFntXy7)T0GP+8;H9MWOXkY%^Qt_^1uS$1qL zh-zmnnxwbr<%oAf@1{x)Ze8~<>$M8GLEyiIAtrVJ2gulFYMgT5O`C+2Ia8u=hhSQ! zS8K5Rv$GMQ{9=j+%@4`RZ=+P}jFNIN;vFj-1X6*&OZ+a*;dwLng986p@)-996X?zQ zTig=$Za{QLwddy4;h~q?(R4!-9&L+VBKmP>8-tbkZyqKlRHh>43pliMnB*jW;pGzQ z(i&c2Ce0BP7pLic1rv2k5b5ohh{V_$;rn%vqHCu}0eL)_imW5d*ttJg_se9JF>9Fb zm~f!8dJ94?A&4^wcOAkxmE`0Plu~XbhpONro0d5>^O*dCBzv3m5Ye@*QoWxvuH};d zBJ+-vN7$-%ZBx5qsQw_$tAr7!#>|UNr3JWoHwswmBE-XOVl%qv!b2z23Q>&5Zqqw3 zWaZGgg=yT>)1%d3kc6^XAl|>XDURm;7TTLI(ytKr=TCWxbE!>llz5_T3sGc|Y14ca z!Np3&S{A4zAjcxc1ak?l-EYI0kNO&?_wkf5ExHs3n+b=)dA^I3z#NJ?ggOY92@Gju zQ=0J}L&=7p)ZP>_4*Gv)fiB7<^L76?ruFaj=!a&y&&CR4iE3%01dIaoG&G|Udo$9Y zw9IWldSN#RB$}yHW9pq_?CoO=}w0ouowM+7?{jD&bfnjv)-Y{x@F zD<-k^<~l4*^oJxSXf{nu5#bTg66tt+RZ*J;UBJn#ZnHc)SmE;3TU@uTZyI9^F`H%Q z1bp~FF$@ z#?YAf3~|#j_*Fa2RpqgKyxkhtq5ITMhv13>Qlc2#vGpl34`>9;SSM^{BO*y#EHvGx{FQMO&%@X%d?lpx(Tgpv}X zgd(;ABAudygLF4YgD@bagrX=7(p?e*igZc~DBZ%pNAD-z@Bh|%*T2@k*0b(sK$zjW zuJhdI-p6t5V_Pl#_;KK&@tce4U6Tf!j}N|HKiP48f+X>8vn1^KY2Zo2hrB9-G?1t( zSd!#4!Pqg?CEcXIi?<#e7+CoDFn-s~?Xsu;r2~vD?VY=KjeGQuYzpuBQJnQP5g#R* zKVEksC?gy$7Lx!h3nEnhZD59=$QbzT2%QNHC@y?|2P>;Zn74SmCz%F$F=V!&|Fjm6 z4BcA5a(!}gBNz;3cW`5O_t$24YMp+)o6cRRG}}xYVOc;4ws_MKD%<}cSyTGP56wiJ#5G+UXB0Gb(ZEEKW^PpFOLM8RErq zl-NrC0P_vH6e#cAyeS|eVm=WFquW^6*ubR4P+z|qCd5%uMFFxkpZyLdNY}Y-Kl1df zfdL^eUI4H26>J8Rc8iM^D%^K%@$md%ZgrZB!)q9q2$?9@S^zx*)P`!AhWFvl6H`;u zdhF0cxCStz0*)Q%4=9X|`wnviQy$#86Oa#^j=5%}(3EHe=MXF*KRm1g5UH=R5qB{K z-2b5|f|K(U^hpS^tI}nGgc;r+7!UvoU=Aa@;v(B3)neNc7Loi1*gUBK0GCrp$Q=^! z<8rV(j)OBE&;Y!GpD1D%K^XpYh43BB3}#saZ_6_SH^Ht7o41;om6Vp^{XJu|7KqBCqEl}YU%UYR z2WFl^=bKL!@Uj6YoCe5M_?e?A;()S-QP#da;JylmGPrV*Y@oQNJgX2Aukj0}Wy0Wf zkmUj%wst`q=-dqX|2zS2jf{y;pJb>D0N40i)!NhdLN0WzSJ3E=zCMJ;-){N!!(iK7 zZ^NFOoIC>MSN_AQAS!^L!ORj{kh0Xd?@}Tt`0cE$l#ewK#Lzy61qhvfj)iLZtN{KC z1YPSZ$p7t8D6G#qc$pDQFGKm~(*mescV}l6Ok@QQXjr_Ml;`>U%x2+KQ@>!cAdG)j z1aoizagZy(EQ9SlPCj0~vBD!7R#qUyieWmXkId)G^I%v8wa9-hEARMM0G`p)(BPcf zUSG!v0y{17sCO_A6vml>4?TANCyaHUnVIpe(>MW5G|)XO#?j@arKjbgz3tZR+YbB} zE?~vdUCGgafx)CDFxT2UIKXlO0CNJ`44c}=$n0>jj|*77dRtgLXc=~Pc0oZ>WJD(a zTZm;!N^A(v+4dOJLXSA+p#J9U{K5heov5JT#bcvIEUlc3vkpiIe+x8)earp$QOzyo zG%YO(H21L5xG3JWx+M_*8~3wb)P&XTF$Ou(NA`b31_vt?*a@D5a{wI zzP_40-B^pl-9%5NFHPlmCq3umb#!$};uIgPPBDBHp56qkYIj$VYek*lu-c6BbX#b8 zrIz=*UX3&*@5s&zmpmy48%Edm-^eL`rJ0&IR!<_9te?gW1l;=Ft5?R0wN}(mpo~@w z-c@8?NAHlA@5+V=W@b}(cmy%Q3bvR3jpqv_#lZoDpHeX0MWIjtfWR&VmBv@^U4ZC<1d1nlu^fU^khWm=r-1-n^Bb9&?cN+I!F#8015X7EU^q*=6M7S$U^C zShr1^>gjv(?F6S_H~`pb1>YAD7KW+srY0t6!{S#e@Q8nnl!BM0#{K&Yip~xWDsC%v zs0GSf;7^}nOiYZ4={q58U>^jd zWe71u>6tB@bYR#%BUs?o9hznXflLi*u5h8J0Dxzl=M9FF8R%9Sf_0WOw)TmLFVLq3 z>)x0|bQd!BMWWA@!?QaZ#(gZdnh@E^gVJC+)YSh_HDh~OKA^979gzkSNCM7j-grjoA@JwcdF=Ok_ zuSqJ#7GlT^t3fyb2R48jE_W|qCNLcZ#@pd7l)B$=TtV3IDiX}~2!BjVhTv=04>c)L z`f5iHf&H|Q(DRC$6wa(*Yuz5hg{?Kxp8D9t$HqQ|Q&`J=4d*i6a|qI%W3P6%wt%+= zjaDZm~*TwHQYML#4L@4KvWZ3wWHvX&)aM(mG>4IYs7u29lgGzU`*I*dB{|3{{vi12ni?IG9oH zNyZ+}Uwn$%!%hp;yCUSF>a4GHE481#7_O5Z=BaTW@L%ZBv#+q|W+s0R%NvkaAYF)1 z{=DK@AgWmf4yl3Ff!+Z-WslK8{CItH)0A7|%9V}n?aQAbvqOFd9$B1_y&3aT`a0aT zCJrA}ZEecJW01NOLM?UcQ12yJy5Xw z3}Tf}q#05mz3I}N($puoB)EmERX6QX^1x#XG!VZ+qM{PvXufsJBxe!?JIK@HZJ$4r z4h!=0*RQVJ_pri|nnBfn1Su38y>6l@{>eL07-ClYZ3u>aSH!aez78#rFS6USTQEa6 zU41pHcM`(@J7xVz1E}q z6yoY|aI7=Ye^17(70)wj^;4F*G$@QcjI+GC@XhCC^N8EnKfqCBg0Ok=gn$h?p9+Yq zJS-^o*$AvyS~OZ48PoudgZ#?{ChjP5RpYYO5BNL-GPIYRpL#3Q=Xsa!2yw&IX|!)< zNVQBQOtAp#XC&Eq>O64-7J@=V+{95i607@tbkm3Mo3WdZo=>_FcrS~0+U?cn4V%+j z9qjEPSP=#*LdFGb0m0-+Cp8jHRD_o;sGiC=|72+`fTGg)CnQf1-yzfht4fx4NwMCF zyb|bk#(I0Y7T+lq7nv}WcrCzvQ~C(0>R=q~L>VYwRBZ^$?|x+t`HDMrqE^i=1)T$ioJ_pxC90cxFTo;B&$ z)Ox~03^+K(z7@~OCG9J2O39q5p-P&1`{9GgQ!6{Wwo+AlWd&mdGaK@Kb2YImrAYcN zkrg|3U@cxRMAtm;2MzA_GO@380nm`RCdJQBT$~7u;5`={&dd*-Zjx`5(e{nLpF~d< zA7=Zb`p6K&_QGm7`e5ME>)7XC{~evq-j67n)@i=5M>~%5kV;)#{o)lCa|H8+z=|zZ z+hKBS8GCT;V#rvrC6yX0VdOegqIW<#wjN(W5k5`od+LyeNxTf4PQ>de*g=FA86Iv% ze8$_CS0)}t>jc%;6Drobko2qa6NNyd8aqDM6sH#4@6qOF{640pv#odmrYr;oTvCz< znBCP?P0u*J2Vsl27gpn%+~McoI_eDH5B9u|e5XlO>E3}oejKb$BRZ9AP=p2e;(wEM z-Qea0whwX3N6#hA*u(u(n<9BFdoT8rZ zreoZ_jhR(?#g3fN5#j7N1O#Tu@j@Mg4TH-4F8E?|L$~#mZK%ztX7baYoXD z{OqascYzmqJ3qLqhPmSG-~9)-C>a4R~3y4B1PzW~Q-D00Ke$AXSg zSS|XUjA+|yWeFb(e$C)Crw9cRJV!7pC9zxKoOcM?_jPI2>TxvqYJTv1;gT0H#q>PZ z^*%T?aq&D?=_~DQk6Tfwe1=RPk#HJKi5JMoyYKEaa0b0-qIsRM26i`Z1Cl9}@#fHW zL{@@rS~+Bh~W<9#E<$$TZB$M{?!chrJrVOnQjk3SLXT( zRNnibx7Sq*m-pSA8pmp4u1mssU>GM^GEmYHxSOS)2Hu(o5lsxZDds~cYADr8?#k-w z!x9Q_(DBYILk(eft6dx{GzVTmMu@yAQ6r&rkvkspU!l1qvn1oB+-`;mU4kmdrN-GF zJ3-4*Y@7WG?sieXc=2t?MM7`%%OG8OUAkPRW7CGsot^oIjLCt%u5A715vwyZc(;rA zlZEEychq@p5@s0=S!M1KPlaxWo1gvJCZ60zYPlA2?o1bMoraQ;7$TCgNJ8bzieH3k z9R8yzLK(7}X_5d6CLi-j?~@BzxEF3*uje};-boye^K)nMLbWb3Tx#Xyv;-dyYC_S%9m!kO1jgtL+-eQ6?!=Oi zTR}dt^FJTOQGC=Z6_n8Q=>b53QTNddeuJhT3g*VJ_!*nylelXOJu)@?G2K6Pj_3A? z{3px@NP(v$If7SR{F>(L$dA#Em{SWf)@M2-(`bhTI!Nq7?|PUDhqU_mAk;|HmPgCQ zEQtn=UVt?IzuyUC90^x^e z>&LnZ<-cBhNqTen&gZFF)&ITZ zdT8w_{VPZKWQDe~*d^_$84q^($8ux;himoE3-}ZNekT^8_s$j=jET@ zi{-{C`R4^m2G2k5@!#)y1rw(5b8+16(#7Ugw(Y0~CkwIJbp~T;_8Ux6a>o{LwY1L# z=IQ@Z@|^IOzgfijVm>OE>Nl?z)%d61#$699IN)6co(Q+MCACxv!7!4nK;cVPPi1;MHwITGbumJ!{MoH=P^r@WJ(Wj^=46L259bi1x<@ovA z`})@5SOLZh66yI5SkEAQ{1hecG=`mb;CZ6K^YZ+YO7~q7w(Oh6Mn+S>P$(!UfOo?D z%8CaJQ6wS?hE|v?VGj`f0Rs$#43D^Yp8?(|TMaSY9vki!Fo$5&Lr%hAL2ebLsP7kFX5jkgUU6TpZb^z)5o)CpWhln1ys`^F(gfZ5&<3kvh+e zS*ik`Ye?HyS!bVtFo+MAeqFR@NWOE?h}glFurY*#J)Ei5tiifj7X^B0xTyfln)IZ| zrpQ3S=j80{{TF#+ zEg4w`4^BxuWpns2)Z(!`S9?Hr4tOjeNBO$Nyhd%|9kGC0?}JJKNVunKQw{Jw0O^)D zQ~?sk0Iu&(=RV(fCUMCk5=Mhq>X|(?JD4Y^Lc69kDjltC_l2`QMHKjSw6yj6bXg&>z&czde+ZV<(p6*Hg0yhInj`>iOJ8R^Kv zknfCXc*DS?%qsT>`O+cNUOtRwA`k@1Swh*=RofR<;>9D2L`TwIxY00#<$M|-x78~C zgw+!;1^Mh3`7fcIm&0IymD$!}A08n&|IOF`~lTear@Q+;Jqie9y zI#2jLZI1B`;xy=Rc0tkuMGYRU3I$<%f!}ZoyF|XLG%yX|(LNQDBlw8j`Fn-{ZNDS7 zST*$Az6-$ZTFR22brv9>aLQ)+U%C&zF=%Bu$3-LnQBcU-e|LvuMUo~&^3v2xyIM^H zUp}WH6BCZK?zc(3=d1~F2Rh>gnp_J(pQKGI^HMWNE3|wq?xB|E%n_)i(8C%Dj1tAH z>GwbAor58B8%f{=B1W14m2WznFEEL0&F^-a;;Ap_p!+hDuP>7c{}{|?I}>^W@=#Nn zVfkX}%qBtQS?A?}ZpG}G^Hpck*Lz)h$XHD8!2}v3Ma$17&!(C|oPzr)sjVvny;gQh z#o`s*2h;rA2J+8vGc?17<`~>Wu~Fp<88tNm^K4~^=A#5H+k47o(oq;3SP2z6+1&A-^MTAy(iFqT1Z20v51K*L5*HRzNZM_+J`=bk6MSoL&%tc4z$o! z&;=u`TQnsr2EsZby2?+V*7(K<)5UKM^?Yx;NZdAM4|`=T{oD&Th%WeM%92SE<|Zu{ zUuyPo$ufbHWhHj3O1C=}iXl4W67yYN@-p)G9T zfnZ4sI*XnZ*Ns`tTa<{Hp`g)>XwW+VaJkh)rPO?(H^sTo4Y114{hmQ-<1$w z(=F=?$5_&|! z&e;rf;aUwPKU98FhO)}Jquh*;nr%>XGRskXI!g?TFWuW#q=pHRLG#e!iRtPjRB~Mn zUp7jXm!X2BM(SU7Gj*<~?1h=3Sr#m&0_LHWAOH}4kaV}4w5_9Bwe~6NQG{OL>mBZb7nY(xqvy3 z<0Pv~>?DgP0wKkzB>bRk1rkx>GKZN~o z3}r89eofE$_T>6*K5vjfbeEWpsM&V-GIHIc*4u7=@bYhNvTPD+GA*6{xiG~?S6HOW zCsEAM;-O)QRZMAs`sK?FwF7fWC#1byXqfD2v@3s!54K#CBi@=d6w{PYNHOYG(piYQqbx`Gb=;TIyEa0dW1b_45lAl@xe}e0>pkO)g+Mgw4W@DvE07o;{{ADB{P;Omy>kuAItEmd$kzvAocL*{Safq-yoP7Xfh^mpWyu+zgJCxYmLC zSYy%r4*7DKofqv}d=5={r*%IOe8IH11)ZB9#`$6F+`}IJ{W^;~vvS-Kayl;l7G4s^ zek{%+g!waa!Goaz&__r5C!2_D8Nuj%$-T`-f2J3bx4ub_&2HGcNvgUv84q|B>CZ=3 z!nqBZ|MQ55z_)0p2H{>~D1hh)iMxW`s%?lCA)8YqQ!b8&bwVd*z|xFkrKA6f7Oxjr z(n36`IC(KE@=nXj+CV&)5ol$D+(gI4DCcITnlUADVEWV@m<^i4M{=WnGb4k&+B_+E z2x$y#$-DvcW{t7|JVJICH6OxK7z5cMbBmhl?6W1Edy+ zooFVGEQA?z7Z6rR^Y&@4R!u?u%$j^4lVMz00r0ro z#kvqx)G%5*hLm3D9=Q>ZqeITcKvI~?K>CB`?cVUayF&AvImw`S^h*#4m_#9@;#-ID zd_DHEv^C+(44R=*TcID!fKaO3$~yleI89a{ZAa=1)U-Yc zStn&(!2yoT0~A?4vwEeEwma+E;u3J=B8cOl1`MBuoMsg^58d%1!LwEA4Hl~vf{l@K zJHtYg7AP04B}vR#QxP8;x*d*sQA!befsY)B0B9CY3*JEtd)EYaRNMP#b8(*NezG@) zf*?uf-Ti!R#m(jM-#mZH(ycx5Y``2PFpJmxyE~&#Gzi`|H8x&URW;08sLS=vdJ6K# z?<|hTf%nJpdzFOlH=~wGCUF<OH?^O6XMd0n*D5pE z*_75+UGZT1gd!MmFd651r@!g#BHP!O3xH4{M6$0wv>KQ;>FfzzUx#ben|*DnA9jZv zUCJ&l>hF@agLAMGGvng55WksexSxKCEJI}fTI%Y(kN1wcZ{RroxHjJ{|M2I45{Ixp zU=)Ae@y`VlfRoDKi^N{dpX(&czF`v3V`kdI;C@ZaCW z-n;+tKL5Gbe?J=glYj2`zklVQPdC@P5E5qLid0%)E|Zwgw8>{@j`kK$Wn^IR z78=_UvhM*AR`%-)V$GA)+b*3S*6#eY@X!gc)?7KK6y5IJxOM2OVsZ6*&7o3`bZ5t> z>h&8Y89@>qocuM$LS1zsR%6z%&@zL<- z`KiaHL5|&}m%e?KeO__@XkzQ&U`Qa>qi@{1)gy}enoNm%w%Z%jX=f3=4i~T7C%cO% zhO-y6vZowM|&QKj*gcWW8;o^6uW7@(DaOg=+WRz;fjDJv3&j@9a6|uiP4x z7)*SwwL;y9oz6v_OUc*hOg%mFx6XdQS>q5}eph=zaQ|6P-3F{Aleub(Br@t2JjhK|M{zv+!rrGrmBzHX|O*ki_G+00pw?8wh@WeuR_m?{r^OKxQi^BqTnu(34(PuFC zJXvPfUzJOAJ!63tX3ozu7P|4ak9bk`_mi84QR>~i>*b9SJ*O8BcYpKt*nc*Z)0Ix$ zNT~EYx+Azi(r1aOpnso|$8#eU6D>}~`728vqe42lP6>-6XY0JjFwyJj(!8o+-tfsx#98RR{7#W`axE^aYg?xF*Z4TuJFM>A(jw&wj+>E^JnwmI z%C5$QalDt#nbW_DMeCjsi0HPsr99psHK^nj@h+bni$Z;UI#PTrJz&iMI#)ljtG4ciU0U`>6RSWv!p@#~JW z*h9@3w&F`wt}X+%Keo~*+@AgLRGMmy3bj6H3L9->_3bgU9Q)?g1feKr-Gy?zva{4> zgKp4nBd^W5H2cXox=u>>^YZg^nPR?q%U5B1sHU=4mM^cg7wa2EfMU@~1 z2z(Gco)?MwaiWs&^TiK~YO0>8ESsj*(ZX+~R@o0mcNDH@Y z+4_T=FD^>|(x~=U`cQtzXBlGY`D9yXoX(!Hsa7HcKb7;c;`JH4n=XYs`h4}U!q zwJe-6w~?GJmif6)@X70bgA-FeW`+;1IwU(Vu))2a`1yS5)6-Eq3Mc&H;=kS8C&i|; z3I$Jg%2e|Yt*2!9PGb0j0^A0U8!F@7KN>tATCQDc+;p!!w#~LV8Xb+S`AI~W{ags; ztX-8NaCbae$0H~_qAH0}0_76ff3Tj&bNub8cW?z2C)Zc6PTHC8e)RbVyR0S^VS#x% ziK0LKLdYsyODFA@wULROFJ52hu|$haYrq3&5}c1sw5*}$J>1Rvy7H4CKYYI?qPbb9 zhUjIdr0C=eD&4v{0=fP{D@kZB@tAE)ma%A{5Ptq}JRu5s zVvRbfxVRvA7U^!B{9Ettm9)*XGG9FEBg#46pU?W;iBTh=DHq#iw9?ZqnH;I0YP`zU zvGc4hOa2vG+ve~)3cGZf)E%*uGYzQ=FH6pK$QHYWF9dF_vp-jlYTRY@@<@DmE6n}L z-d@{vs{TPDY$`gi@G$`2JJ_{nzf)#TWa$@Dx<(j-dBty84*MmoM|VsAcY@f?%}lCa zZ5Iq)p%U!{bk2XD>LXt($&nYewKabfdur;*NHxJ?t#^j|oLfmWV+a}f74=S3{PQaJ zK)Ke@Pg5_KvgfkyPFGQVdhvne^uwQDb}xHQZF*>S&WNQYpX zmm4kBxC}@)^t|`|gxpM@P|(r){5%#u;op&;(!afz!(~%QvfXEmLy} z?^jFOPn@s|9*neWYP((yTvc3atSfXr%!=!Dp3C21VrDy)jHBRf&-deCQ6SH|+-b!V z$2mcX_CYgI&x7y!Z}q3DvzKgKd2REfQh(?dJYL+nu(Uk+vpG13X8L2x`%5erUZA6+ zBlWys2WOuj@408^*tVpl=YsCSpu1}F-3OjH)M}0{Zav)}zS5XFKK6NV=SP3AXjT=G z8+3#(l?&B%6d%3m?pUMx7$xIz^mNe6dOG$?ll$(+EoPP-Mrp~Ah4LtQH!8-M1U4Vy zTT`wh-g~XI)fb+Azk*R&THGkk?iG=*CW_QLq?**o&EOI8PC7haMK!z}$Vg?TaEZRJT0jms^;>RFlB1=HCc{Az6t9Mhsb zibpd#Z_dwn2gf+1m9sLic^$S%c`Sv{w~muv6LbE4Y&kaJ$x^eLrOhMU8oqpM z5$unQx_%>G<1ed|K*^$LQ04IQtEerN!In7zq(pxi9c*g%_ecKZN&o&PWR8Dd|4D=X z{P3T2@_+f}!HZec_^**?b?QvNm1uHUp&4||Z}3HQFVZ#b5p?agZlcju8w%g(SWf+T z)*{cL66g9(d%pJAm+H&($p@`BVNp?a9^d$+q*h)Xdlo^;^!v=6;InvgyJuPQpNW>@ zx8BYBX4X*jO{?}P*E7elJ_1ZCCdD%Y{z=Mxx=1fVFL;79@pNvE_64GCw)5u|zLx=C zL_je4xcBwUm%GcErY5sNe-beOa{s1#e_s5}hyRuL{X13ucMAPqxf`S~e>1*6FR%&m zpBwkD6#0MriuC{SkN@??|74;N2@rC4cv~Ca&fNbn`8vISfB!hxX*;>puvRVY%@@q~ z(>7P(A4(^&_fIL4SG#S~;kM-mohnV+te)!&2L3|*k0k6m1a)H`DLs2xX7eRSsPk=} zMryk3^Lux4NAuKArk~%>i#wCMDd{m5A6A-7oRLo2d)uPN8C|TQl)HI5Q>7E`+V#{@ z<$GB@81CST?05Y(U$ZgsNgE0;ybdT?BNp(2qF&0mzZ!#gO-OZ1eBWn37Oz{x`OWLm z`A2v2br08XTXu%^?=wkStF??=8-a?+;XuD4Moy`QFjCksGX|koZof!RIo`;z^gsWL zU0T>h^{+(v@9Y0BH~4Snjm_Hs{XDT%&fhouuR7(Q(&yig5qsPJ`!W7~pQXYvg`O(+ z6mY+ZKb@NyKtvUp{Hed{1`xJ9jdS90=~YwN zT?*eEe;s>j^bCrv1vaJp`KW}yo9xe0`@6}of9e17O_IRiU@^TCD-dWvd%(4QTd=GE zm2y!gxJ_{I!1~s(^=9V4Mufk+i#nnE1Fzs5YHEs$DE(`W%RdBHf z^lAZf(&jJk=kE{7LCvcs5?fzAY$vKK051imD1uSk1avom_;!#^t${q=p$g>IF7Ce% zp3U3>1wAO+p(0i=s`mjT<+8fE(kk@|wA3|&QP;3#Fp!?WkiCp4a`%Gfet1Xd(N@A! zT0|(pkO&#mR`Mf9dwUoUudyG1Qp)QqVh2#@TMS=il=J{55eCB-^iWKA_kP(s}Z0MN=4;O6Ol`xbQyvSxn2&XmR_cXOJZ^V+dGbH7`22Kp=4xw1V~xbLd=$6%fOD z89;##=21z5DIZgg0Ooqq_qqkNwC>wKT5Po4k+_P72g~_5lol9pbobck;DZhqfj5XB z)*z3A2cmRxv>C^DUGFi@=jz`(qhJq(KlvdvD8jCKolvBsep%nh$f&`OphHgMa>g_W zY{aQd&W%Tfg?*PM!SHx%Yk?0xf1WQ07mn-frfqb?VT^X zKm;i7tx5=?mRK1?YY^1GB; zvFJ~w=#zDAwn+&1n4^_q46zq3dnT_)fcbC{R6fua@`*Q8)4v0sme`JbmLTCfN~@_O95lxM-zohZhp+GGSj z6C&#YusPsxrcU3l+u8k!S1Wv%Ab3|0b~-G#3(h}N&hcyAv#Z&rx50m3ww}_FLJBgM z=_J>!x7Q#79^7XrXbF`DW(wNvA0WEBh`p|*E2a*av<8f#<+=q%K0J`l2?b4JQ>_-0 z#ke0MnoIeISz)^)LT(n5p8l)^J_H%|tPB+^xs9yx^PjII*i>9lLW$p2B_yrWPBI0Ls=&n8Bn210On1n|2IAVV6WG>ZA&1PA&$SSa07*^77bAF z&BmNecF-Jf9Vtnr|H*5-E{+2ovsOzRHLk=YiOQu@uI{Mz}1NKM93{ z$eAvt3I{VWW~^p)YK>I8qBdVCy%D#es^jp7T@JdJe#Ss5<+ESEk5!;1-G5zJzQ@G7 z3=E<>=qsEs-D4C@;>D6?p8(3X<#j0bF%}NOqZf8z?77$}j7b2>rdT`^w}j^a4P0oH z;z0;fl1<2z)uHG4-S_uAf%nMsYke^Bd&-{*dy1ag9(SJ9-6CYO{2WgrQ-G;R!R=K8 z=?EipP{GM&1g%1|8!_F_ z(ekKbAKKv9UXtL>dPH}1gRlx)2R*>kfe`^bG#cK%OP{!X>s)MFB;$~-Ts^a&Uyx;) z>URwl(%&FC99bs%Je;bPR_u?okkGAonq3j&zaMZZHJrG>BF&N=-6habbn$7LFOC~1 z?Njz_!Br;f!;I0 z^uSUSo1))iWS1JC{y!wAMr=Tv?zEw4kJ*vS~kLY9Z6W zQ~MPJu&fh`rpuyQCkzo}Ba&0GmVj1^wgoRj6Dcj?7u0VOA1){dR>Vi&*6p4 z!X=|UAl+4d6K439frJBR5=q+b_pJ*l#f66_^}7-}v@FGWkyMgRy@sc(J5lEc-ruLP z>t6YsUVra};N2<7D|(Z@#v=l{Mcgyyijsv2maP`JeAOM50&B;Iza$Ozx*qg5nc+5- zJ{ffao~-IBxm|AiP{LK4)-Q$jM}+i*Q7a`WsRZKM+uN!@-)b)T z+6@rjBa+a>*(zRqJ@_qYGht7E*M0A`|K^D`Qi4$#^>hF5@TL0&-S=qz@+YIMboS7j z9~QIG$eDtT)4UuOp9zI^njoz*O6QMaQ#zhYG!%VM7EaR7rr9y8+F$@Q zjyY6|rp&;y+O!f{&G$kQ4Z}GVh93q2q1uE(-4`W0MXTYAYu8)WJ^XWvx;H%T{6uk102bU^`* zS)TYFVybAWZw|DqQV4FYDEc(-FzL8nQ3Zdl!0SKSOaMzzEcGAIV(bzbU2fSc&{VgSwPeSn_nFqI&hGZ zL=nF!s}TOItZd4SdED?I9CAZ~`Di&@Cc5+G1ITG4%T7Lto2zRBEjhiT3ArU20!hoF zk<2Sf+An*p?o{ALiQ^?A+rd&Eh zeacrf68zlhTKQ4VIGY8(g>1ObQ=bLT2Xh5WH@^0yzziq6UKU*X;Pxb#+(!uKZ`Tmw znhZ|w7VBil1oLVy6(h?}4HBUfFSMTi?=W7&>mSRk$;Zmz_ zc%z+KWayg=6;2-+&+Q=>?XY(KJP_XPP71FY*I5Nb^x+B`Upm2n8Uny{~S{cmpy zKZJm8{>r4rSkP?U^danMQHw$x8Z^6%5@au!+4fvpZS96$EzCp;SwUoGI@cQW5adi4(eS+e@E3i)eN2X5 z`r@f_+N0yN;Ed8aq==?~@*}Ci!Lr}*gj)obO-3&k_*#4IDHbn z1jw|>Byl~) zmt0Y*Sxh%Jw>_2vuI%&Igv)jG>c-(5qqpWrm(9wn=Y$K|J|ta0>vZL=U$Tq4M#Ed( zomx9kHE&FuLx>V#H&v3RMiD=pr#LEc3pGO=LEozl#{0&?{JKmtpW6Suowshv5t1A?pKY%XJ=Io4>w!e%R@XN$&55D7B*; zUev#_K`b-;7q+;<>O)CP43#`GYE=sBbEu&K7UozJJMfK2-*d2Zbd<7Vm3`XW(!zs8 z_Ik){fI$>kA3@JE^t6KL;H0@*dBqDHHJ5KnsLk8nS)RV42FdB}#>RJ@x8|;eGP_J!occ7`6B0r$BgE*lT4ds-X22aDgR{{S9oLSYD6^z5oD$0Kfq`%)V>q z;NWiPSs0Ewl?jO@9QNWZBZ^RVDg9o$4#!UE&&hL!POoP9(mi5V`p|!G;+K38Vi$<1 zkSCop%y|s*TL@?{;EkT1-lA3=|ATJ}WQKx}tHabikbDs{!MIIsXx-|6Jou5-dt+3A z-X3!90j0UpWf+i#T#)}&Ju@|BbRmUwqbG(7wCMIzK9?slOySfwaGg87EdC>xqIVVt zt|aI?G?RFeWpprR7;lhoE`UmKbGF0u<8|nSaRA}vP>EF^=i+dISlJHz69~Ebzba_X z>#hTwfY^d$>y|-)h5DKs{J9j&QdVEGLS6B5a2CPKcV+`l)Gh-ZtX(P#KtWX?7{J0? zO9Z6sAh$MyVx+rx7DT%Q_=NcQFj4e0$f3|unDaxmp7NL4>_V_L8CG%uPVJ14IZ^B- zsdEp%Cy4az0&z2RO4orj38ot4AQ0e_aqUye>O5s&b8|DyV!Mb2i%c8_LvrN@&5wz! zu5Kwoz6glK?Hn!faeF7H9KLGMW#FCHJ^9eBue}5PM5P@XTE=d~hqr`>zdbvaIw6#l zlIkBFm52Gt$Vve`Wm7i)n}APj{>o1fGo(zuJvC}qclsZ6yoTSm0$R>Ni|q(YebB#Q z5xT(;-4M1qlO*}_y9s&$tfCjZV+CNz!%D|ofH1bK&fYHqxd<#F9`}T|72_E&^BLv= zetNci4_1NiMt!|gwyaNNSQxGuYzL_0$KsB^G`H%MeQ{wHfDVE}PTHy$ z@s%mckAO@Rnh3z+7C>+86Ut)Xhd~gerv>xC1f|ls{!}j-v~=>y{`N)&aIS#x_7Tw{ z2(NkZpxw|8mE;NvQ%Q?1?BxL%b1&cEiESET2ADmP(nDZRPy*YBNmyu5zrx6RzHS_; z-}%s@1xwuD-+z_ogL}r3l4q62evxYrPbluvWo2b$P>e6~Wpz5CKLJXl8bn?=*{EW2 z6BIVs+2(F(F4@glQ2@X8|wlx zQR{i;Ee6)=i8o2>x$;ZpkG_RzC%;L2DFLEjQPA5vx`UH=8MM>V3El}w!HXq)4)RE| zHwIJi{m^uejnHwo9lw`KhbxkM?f`(n+G%iGsn5J`;7XwO2C>R*Nb|21NDb}J4PLNv zBn_WQLe9xFJOAq!)m_lht-(rw73T(V1F!>_>9JQ(P=H3GL8&?a`New>76PEyktj9^ zYWP)#0#;Vm=X3%R(42iZGX=73oH>}VATKNHn+*7ZKe_aik*)oOba=LMoSfgGGJ>{@ zNvL1Q5LWecWTHoJV6NxlH?Rv`Ia5L4}wZ3gNePaCg)ewzg1F?bAgD^$rZGm#w zV`9*P#Lpk%X3r%qzJC9aNJKTPaR5uiQ!*7aVrvlH^6>ogbc>Nk!|x$8wY9Q(>$!4V zZnD_T-rCY40znhvG{#ey1^iN9$HG|zjjrQIiGZ&De&5>R9f5z)-aA=Ba#a9QTsau} z(E5B$2lgtQlj)2XtyQGD0=~k6fH3rHdU}23TJ|r0JkV4O-U`yb2dCJOI7j(wlUP&+$vC&J);W3A-;N#nQI|uzT;r)pA+X z{Lx}&)KCqaw^QuzB|2r#ZQ@cE+mAgaG9I?NXImUtc5^Ghw z6mzxL@b(w>{M3LvH9fR>MSPcZBpKUFiKC8X2qoLAqlqy*-5K2gIZG5=-Q<5<+Wl_q z&P!ow5fe<1tVGAKmpLfbTQ&yCsPcZoxk@n-Z>ZY`9gVPa2n%5s~?aCr)^89GI1aB zeGlKfUe4WCBKODKs8Oj{rEA`|)$BgU&SxPrU6nNFuX};jcP5-Q5bn%iA`cgfr)a?^0Yfy%QRe4qf~zOlh}yhg-a2A*SNG|*Xm zJmLQ9?gzbXI2Dx%S@F4G63NdO+*Y{|FTwEl%^dJlW@2veRRlE7-gp9eAa=0CR6WiF zw)yMm3QlY=1XTa_m7(|d#q5)Ez8H=;LHk|7C)iZbuQ6}$7_kHJC*EQq=(A;UqFv7=1okjx=NNhMQ;k|Y%+j+rt`A#*6D2#vxqWGHieN`sIgl|)IB z`0dkY-F5G}zwi33`^WW{)uMB}&-?wn_OthX_5(Ov6I&nn^_|o1LO z?OFZVGkOV$iJG;9T*=QRmz)_H5Kv@|*pbZe6LY69ht0K~dEgrcWro*1>i!A_$eYO^e07$YQ@A(= z@wj$h;2STigdRyg0h&31k#!N|5&Q1`N05a0EmF3;6VQ`PomSJylM_ zF%2E)>P|SlX8SdoE)d~rHoO=;`g!3 zFNbkWR{PZ{;{-~w%*rDC8qQL&oyte0-rM4T_6kfx&GMac`NJ}M3#z=%eq z;W)HN)m`PH@^NQcMAxqGl2>Z|2+KXpU~Lo$o`ggf32$A>k998Rst6F zx!8$&rBVFi_uU813Jo00Wx1lw=fRTNymxrR+%I4-DnBCHV83>*@#Y_MDl&?<*8g*X zqp;_t^BHbK#$k$_Q36(shhPhFSLik3wsDD^}fXaJWBvt;3xUJtas zGW&U0>wB^_Ijf2f5AV0M?8FGie1}p#O^(VI+nuh|gNanazS9i#DfSd?>Z(LmF&xxZ zXUey^e*|+{#z81*z{J4P=;{@=>1{g!#c0Dwukb}DDN~e{TYdq7RKG-u!)-Uv(^So01Dl6Jw%P1t+M*C8Cog~V=?pnO zsxdIjy-z?!bhC_}I3;-I(v!wOW1_8xWbmc#7iD&_q5LdFUe@R)2D?sD%C(jPCC{{L z`w;BQ#P8W6Z4e#1N-@3g@*LxPC4p?RPt$!$$$Ss@@KJHX-82JkOMy`tk{i&-<)nvJd*-t^q!xUd+3WDXGlPw~L+U zx?l4?sJggX_av7+v*XRr$2WAMNUP^p?%kb@vdEd%R7K|xyQ)uR|r z+zyT0x8V&%Ii>Lvk2U2Um06Va>LzptYZ&;wxk8wpWV(kxULCfiPU&%`nu28W83C$f zkE0zQ7O~hmV`LWh>3ffU>@GUFr$SENL=g5jLzh6~p(bDKJ#d4QrGRtQ8j3|w*E~HQ zE^MY3I*;roO1d;dDwU1G()-Llj+hqnT`LeXuoU{HqFyO20XHV%M=4FpZb|;R!tqs1 zlCqn)3JJbO6Ex1#M;gxjhJaJUC(fe&>ipSoAJgY;ZLuetA_~+@Vh6vho_=O(D)lX9 z>#!^>h6e6XyDikNw~6yFs5;`if@@vx^ZN zU4;$M9Ah%4adus5j z3)a_q_0ovu**b;R=e0hHyK-H&)gE*W@yWm`eWX>aDZ{RF>Tx`+c3AbRfHIB(l8t&I zxrfNY-gDhktkw_waYWUGoP63>00D&d9=Lv*&gF)Rt}XShqZ%P zM;V3HYycDr4|xlT5BvMS%iD5B#wlo3f5ayjaVPo$IRbxLDb zV#k8GME~hB21;%Nzv1plgn|-R4rSJQF?1fCl3= z^IO7Pt8k_#2zVQ8s9bF)q0G?j!XrsYn3M|9J~`$1je(M%SfdZ#hAc2wd>Gt{U&1qWsre9a6+)N>kOye-s?3 znheK2-SZYmH_&lT-$?V~(?6)iH(>J9)z6KN%Yvd4UpP3Y@bjtey}KcXUdtvhKFs^1 zrj6cldJ8d7Ol{I6S(3)?lO=9+CH6hl0l%7<+E?fsaVSA=!ykw*@Q9_xN>n`%{ z&Ns1?3kDx19UJQ>nu!6MHk6ccxdB@NSrLohvk2Yw8_edoofNLJftCgfuDXvb@JuRXY9T9OytSJ8Z>5QvQ&uPiW z8GpmZZ;oDOB9N|1pAj;q4ZyWhq4xyT}A@b-fbqKr%jU8LSc5AK;N z+~O3lIOo$IKc8ulS@Rt!dQ;55Abo}h5K`-8)Ch4p=x?n5)a0LSOO#-hg1E$GqZZd{ zOo*$S`dO0c$4I6x9!p!pbpGD|kVpAl)aPjbzE@5!*T8b4?RY&yIh&TJ!a?q*(ze9eFU-BWabHAtrSfc(PXl%S=ukS%)#?@TQ{d`nw+t0Tll~y@3wpK8qe7^ zgLGGnHj{!jpB0kVboQL`7UFhkLuO@%UbtCXO{ccKLsr2woJ_B_QVfj@;qJ*e}00$Ar8PJDF30Oh!5;o+n0{n6ph}cFJCf%;yp=Y@ePuTszQi zWvVeK4Y`usZ~!j(fDNvWJaH(TzIqSd7b{k5GN8TiNnLgxW-qdNT)|Q ztZl1~9$MaDaiyqe>CYb~Mn+VcA8z@sk_cA)*sqX1t1lb$iNUc;iN(jG(%a0#gS5+C ztLu0C*WpsB!MZaq-@Lh8V}8mHhTs3q+yDPyF#ct9&HfI8{Y%vVjTD2HA0R=#{|aKD zW4S^jOp!!#{JQnb9S}C9PM=mPRy;u4>&VDZhbH3UfRgz z<VZ z1(a}{eH92j5<8;D{e%vHobC4(8b*6|W#2@`2)+~svR;D14E}QGCEcwALS`Jv8kI?D z0uDF7AcGgJzo!f$Ok|Z{cg^DNN zQ+BG+Le5R}*_D5CJTzCsq68EN2j55ZztsnyzGw&w3%H&?Fs}3v%6%68ygJxDPZsd9 zv(u;G;rhX@IX{~pKJsoyNQa9ekJkTEJWGBsB>40ym+bcs;MmxL4CerKcO9NR1WFhe z@Dt9JEqZ$3ovYDUn=ZD*3qAsKUDO4|<@2j*CCjB}SA2(pkQ;0Hxxa){)4Mwr_dl`s zqq9t(#L-8$F+t46!bk6G!X>^dP5nj>MO%YOcovP*C|*}&HH=qbo3_nMbkRT?+(`t58FNQRQ3an!teivBuJs-^JS=<_Snx!W4(QmJ zG;CaARelwm#7Mpa?gWfIi*s9o7*j=a+|&uM-neZ7;&`NVIJ;-TAIgVC)1mkH}`t&cYW+|+NsK;1A@+EPRwl) z&6H-F%&qhNTi{ z421QA9=fa%%t*FVvdrwo>arx$C*j(d$Da$tkCS=H_-GB!e4}>I@qWf}t%bD4K?KIg zH&I3<-mr3s()icL%Zi>)4<15oawQahG@QQ+?f;w5nOqIG^UOD&EuQV~Hor zn=4Q7Txl{cPusQ-tVxrQavkOX)dwLZ$75LZ*nCx`G}A{Zp|m{=pIel>+(*%~PBI=D zzyrp*I)v~XkG6jt7{3%tTkGZ+Zv9_lf?9c71GK)GrF-;yo-CGby)aah#qhf7Am>4P z8i|A8_UFm5l1Zo4!ex}kEH^E{v;|RV3HNuyyothJ$m(RyO$~+pQ+8wI_ZF zeF2lRxb(fJ(S$EO8{_88lg*ZNB9ihDoKbbEJwq#=?w4Off9lfMwzdjq6WgI4O~Ukp zRR4P_1rcdBuh&;R>W8|al~H$X_30~raL>!u5@zWJ+gcyLPgL2(65PWO0v zwC4M^C`eMzzLMf$Dhs#R=a#vKW-KF6+4G=CDXt{z&yp6N4L9X`>*61&{6@>{=(S;xXZO@$uQ)Oc_%ej5jLFKQ`c zF9cP$IkS79%&(=;&fpOtSyl^E34C z$P7V8ia=gsrO?D+x@U<85$|;|EHd6E`s}ZvN>`>DB5P!WFlnb~UZjlcPz&kHVC^OS zvg6nyUilc*1(OTiqlSl~7JvL8nUBm)sfk+72Hg3{tLPhXv@* zyqd;+yEoNBY`Ay$CC{=h4g|=ARBaxEQv>Ja(`Ym=Nw$+?6f(i%>K`0AE-Jb2PoR1C zC#;$%s8pm{acWEMM*v2N)29(WdYHFns9;ljIiv{21PT)$)yjh(MKN5@4^jUotcd@{B^zh zWb5x!;b)QWlEk>#PBu0sem1|-)e>C0SrD~);ISyf6A!jWv_rJ)Zo~N&$5P9zNgk=< zfQXbJMgAh`*SxGuPZV{F)CZ!KNr^+;a|D8*yqS@Ky${hAzW0v$Nhu{xEqDWb(kJ@3 zG}btktp6LLN6?WckZm|WBk$%ImJe4v`o>|y%tt-wA~!Y*@)pNJARa4<#CEsw*$6mS~f0k)3!ivmLfWj9aZD9 zqf&^p2&?OWp&Q|L#)drty@i|R<<8Ii_z^u^_GD;iXy1dVt8m;B+567Y&P&NYfJuAc zs0B^XGC+c-QJdx+T(c?MO@n@p1xdsLPdEV;S zE1!)<#jc<9u#@sPm`v?lVX@tU*&ERv*+$&=pA;4zVkrSjL3~hnsGsSzv0=M264MIw z+B7zQvSBhfD8Oqd?szp&KblV{K z{;`X}Yr5?{jrCB$SK!HlO*10ziv{p3w&x`}8aTLY9xyo~lWJh;#Qbn)T3x|$XE@CU z=S0qF+;dF{cNBS=IFM^TtelNWC9Zu%i65h&e&fIe5WW0bvJF&nyX}jvA zNVCDHtIdXSM{+${VZ`)HY55b9#{7phV+mhbcqH2g43@}2z-8pL!a~wbw*v(61Qtha zouIbEcP}dy4OCF9bF@<38>^lNFE0i27+AFXPpVFUxQjpSEgjFm^hB18Bp@WZ`PCl@ zC=x}v#an``L}}$f_nDn4HC=B7d+O(sPPra#Vwd>Hb5m~BN39>#v*67XWA!bqu`#?Ufw`P_DlIKID9B$+Cuw6~l*pWTep%v`%e3YtNJQ8|J|kSqH)Fqj zRiBeD%t{_0M=LMP+r(@mJj;ydy-;`+|IozE7*mJ9wia2_b)9u)4+6$DF5S@lKtFKk zx5D@E`+d*yXfE&0wsMpatu$)&WR#?ecTU7MGCV zEUi{0vgFfK+Z>5M%FlNF9I<_v>g)bwQTf?a^7>%6pUMWUNc0oxk_lFGc=g*a%u-eX zv4$D2hxk`}_Z2;=9O7{M3Tyvo<==!Y$B4?ycVD*V@3PAm_1(oizj(85s@Czv9*LVL zom9j&rr-G#pX!;+FApdl%xU`c8zM#K?87rs(vf}n=f#)&5`@s4ME+P0dV3)0 zLk;6P8O-y+r%xmF?&Y#j-Bks5^oUWb;8q%e{ha!Hvot%GoF7`yxIB70_a6*v|7T7q z9TSo;y8OS=Pw9Cm|2DG!ufN!!0Okp^@3ZkM;yUVPJwI1hhIQ08*lvbFY31okV)gCy zLJ#L>Mz)D}hUe|V&fjoGG55f~FM?nb-tYq;Xug7SLxn;)xR3ZfC`4P$`!NY*oFLi< zgC#mlRLZlUqi5c3bj!!~A<&AKp;Of3+|b4lt!mqg4;Qi87j~3#`Uq9S(zt3Q3-`JB z_~82MSTeTRcc2$n5IHoc66LVLVY^O)K%|>|^X#V^6`&F{RaB;7kV8iOT+=!}e*Qir zPoe+&<8>!I4R}rrm0SJ|Ko%YVdJOW`t;^9|+{CX~e1-)xUoX-B7${^l>jz5UE<#0; zMlwGLdKh*Wq2@z?QBqU$M_s3f{^7<(cX@oWS61ya#>h$uq@M?VB{Tt~?)vfR9~fh? zB`O63*3h5nQvl%6g|DRUdXC}^q5HHN!FdJCuUl(^7>v=CKBlH@kFkfXHfYujQWr;M z?x}D+tpEg;LYIyiU2CQQt<@PA$KZ zI?l%qmmMQ$Qg1@t#=sJ&$&Z7zKj{Mmuo%MZvbVfLMKX8Q<5#c51Z8|5((2*zX(pd; z*$E>W)@54WADTiY&>&(i)v6M@0eFB(^OUTmml}LAEu?4y1emq8kIkOHmua)eCz#?yz=lcqZU)9%brO=%4&5tBfPtz zpi%CyuoalF48!Tk=(v|)8Q~6w1L?wG_#=9xWu#$qMZ@xGST2!&$7>Wi)5t2x=r)Z! zQ8bGnY)o_MqjH?r+G+_0KfhRM$kjc6^VnX3*0P+ZoV1aI!1XBc2-6mu@0JqBcW#Tj zohWYZR!WSQ6*#o(oT9*mbpr9hc^@*j*w{N57V|b`|kun^269D01}@n7^1fY z226`uva*Hny-4nQ2={)TuUXmkE>6g&G7_lR-s-o@G^cKl%j&aWM7hlOBEPR{>ju>uZRd9Wv+ovMg_5+h^l(M+dz>5`jfDu>kp&mQajYbAKhQX?rO#^DiRDG*BMY;mztmM!ZsD$tGkFp2 z?s#P7X$DEGcku|K->q6R*4in8!~Hgx5Vq7(am_X&KJa`9!_P}Sc|(taw|zM=*&F^I z3rx0;r8`|u?UWmpyCc;-voP!Kb_1n5X$o4cva-4+aNY zoUEn$xaX>8>)BQaf8)!w?Z+ z;nudcXAxLz_1rm5JINPj9FrK;8?Z=TJn#3qCcf#0)16N_+Q&`9Sd7=(EtN23d-y&p z?E7s}N26q{CQeZyNUIchTbZD}b)8Z{`3L9h|(fM+`8&{vF! z-6<;kI69MNdc6AJxwl>Zm-DuseR45|XJXnP49pu*(zBlKZbg5uuI%ke)i+#qrkou& zn_trWunF$RxW@n|q%-WB)ZmShE6CzMb*^@s!^UV}m}3n=yDPCg@=>j*K>GWnjrN<= ze0|uC-D(ew;hfcr@8JyojdgKV>MnhuFE3e=%rpvq9EBfeJG&+j^*Xu2h17Q3zN9#I z)u!mV!onC~-M4uJ5F#gW4g1peokDXX z*;)DOA`Y8){on(1KYLPzO(El%G&}{f0lO0JF=X@%x?UR@_oZcu+-ThIsX5>aN8wcz zZqS#mHSmLs-LyR|oJN0ubJsiuajK}GfCS{!0#dG>o!#2PnS=YT$9YXY#qGdv>tVBS z=<9*JsAcBC{<96g9QW<(ROs*TXS4HE_v{pvj!mwU zx0ClM^RQ|Pm^FNXyuWQ^^4tMUm77cUsYX?p%XDY-c9}cc`b?gq za5ith)y*FE;6V8iEOUh9_$pkD(>NnFwNW8RXnoJ~puDnUKJQpMFMYpSTs%X4dTa*3 zL5zkG3vKNWu1UV`LFs(&=Tv^q;+~xf!zOO=)vJ4MQFED|K1~>qy3OpCQm&obHJ@;L zq8;4x>^B?&@7UkUC^lVA9Nn``IRAH3Is37O#SQewb|tz={!%}ub_|3z zw>5L6$#44@AgXLNwp~%6epF)F${AbgNO8^2GlR6)y(UPhHk`>TkM^=1_&5=rbA2xM znO=`Yrg%@RLCo5yk27gAAq%g`O+#*D3aLsDB&}mq^JiZ7TVre0d11W+i-);Vy_%jx zg>@!H%WVN1PRq&ZEFIVXaE?%U;%Rx>)l~=MzUoS7`wC;?W$gCGlhCdoE=T`i;NXtG4Fnh6N&la}HDjU2>v;U_ zIR5ju8h=M?nCRNtfByF0R}%Y6E&YAb|Mu f`S+EUZ!ELl^$qumVJ03V;LB{gmC-Fjmx%uYfOEiC literal 0 HcmV?d00001 diff --git a/docs/images/async_compac_4.png b/docs/images/async_compac_4.png new file mode 100644 index 0000000000000000000000000000000000000000..537b2ddd3185cd5556bc2f4852e341176b8dc596 GIT binary patch literal 66932 zcmd43byQXF+BK}3?ha{06bS((MLLxh1SF*d>6DI5cPJSB|-LT-ZlhqF- zz2m7j7Sb=Qcuh@R2>WodKqG_88bT`5X}%fYiGTJhZk80q-v^PG;Td=gq5Ag;thBQ~ zN$~|v--63ZX`O-i`$~?!zpvCl;hjD8|8a=C!2f-a|I;CUUS;_|oyY$^gxCL1E+sRa zIpDA~H#ax!maD$LzW@HOJjo-BzR}V9y1E@E9q;qrym?bwdte#shRglu&!0mcNuRVA zFMPbcGxmAQb8>=HXL~SVjB z(QkXyYRYeCLhB@SWu$bD`DC|)-m$J}^RQdW*nhcV_}Mk1JFXY!_o&d04SPSms_FKB zpR4jL!rbrTZ9H7>VBYib5_<5C=l72v)aa{PxxVXdG>7hzHyTckeL9Cu)KFGE>0ZCT zUd>prYX9;=nC6kXaE0py#IPh*4d?ooZXUtean)1sw;~s zHfeNla435725;G*)U0i&&>-;+Jw1KE@t&34moHx`Dl3N$JqQU26FL`v{}ytb^xa+> z{a)oraL@F0eSQ5;+1=e;N=iyM4-cz{4{wT!78-sitE)RqeEI0nqYs|z*^P~jp`oFv zsj1=NIQu4ATEV@1`@z)e8Ml7@{E7XvZ*cH^dHHy)2d_u@y98Y?@y@w}3vqHEqkP0^*kdmENoT7%*~+Yb&UT$1kT=wygEH-G<5 zU)t5)-ab6MbGWyOepNz+Au1w5yGTStM8j(w)6YmveYusP6<>~)i>n|IJmK})}WNx#&1 z>^V9A>8Xj&as>g+HI^^M`W5&aIAl#L&DB!Pj~_p7H{RIT5K+KAuP?{hh8(YPCwLsE z9WL;$xHx{_*&2=g`!pm?t|A?*tw%~sgA)_UE_c&nLDWRh(o#ma-@bi2WoPAk70E;k zrl1L73`_OTZw0#iwnT)4lwZnCi)a-&NXf{Mfe&nLqxX2)_1)bCF=-ZAtF`jf+Rze@ zmuQC~cIIJaBO@b6LKtObWcaRh=*iwf4mnOWFlzo7tFVQl8Sx;Tgt>iMUbkOhQl>AF zh%91yh;}vqe3L3Jg7!}Y=h4AVrm!QE40Vr?Su;vHcufn}Awn$$`@zO|v5BM{Yf_`j z4}6=EN_%=@{2ydLdeTLMRd)qvLZ(ScxA@HS&MF4)dnwu7YeqGf@OKh4=dmw!&CZ%# zl9ZR1f5m_4No^y7UtYmW4<$dnmPjAM^`$x4&S_)*cJeT>l4DLC@Jmi>`I(g#zlE~dCkqu%cC1iZ7nTdzjPN)nz^J?wB*Ltf@B zl`^M{kQkvABfanYm4DpE)|Nk?f4!Z9LLRU6X@)TqwKZAF72f-07GKf?vsvRG1SQ73 z*v~R&h?u&y^jy~CMPF3 zsHv&LcEc>c#?Q%+z&ZC5VX~*2N~28MaRZ+{d#0(W86ksal32TQ=T39xFF8bPQB>N; zRA(nACuiqJc6NvH3+ZPCwJrof1my@rR837y!6sG0?iTo~DQ^^cfo`#ab`zhkFS0#`?mW&7<8GJIxwdGU(mKm7 z>h>jOEFKN+T`BDSWl)T z$)Ku|65L8zg%A{5g|4>tH0eIBz$=C;IG^r9dfS_tREwQ8^m80)S>@9uMQa`yN4qecHgA14%-uwk_?;fWMMM%~37 zz)RQLegDYDhEoR;wdaMf`mrwQes6|2!A?a*1vHN*$;nhhwvQgk%gM1=3JD5Io5KCX zf2}~?jM%`x{tSJ6fVaN4A@F^+dw;T{U8oZdHulADP1A{Ni3JJm2~`P#rBol*J|Zzj z2zV|ov#Rdz?`Ma5tWLHK*$x-6QsljUJ;U=>zDMB+027(>DT;URk`J{|^|@}5UXOJn zXj9ylAMAVsm4VNtQ-GM0g+*$MgCt#Cn_%mlr8H9y}RZRGW zoMRH~VgzG6G;3)a^VBoj6Xe931F%m9d^%ipbcUcF@@+nU_AJcRd`;##nT`LidQtiu z5(6@3(m@SuEYhG7a!mx)tdgIvZ%Dv~>@$HIHt= zx3?!9ZEI_5+Kr1xz-}FE9 zl$Jj4&Ea2gv1c-rfTq&^dnM!%fTY%rj;iu+1uS zr=p|VUmM)wIK|vs1ef}DRwk5HR9f>p`mS+u-gKFdAZ35NGVb-W`$9VAda-e{3*eTT zSOhFL5izlvx;o8~qKmEV;%nvSq*hLDxI-;V<4jCRq_i9+s;8GOjX$G z%E-uQYk%GUvq&baqM~wFODn4<;!?_kuC6WvJ-v|7P*#=)yi`}KFCm8YwHK> zG9jg<{Dutt7yht_%wfcc$o^|CkQ_!#^!JxK&nYM?cLBi73^=|ZIrCh}v)Ozjr1*}r zyZb#2jX<21x36EnPTY1v$oGa96%@1>S2<28+=FVg8=35}G7gX#89~6_`%(-iyqz^V zAoJo`sqxputaw&dJv>J6H9k)OS^8_LhPKYjWntrvW4b*izk zzTRkl3YSN>jLtIhMkZTqTI6j=L_krJI-tBcdZc?954Qrpdm6w}~&+go;xg17Be>{Fe z9fm0M6^y4=o&4#vZ2P9jxn#9!JfUve&QW)zo?h2FkLtA%WoP8l4b+ zv#oeuUVGEi54a`oq81ky{rvm{1qI2@_gOu9^lLCrJ$j02VsLQq+c&6az0VR7BBG*X z`tsksYc+=GgKEp+`r~u5*XB>CrDqubbO`OO0Ic#0jb56lt@`lcL%bB*3~PO8NJw{@ zU{^~^PE8Hne)Ov#hDZl2xz zJCySC^K-Uetg zt~`C`;%x2g>f9HHprK@BWWc%an;@y#O8F6A$8e;qUt!Um! ze=?|YVAa{$drEov8!jVcB`#U>gh`6NIaAzzaB%R+_#>fzDzW4>c6Romp`l%ssAJpX zN1FHVf3W%X7C+_E}nB8~e;OW2vjFANuY#Vw@0urX&<IXsIjigcYD}l>&cD|3?@P|!`+t}F`Jes>L=1Gw zf9}<3-U5>Y#ilL|aa7fKwCuh?xYuQJ3>V4!Osa8v%$AU{_etZ)(Q0SU-}VfA7Yc>? z{Q0xOhRVlPqx<($5vs;4$OafLw6rL2R^R6VGnZ)(w_skB%6hVSoqL9dVLp_?S_Y`o z{~@d3LxFJm55@{k5nx~;ewwNNZ*mQ-tBnG@ogg?45M7|Tr6rz0 z6cRp|&t+}83DEG=7#Az6DO8@((a~A1)rs2hukWF^S67Qj=V;!OlWU!635TRUrkOjX zqr^w695XXBOfD~ z0UjRvi5hp2?vvaFbPJR@ataFQh;hd2IcjGa@K4P2)=B{vll1HvQ#AVr9{&Y`=%7f@oFS;sK4JCU=0rs544K>!#7%aFt_Gp zWg!RMH-D-MxB<{06sDNL1^Y(3&(YwMzpu7l?(2#^wD0zS=#y!AC5MhOj6`3?L z1-JF}k$#S54Nh*HvKL5v`ZSojG`gl{ADaCdHP!FmzxD6mm-Y=cOVH-dX@NTfJRx9@ zCjQA9cjfYm!a|~O9U~*6miF7YgoJWlTUzLIzQVk`B$ct|P3u4ottR?W;JL}lVuWGb zxu#blrKw5wMY@&D9fGLRZZrr(ZqXXcPe>Tv!b{^}n@+95M@~-8vi_gd24BF735$+% zX=diUGVqE~(b2(UQ7f~d(=gJ6g@gdFxHSGj%MkVc{rl1frl!tE`|e9YRUpfI`c@bEB&frd3OFgPT{#}Dd^!rcG;t-whxDSWJ1rxZIhzy!r_tkJSQ z*TKlZFr6kOBm^MKXTI@AmKw(!py1Nd=(A*SSu zQH>1^4S*zPXXA820uoz|MKDOGwznUT@Tpz5Gsr)$`mr&z;l94UOsJ+6GcxR6&aIna zZ$e9kAze!R54M*@>Qgjv8?Q%*>`VQR3SfK-NiFd@0uRv1nmnKw?Zt(K=b4!&Vu+V1 zDb+ULIFnTF+zCpCNaK=_V5HV7SL;Bp%n0WEw;~j3{H72 zV=o_{-WS3og|ZQ)R5g$~SkHmP?Ed^2pO{$Timi=J2~94wNc#<|o@q};M#las)fYlp zq5_d$Df4ZQLYJ=28c4zsc&E@1UC2Bh2nH5_D6^ae2ih#pvv_`uF z5`)nPCbm8zfD}Mwp#K0Jz>>6kEC0=#w$@fD3IL7aYO_3_Y3bIK1SAKG3?VJ5*Nmh-S6M7=OiW)AFHI?4=G#4)i>4W;o zl}>j22fR&tAupc5QvF_B)GULt)U*JXwbsMQ4<#(aTZ8(|oB30Zu>O>nmrsX0XbudM!-=aBvFWd{3B)fp{D5dt$hms;>dMLr zz({L?!nq_f#jM_YC&a%kKL3xR1nWp6i#l;*NdoHFXJ^ITD*~og^lG%bh{iGm9?*iR zX=$MFt&UYJdNAevm&6rBz)e8M*QaZ$Ud;EAC z3iCL8O-xkTe}DY5J2fVTu>VT{Ud%OCVG2j7C^Ooo{sFTwie;Yk3a2lTqxcW-&iHd%xupNd$2c#Q> z{~H+PWJ$SiT)U>Gpn%O3X=de*B?zR&$3k~Ex5rB(tlkooI>zgp=*I3pU3=3r;H zgreaiTY=`>&n+yBRnYtn*$UOXBaSgzff?0gQKuc)YK!s^G5Qg)&BruWNr9UQLd+!PT( z-4HQ`B1c3N3v*OcWvGLmo?iajw{diW_&6N6<;BIkHAE+};`z*D_KzOn4R_peMpE1C zzTQz#pr?O9#ijG{!^xZ|2qdR^|LHn$^gV$4NBSA+#K_19OwrK7n^~6q*{@$G-`)$y zq@%NW@ZcOpD5j)A8=d%S)$#YQ3=ekNJ%o zH{MD5Y;%8Qkc^FqS)-uB#=$Yyl9Kue_4ArZ12^tgBAep&%EaQ_+*5#%Mn`VbQkP-G~x2R`YW?Vz#tMeHnp7Y13)($X>| zADGXOkk)ti-}lM(Q>X*cB;+ujpPP$qNu#^Bvf{ACY!t-73sA-Pxrz5`4Ru4txn^(B zZb(StK^t`Y^ySMkG}g0(aE}H}xH=aoxpQgO5?2eLBpvB)=;9p=p zo~sXXA5?HsCJlWfqk4yl8X(GXu(054@T#0}pQqOb$X7?plA@v2!x$P_9H56{J=N_( zhMPUUdtc??kUFoZ*ilw?eTH1nA0h$5yoLNhIhv4D?hDjp81(^~L6ricH2i0CiDvLx z$QVqgK&>xKHwDMW#xC?`hW|1%NhyN4w!BS0RJ{rpFF*WZT^a=6(7NVc z4IX_>ZSAc;e|WgLDZkzV0BK+ajs=%_r6lAJ`U@Abm}U;zf%>0o(`>2C$^ z$u!Nh@{u>W{T)3|!}OF@>7cTnp(Y^nW6~M3vduJWtS}zg>ii)gc%l;kQqYNkPb7_} z=OEj2`FoMNyJA0>CGg_+2zAfI`_oCQ?hv7xaq$#@o@MUYBzc<1r}Ny|h;|Cm>a52f z_3$d+H%eP-8}8cOC{mgl*Ll9NslM~i7`&eey~J*+L7bEGEvy+(Vqf2ACDvbJXNS%W z*d=nxVti~YEiut~f9n_M&$Aq$#fB79K@9_%U+$;H69Rz}y2Ug1HYk444-B(9q&`cRc}Lz$too>_Aa*baaGv zCM;YvFmQLt0hB;c-~JpQ9|JcFxUS_*Tx@LHwdUspcTexV^#(b?*3K|v+Vz(KoSTPx zKD-Uaxw;;uitcay`oB?4wT(WTT5~K2t#d!i4z#W=3NR~B9rhL$NvWxk@n-)15{KlU zeO4#yO}v?%&WrL13)3`I{b-M+$H&K)kdXNC<45ztHmJsW%#4gifObtxNP16c)yv$n zSE0yYzm{SL$qO4B`#G;kSm-}Vi2S*T1yOM?ecDi$(tovMUw7v283JRLzJLG1k{8@x zpxQjYE26Oa0GWJBNlAfjsgfpuX8?8_|AOZ1?2AC^Kz6~s=%?Y((bWaf1Pc=rx9}|H zATWL|wo#!2jHA>x9g9`>e)$h~9W|Yf_gie8-QDQ`9U;9S8<$j?bkB;r#-|VbRV!Io zTXUM3p(k}PGMb2ph=_|L{ugKBTQ7j%si8D_!8|rGy4A(Y`~CIVy^&0tpSOSoRdNC; z0&3t}Kzi`y-4qE*;?k!4=x(db^BEM>OAZ!)NdY0yGCwvG;NwFgg!Bq=hTQe>@@gsx zIMzDpl>bZ0{9FuR?ISUE9-7K4vMpBk`Lq&swz!Ti(8}OTZ&)ip-}TqVdIR=N zfv-D7_%cw3DtQN+tDcCXIpA>Kx*lYxUT_2IK*_QRSa`$umn(udct66d z*|}SO?|ziy0m%j4j$!K4L+#%S|4Rsc89p&Yop@axE4_TOi9>?^t9JbB-KR{#ktc7# zfTQo&;-E2Fx&L2-^xw1beuy}lzBh0X7+1u+XxxY6b|xyFT9f{+YbyR$F$r34Q0L%F zwCB|`Uh4GFd&A){?&cK~ECN8)YZ&f&D*iC>jORml)KV@R8judT=WVy`k2OCxL}`A4 zX&06mJP(G3hCn_*&zKw;iSP|fS(%wBbzjs;%%5lsIB5g)8(sYWeb1ZjrrAaZ!cX9QH8(~pVY^Ioe6 zedBL0cw6%1>t?oZiHQp`Ok9>Ik&v`0kr0Xm!TPC0JrCmeBQUG9gb@WV;j8b-AAhg+ zsY*`12dsU9eo6GJSFb=Q2ns^ny7%=paB>Ct`5^2~)p=1EO_Z8ec znLhCEMPhC7iHRLNEwX$uan|@7Sde#}LH2Jq-;_4psL+OLMcscr|N`&eBEW6F`W7CHeZG zJd)kLqpV!2F!FVDG*UI<3i{l`ueiiPpLKP0&BgJ}*)Q+mV(ki=Dt-;Dc7_v&Qjn#_ zDrG7@IeR4z%N9l}u$pP`G?R_VtVy#px1I}!+QPmR^dyC8S_=9xX`%)X;_?;^)+vKF zJA09q@8YF2*?ak<-;O&-|I#m8%)V0LgTc!Ek`1)avte=KqNC{n`~c)L&4iND+~Sca zpTNh`2N4zgLMAbh0BkJa7KmbvPfJOWx|^s9k`M3Qo45i)Y*l!Ui4TAFW&%UWX(*E+ z@QD1`jqBGF#1gSINgh)csmP*07zo*r+zP@1pni$gf;(j=Uo!_7zLoJ0Y`-P%UM^SA z)us8Q?k4k~Cc(v`EYH(2`V5&7Y<~dSZ?|+e*<21=&(6}4U119*F0iBl3CcqkKN95} zwH|>RyuLU+eX0tFgoGp|Q4%`!XTHEBjZjflB~#~92M5^D@NjrnbIMG`X)CYZ{_{tE z>m#Bj3gxyuCfFK|ML^AczYMbpe;Me62l>smr3d35T>F^?LX@yrsbz2;Zg~2DoTDDO*vYB}HU)1lubB~$(1q-w#s;(Wfmq($ z=DtO38zO-kSv^^X3p=JqHs~((KUp>m7%XLNdBZdOy1@F!(ZH3RRk1w>ujkw@X5C^N z6Cr|V6{|!s@6gZkwkyD{E@9;izCLl=h=2$Y>cqWU6^5@(b)K<*^WqsyLkR`)2G#I>!do_ZykBl=G+eq zhJRBYl;7xbU>}u`N3PUU+8g+r``OxWsPC+5qE1wh3T7f6E9oFeW@IqAP{rJ@)(H8G zg}lbi9kKyF3zE?S&}2bAhdUDyK`ZRQ;Ei}n0KDL~JU&^21_pwMojnx1V=^$#ngUC6 z<;wGnjJQH;vY|NYQgWV)K5%=$*Yn^hOJAo~b3OAp%l3HYVsJkpsmtiR*)yC=W9mfU zy)T3vqZ+z`PiyfB8&bh3kg?f;HGsr1R2J&!*yyOVEc0a4IdTSucVHyR&4qRv=2Z-3 z1Ch2nVG=b53=J} zv3=TuJoAzGii(OkeC5^U<&*<;Ae}+`mqr2gem+y7IF3m?%5p^1CQLo&JD|u$QgjA+ z{3|7jKYMIes(X44LkU;|vdI?V->;?mFLSX)OfU;8_inj%Q7sT_CDY%3YQy_3M7$%Q zVnoBQuAr@ZWfmt$FJRF0MOEGYp(g#r!hPP1pO)gahc#cGe&Ci9S`S&bqMIv>%{x*= zI`q2765SvieTOjhJKWSFcVw(l<>4KY@qOZST$1s295*#JSb$1l&Jrc$zmtZ=KKw3S z#cGUJy{rB~<KjcO8g=C6B4gZF$|Ogr9U*=(>0QpYrV4eE>R7fhsi5KL_F-tSJ? zB6}s!0ge)C!fLbzKXW2k_=;{#_~k-%u}=3{oOeRT@!`725(3H1p~q%iW% zvA!YI;U+*3xyn;ZD;!v+h*-jqKa$%6V`Jd@U^hgz zBVl^I$<3`|$k4BDVFvhLdjC{BJCdXVbK6bA+e&Yeyk-f0&jZ}8G(lU;KXU0viX1foD@Vs-AThZnKDb`V9Ytf{vst6g zVPiE@rqg3D&CRjJXe|YqT)^KUvyNdDIFFoc@Dt0Xo=ur5#>B#!(IwP8&OyXha`W&w zBS2k}uVCG0P0~btI*~y^F(7u(VA^e>U!i^y*%50VM?1oky1pLn`1?&vVXdjjvj=`= zQ`z?(yFM<~?7ZdZUDa1F*ulO{(2~scFqJ!P+Ex;WYt?UO`uDGI<_gC}W#9dNuHMh# z^P5#vcE238->3CEm99eDVVm^cUSfstQUJHpiA`@df6LT+!PLjsHM$%=xz&X}k+~3$ zneVOtg?RFnq0?yDrAf|@2FHPo%;UwM$(%yRQAAZot64lv1Ts6tlduHvokJ_$IT+MQPy*|V=^nUn5 z=Yx(_3VULE9sU7Y$0Er+3zSfguhq=w*=i~>9~#kX&qt~pOx0pW=S;#BKeZF)4P5H& z*>=I-%DmjCVP$dI6AaKy2tOYmA8BbM_~G8Y<91;!i(~1t#=T1V>0He%_utw{5JKa1 zUN@qo%Swk|8iTN&C`rtC(+vzt5>-wSA9KOmgbIs^i8&>tqsQ^s!tH6Osg3JA*F9E{ zVWchZp`1d!Ezt!D2PT?0MzIiuw(r@e6WWOqYNmlUkMuqv4leu8dKgFag2}49|Jcy7 z&SHNZ0)31u2jVxFa?qs{faU@CGsqfriVVFr=0Aa;w1oAoos_yxhvmi%icP0<8@o`~ zoLd+1{K15(2mGZf3#DO$?*&Gw!&8z_K=)E#%1BEFf2N|RTns)g?nkm2D;Rp~s2Qaa zq#ZRC^!4cwv3~ueDl`)c)l$*<7m!#&F?Zq^L@%lwFmK`u?1vdMO8PT|$Lxl+C@L+v z2IFI;-QqzuBwG;d(CyC9YqV1wl%wLyo&?Y2WDLFMWj<1|oQxP9lxW|ONu3!W&wQ0> zVC*^S0;RF`$GdOD15exiQN=fjx$5m6s?$zP`TrtJ3ekyUJf2)IxX0@IN0`xH$)mN| z!&yhU{7TPrx-=K>+vCp_^!POBGo8jg3+Ki!P)Ob^zrEM$&l7o@`Aui+T6r2f{~lX8 zJ^jP{0iQ_WKU)Lx1d1d~yU%i)(TjNlVtw{W(uRL;#R5P5v#=u?RoWmlZg;9Z6zNc8Gd zXai}r{eH#|%YUu==?`8Z)Z??V{`}^3&_T0UbWzc+ir-(P( zQ*V&wNr|-OO3ng9yn`TpJk;EuZ_|`+y25pdp9I-) z;xtJ1<4W90z&@&HuVo({nA&8aBIBel!3b=~9|C*T+%6?zi5jCL--yZX+svPOnSyRy z6`!Lxq}DHcd9{!Ci4?)`Q!v{BGY^AHMR|F`(}agEuC9qZ0|NsLjEsq#azxT&3Nf^% zpj#q3r8G>P!)4!&fc7=OxugR&)V9{m>|MdR!)c#I6|%p zzoPyD^Fbd2Les_NdOGdfsMHgf9ZFU~b1IRzdKC{5o57Ou(bBZOxy^;LB@Ofwk#Z#z zP`W{TzJ`Vjab{p@=gyv7Y2fVmj3VMIiw$&(-t=2e z$uE;WW#er=B*UriUGcCs{U(aYGW@4ZEl$AJkiO!tLuG26=)7z^ zC11OgrWibo$$$ms8+nB3w~kvP&G(Dl6N>6hE<-Lyg1 zCj9C#S>3NTK>PD|JB{UGNL*OET!Z-{d^p?u^*YIAyv(?jOy^0rVH>P1-`c7}-C99S zdgJ%GzPBvL!$%fTjZbz9dzU|4D-^vp)vk|OWRlM!zBPB=(ais@Tiab#|M@LEW-8;- zt%}TGlk+{SKZSy^8!EnOF^M(T^=JO3^ier?ynk|h-J$g$pSg|i`opmFqB_ABibPGmapc^!^{-{uo0 z9Xa&}lf^;J)!MXPyDoR+`P!6T28rF5^|n-96!t8AeP4_UXg=l*i+0H^-FFkxqbU-# z8oSa&pUgv;M|ilVN>VVIb>0SJzcGu?W2B$)w*&PPS;>{WYm#NN!zZ2x68*Vsy?-V> zxMoH5 zzuv?h984V-kH2HJ3UX_dc(e6uyEe9 zbP1IB)>Ho!T1-dn>)Oo1Yc0ZuMwT~vU!?s~eezONWTu+`X5f7!H2G2!S{c16@6Xc3 z<}HNIo#+<*ybo{@#G>uK=ZwW%(BGx$lf^cU_aqb7Cj9)PN*3%THsyEj=3H?$xwcZz z?oIiQ##qdu)#uX{)wt!MklP0I>a`5V7qmTl=|bBH6%!+J{$LHXFsQOmZwY5VtJyNX z({bLIQ1Ih@{daqh?gt6_n{D5vc67HW;H=d57s6^R`%&|h;o0qZBLR==$+WlDc&u*{ zk9K))zvd4q4`Kfr4^dwFNM2|m z`Vu#Ft?cLTjqa*qwS*N}toJ@78r+&^+VzM!k`7>Ba`BA*446kto!7`zz&-c{_bWqf zE7xZ?>h7yszC*t*zS+~E!#TJ3V=;ZB=kybwp)bz1u|p7SioeCqyCehu|XVH z&+R_!*S!J~$uGoPCtP%9o#(aLinS_fHScbj%92@8rFzn;v?eCX_b0%E=RU5#F zmXRwJkZEmR?>TnD%1g7l6R>wcw0Fm}0M~!AN*wukb=i?(a5O$noprq6WE3bF1i;K%fQ!oxtByzS) zG23x*Fpj2+`@qgTs;)e>mok%7eu>2_48Yog*{u=y4`AfMegh^25C#E4(T)8f1*6lF zYZse$XvRC(

    tFBVJBU8GEq80<$Lq>bAUmXuf2AV|n=&SjEntKaUOoyG2X9k?`5TrynTG;I5Ji))Pf63OcA|1=g#M$A?g<_X&^@?$Wu}`H28dbdk+L?VId*vfc!StLwE3J5&ZSWpz7bd z_Zo!tDy901>}_|z_AzV{z}OJ|$_L0M%^Za$!&0XE^T4||H8lZgD4WwKT^BkPJ(cIV zvtl$a3DlRK$aT_ru#%5zY;S=xFNyN2sMm(Yu0VQiR#wx*1fwEsFU8IaQDD>jlEN>$ z>)`b2adlZn#$*DktVU$QF_z%)&`|sM^bYfNAk~Yjr3g%X$}K*T6LNGXi1{>)R=Y05 zt@*uf-17Ir4+;Zov75#&r`-%!nI`#pX-a{T7I=s$@WeQW@x8g)psfan%AT4(hw5Ttp`b-z;VE=ZTJXiZD3=`8X{?uzYYyubetjlxckrcYE5yyR6tyMdJ%q!h{V)BE;no(cFy4R zDBpNr>{H7z{+1)2z&x#%hRPlOh>+Cx&z@0V{sy8St=29W+*L&iz@X1fItPU71+76I zgGvhviwDKYT)O!2E-i*0zzcdtYXshbxr*-`Kg!U^NE0Ks3?h$=j0~jP6tx`r-P5f` z3f6#76}YfPo`I#R(sSJmQc?~(Bs3h0?xkU)zc^wHm_rX>A|Y*kd=5rN^b)0V{F7+f zLE~ybBBiypwKKaWLEIamv7bS$(R;YLwbn5zA|q?oP_`kI;J3hRhp8|LoGenc`P6>0 zmqHrMJ5IN{Dk`$sU0j*za}p}asj3pTk+mlOLUHO|{z8Mb4`#~BN}*l$jd8XU62=+& zxai$UE3R8qQDGM6GZ7 z3u;(Na}T&}wlNX)w!_!7uX$q%a>@xOAiy=%N3?}=o6IHV)~o#d2#Xkr!=9WRCvq_R z!REKnbzta9VLL(mUiVcfnEOmnk%+sohe6fR)K!X8Dl^JG^+iUqR*g^wK760WH{js_?Z z?7!u#!KD=(2IDT7cT`^@NCTnW-72(iU%a4KSOKYAE&CFp4HQV&9Va6z3mP+-FD6o9 zzRv9z`NQLghxz!@803T#jlO$j`PiKTK_oKFcP=hTC2AVOJrSVG8UQPtDG_5Jj63E>3c6(ESzXozSy|^b$z-^@FZ9W7F)#+N=U@c&MK`mYDPP%Al6eTCjvy)6!@9kv zaDRrrE)feD>O>ZK6Ka+~Og<3%U|k~wyWK(M=oIc0P)Y}jg5qNp*xjH;PJ;&T15121 z>!F;2bQ)Qaie#!3Tja$*u-`IjNoSYX>1*oCm$6B=z>d&)v&J4uPk}-P#fdGqKG~7q zULu0%!6B3}iq(N=_CDD#!lv6PG8t4y`+t0}u;{oz{^tK3T19Fd65ZM=H{((a-qHAY zb%~)#SDI0 zp*piR{1B1#4-WSBJSvpfF-^oi?di;SqfJ^{5J$?Z!pNL4sLJM|r~DcUwmQR_hJO$@ z1eY9kM^GX9L$#D_5?L+j4bx8?fj#Dm3~b(U4?JOSFxYHhubK2>sH6qVMxy5!yy2qq zVgCn#PnH_PFW~(9F=b|X8~qbmWiIxVptq^cv|)LqzZw=D3y-vMA~GSLKd-okc{5wcW_3D^{rPn4=~`!?Pq z+B5=tKQp94K%cAy4FlV64`*jl5Yl92gV$cqc8){aRziX?SV`Nu=i4``WJq09Cc5$I zQ;O?u&~2Tfn6CD9hQf{{suS1Wuo%#bcVm5JGw?uYzkdCCF-l5Soqvc0mCeYViAk!k(pW023oY7o*kt5&5ez|Pf#5$-MVq({r&e3S)&PdE`tSIwgRO4#P4u= z%$wz#)N#70y4%#m-3k0Q2n2#9+B5nMcM^>`jx7D)+`R%FRt*+h+$wiz98GLj{TRW{ zg0+gtl#d|$o6%6VB!qZeG(@Sv_8vkNZE_kK*xy~|^5afEU(f)w77Z$F5$x;!G6Alx zu9}`;O`x5CO)mNPY>d~Ujb{p-`1LUTrO|{ByY%9sqMFwf*QD9kT&hLl5R9@6?@LMo zE3=sxc!QFyJwjT+m(h*iCmV4cahXnrgzR?TGH4Q}1MvS88o<6BqQP(giZvN)DM8RB zZQ2DKO|qv**pg`;#xkB3^cFQIB07RK@o{F2>D1K_jW|N!mY*Y}-3VU_!dzRM`z~hy zdh^WlSr0)lLsnH*;**^~#sxhF0Ut%k&Bm6^hyO9Sl&MSa>bS)D67EAQ@k&tCqN@ma z6s`v&BsLIy+xqq1g5|WSx6y=jK}$au7TA2U3j9`TKkB^WzILs_ees#+OElj0TWSoW z<|ZcK_1@!N`r2_4b^V`q!61_RSwl*tnV*xjjP#C57C$K+9bJe486Pc~K#294@+zBP z36-lhp@7^AaQR95&Y+r8s4g#a1ZMHj6*kXt%dye?VKIdNMG-lKIyWiPL>5#+ui}r_Pb$pRV9R1AGfMHrTCYJM?ZHj9^?` z2k{kn0d!^HkOSBJvEJ5ae@~Wi_$MK_mLvg^u;p0;6|iuR+|lx_@F&wzLijLfvGJ^j zywh5eD)TNgZwHviPUTX6%f?6z&wN*1-REnY{LHAUhaCvXTg^AJ+L$BHL|@2TO0+K4 z=}2SpLp9^tsTackH*b1N&&3@O9>v+2W*2LfZ#gV{cuZCJ*TwPIZ~+I02_f(B?P0sa z^VOhl#IDyu=-jAlrg+@cMeE&G_1ntdclmQ36x%rTxJZ<$cYV^9@$}OBa+4gT^U^Gw zdgt9e%0e3B#dOPo9`0I`-{*ID>g*z`{P6A8lM2)-RtGU{HcQg%>#SvddTI54W+oW# z#r)O%k{=z-_?uMY^nB0G;B1?b1}bx`pmt<#qrbO3&U1CO_{aoAi~Ic#-5(Ld1e&K# z>vR-?x3v5?!}gfHYnDY@=e=^y9~-+mX;kE45t(Pw8R_wFVh&#&z$MbytV~Yh72I0(FaDJu zGchf(8rXRElOWgEsXr5+9nHUZ6l*&1zYw)N3}^OR!=@H0OsA<)UDhe%p^H5pXE_2h zk)Io{pv|u&XJ*>`IIe2DL<;k;EjarF>>*Dq=ZMP2!f^;eT`APsDj$ zu;6{T+sP@9HUG3K$s*iwJ8AJ(J^H{LX`r3ZGw;ld=7{HMbkIsR!@7Xa#Owb&0I|O8Vv(^GU)AZhUuih19({g$65>ki*v!k5~c8!o&6z zrNFT(%=Lo4Yu`ab4EH@8>dad8Z@5SFMBMTa2e|1?voBSi-7iG<*!D(Aey8#~e{AEE z)uc$llUo^Xo^Kkm(03lk7bdukZdj9%EuwzX_fK4$nu5*(H6UQSO>hyooB4-CYPTGg zstpm@7wRR=HImk&2^!B*J1?aA1=hDdubH2NOz+m)7(MC>o9+( zf>P1sz-C=x$4gdXrRtkjgZS`0Hw}$V`^n7{`2SSC*QdXfwoEOatVo{nl%S#+AGkPs zl;Rij-QC?tcO#;tbW4Xc+zFoZecyk_xZ{p-#u=P_cD(Pq)|&CmXFjv% z*sN^c``gKCSWmh04tMxY9sUGTEXCNFYd@(h|EPAXxT}a^B8UE00Yh&|T2m71Ah8Qc z^@z3B8nM$k3Fo4>?)(HNAkThdh|7ysNIHyLiTo(oXfCR+%vt{mJ(4P-ZC$HmA?P2xCv(Ak+GIsk2cZ&l&0wCE4HuCWQ@wgN(KftX(N}FmfLIy z^QcNiUd}qDC+tTOv-H*)6sT=~haxIAlwopk$v^zooX0y!o*)$(z(`r!W~nT^_q;wm zLRs{l2$)`=eS|J4Qv8vrO&K9n8J4-#L!HVfH1^%Gv?)1=u1d}KP-0r5EMrcSLnm3l z@#lbtLWH8ph+X?mMx3jF>%b8=vsFoyORmHpG0M z-m=RX?y(<@{oVg@Z76){Nrd@3AJH#oYLj>jmTgu9r)gKkq6(vZGQGS1qxoQyXbs>5buwIt?sl0`x)@A|O`QTh8S|+~t+1_?mq@1Cm zXiu;AY}u@m;n0^fn&3!u`Q2sOv9b3#xJw^$?)JFU#BS*Hy7s&hpIZ!Ce4&y*Q>~9b za7)-ZDa*tVVgi-o2rC=x|JNf&Q_X!)s5L_;V!DFWxyy99j0w=jRWr6*oJ$OThTJS6 ztFihvs}q&6I0K<^B28nu)+)tI%@5IL?h_i^pcU4mxSd0)B==Uc|5{zj+_bsPT@w4XcdtA<8Qa||ZVtm#Bu$M1p%6=XBF4(UulSHTJ z3hqNMAGZPp-09z@yh0~q*87Wr+;VRY*sL_!ULD5UX^o&s;0|YVy*~D{lS?Jl(I_wu zY=Bbz5=+pDb=mxnjb3oS{7xgtn@*r>aT2`Lh*fLxg6qUQ|7F)TG_bsF4{+2Mc=nF< zTyCxtT*uQHkxBmPBh?N*7AeMGLCI3JJ$J%SS$*gX8(Nhbf<+OmbT^>d;E! zuNj6WlXI!NNAW_~s;(u**P0%=oTmG5HP(*hymDT@uHf@+~TsZr1aIsrkM96Z^lNGZKVUy74t@ABsB;y_Z4zDJVM{DenZOzVpi6nc1eGG5O8? z$1wrez91eAQrT(HF~g)8ieKT+u)8_EPgL zTCh`QAlMHcNlZLjrDGjq+frT^Vf@uhv*_D${K&HAbP%d7#Uvj%e?-~t4jGG)l&8dK zeaM{bqjp;+b^PBRa)@F=7VgLY?x*@$(bg5s)z|wB^ImomPfLSU2SsZI)b|Z<&}xSL zP^F$&@2*uK`|dVB>p|GM!>`RV)%h#Z%$fl(>T0v!P7=1>1xYrdluVcDD=JDi@+xprp)Hw{f^?hJOh<% zTIYz)H@Bm>VqD9Y+!cV(L|;|&r$1jReLDtqZ-nyC(A&d2Z#O$eB=J_Y+pmUAhKWH{ z%E8#*hZ~rc|7yIHw!S9cy z*3|Tj)K@=lD>ljXMaA9ANfTvUk?J94akl;F+i!1-5W3v~xeMs4a=@qrKAEAXmltp; zZ2`&#A(r|1c_8KKy#NsoXm|BGqrORmmtS3dvNhKUTsNRY-P6kkKoiIH(oS-2E?!(C zFsX`+djX_}T7yU-H<0!$O-W8BaQDznRU^YJTcbax#kjxk#tzQho3e_pyRiA>j8cV zXT<5*un>NJX0EN9*j|Oj{CGt~yL@W0V8AUVioojT+@bqOA9sBe;;YwC?ypqujXMnv z1yVb)+diznbj4HhR-(U&^_is_|AHH4pFv_GW9@`Z?6&zIZ4d~o+09vZ8Z?Tt`H*e7 zKCl(id%L>Y+WI!q;Xz)n$NaC(XqL|hFVgr7zc(JQNnzZH8ru4y>Wb#jOFf+Y?AM-b z&9>i?1O-N(b*^f)w?!j<8LJ7lM$nq_LRaXAZ7F__7YjkAq14~`9WD9UWT(ElEayh| zXI9x9;E$g?k4jC^|MNzuvPR-Z@A&@LevNG@-l>Nng6l5(YqTf7vS_qNYivdf*FMaJ z%9~tUd{o`#=Vr8!p^)q4V%G3GRQe8bwZP-`+0L@1i=fNICsQv<-aOTX(mzO&d zHpNUjKHrMxAX}cClWOH24H9tnOz2Y(So)4(f(7{m3zmva<=ylLno5f6wlV{?sXj{yzth8h(Sy`l%dJ)ZVNd^$Z+b)8yOFEI4l6N&D$3^xq3 zP9;@!SO`=#!g+zdGT(yPLj5P2_v@>$Qp1d~A{U>#43Q7N97N)cT%CQEq_rKiO6kVa zS2bViW8ty;-l|qhZsfo=d!XiZL~JhcpkkSV;9Rv~jl>sg%RA%~!h2YC3L+CP<{wg$ zj`n{Q9Qv|po54Y!e!9dOLvyk*Gvzl}v0tTHP;_urn|5XUV0=z-<*Qr4`cgObkzHPk ztow%*?d7(<=#hZ}*@{68zI{aYAT;g8r|1YE(VU^zw`A;r?|Vuf6NC&iBO{^@`on)g z2SL5;71ZMUwsFpuH^=ht2)M63sZf1f2thhJ zccZ=+r^zT!dh9(`tcrM%I<>C9G|!uG5pRGX$FBW#KZ%bNDTivZMCyihcuOaMx!P~v zS-WUjeR+XGUvph@SS+hX!eoRG%xi{cequ~SG@6f5cix~{ z-!wnMm|j<(R$KZ$=&>^76?;fyf>OMbg=WHI*g-T&wg8H3N-45m=z0Ko!#a%zJg&a{ zj0m|fKVR4iRYFvE%=B|gI<%XGDg%peO4mkccnc9w0zJ*<78cO9@cd*4Ght`L> zLug=YoM4c8@9r*cfvI2bi2;18Os%R?jChbt1KS~^qcZ_TbIYN86t2{sS2Hb_p)(ED zLWXexCdUqD@lp@a|>Fe zq2_FL23@!UzeY!^pkjz43BbBm8&=8KV;$w9|e86%%@pD(;sr_ghINd)aX&*grfhWeLwk8!?YMq;D#yhD$V(3>Wh z6j%|p7~%PQxtHU22`=scbukXT51|kJOVGORZa>#S+!nk=5~mjx!+CbIPyojf8fBA{ z8B1A@*mY`f+(kRSgA@bk=-6ya#*j6QJt_77?lAx%hVyAVu>!F13Zf0y-f7UL7_kvi zQ{%?iRL*@`gmUHsih)$&(NqTW?bweW5h6D9k?7h{%IU{ZmNlF?`XzPuD@go?;=||e zn)f8Co88_op+DaOtp8kQiVg!(uJ_^pzH|enm=X3ac0Bx{4eks63r@q|KYq$xY#Z!& z9V2&u+qFTN6nt2I7pwwkpxoO=Csz||HMfO3gOb>XSYZbO0^VL)4YA7>uHT~^7b$Ll z%FXM;9aA>?|NMgE#jae`K}CAC!De3UPw51~599)rVLVYmDgmMK1T{lcWMu${rTL<}-5M8w3+f1tA26{en#mHF~}Dy}^>i007Hj3?g5hWg^$i1qr( z9jFw8PTOrZHasQYg!8T&9}9DH3O@5o$RfWJi2dxWtp)8?)Dmj-Z~o`*&L0~X6fJXv zps48hj~}n09tX6Q;|TotKoCb)&v<;p&T}LVWZl8eZo4eRTq{-}2fvC-$tNGh*A34Y zQC#sBhzvs5r@t$<{Xro6 zBe+|C{`@mk@O=JY3Hqc!Vgl$Y;K?riV|BC^Z?hKfrB?jWUhS;98@g@9HZhvhF;I`o zU{@tpJ`g5&x6-ni)XPP5Z}}m8U(#X6tsFKir`eW2MI5g>hPCCGOBeq%8$$$FS$wsuCP*tlw=D29z*M#L_Oip?7*n4=0qY5s zC9FK)CMiAr?9G^|l5+@xV}XSDG{WIAF}O&Zn;aY*oScs?PN5v?;yp_m9l^e8h-}xt z9=ugbljXR+uQA=N+TpyvKpdj(>oWHNZQbb` zO5LUpwbq@yk1;QGXg3Q#Cl94o;PLFP4(cQb(OGxPT2^mWomrLKMmtp`AK0*TJGG3Q z`mK_pv}Zg;x-1p6-}k0Z z*N_hC57sT8DS99El7HJTQFZIA^!d);Yco_C-(7%9og24UVEW-?oLF)2gVXbn;2YXL zMHLTKo*>(L*qCcpkbCF;6lEumC?3Q}fq;c^M8Sj^PyY*az;Pgx?28u%<`u-%OQ19S z;DNt`^8x$;F3e#e<_%Ccpd<>UZJ=a`0*M34vubYFTA`yghjwam2dD#gG)nJqb1VP5 z)}DFM?_BnB_7jbo%MjA>e4EL8hY>lypSUc{?CDU>Yhwsl_ zxDnN|uIhaKoSS{_&5n)+?{WN-_Q6V* zM>K+E!pl~-%AKVBln80<&EgkY~Ie|*~>lc zWjSn1t&LwigQu(f+OJ-yvTZ(5Li_yY`mu7)A!E$)UBUPsO1BeQ)g!LH?YHeYk9&6u zckqmS+Lp8e%3Rm`c9I|m=$z0#T5m_A(9KP#HmjfJdH)*Yn7XF(q%M;^)Adh_^!iCa z3|Z-$untu`r`)J-mB-9(IohY^$JBj=+jlsha68m~d(SCUsJ?an9+<^WrV{6~^*lw2 zzR9jo#lRP}gUM@7*PO;YPL;;tlO+TS%U}1};WhF`2oQWzS*U&tgx`L-(d%sm~zcSErvf8kFuntESZ{=B$-;3ssIqkFk z6S17AVaLWhN7l4wDItw%{EA1>BV(UBvQ+K2>&zDTx@Dihr@fyv-6~FoTnRkOQ(y5evVj@dwq15mhAF^dwOgJlsWP~!&Wjn7nigyfC zIWNvsT6&~ppYh;*@3Lv>9Ym#2SX$74v#a!i%k5|um%}o#<6gJYbRFyZ@Kn_x$NQ_R zS2WHx_i9r*Va<-#G-~(Ebf)?!NFVO&oX+W-&P<%`@}C`8<{htzMGd&_HGE(b*Zz^P zO#N=8{?PJt)lKSfFm+bL?TEHVU+Yn=R6yD3VbziZ<*&16_)_PgDC*)gZXf3Dy_Mnb zAU*(|c?@EbVU<{*tdWtK)RfQoR8=C>36OH^tVrhA74pSLxVGB7B|mmU#<5#@y_QFK z&&axD@BZc%j8}}nKNIe+?b#(9C&jmo6HKQ!QOj8QU7r<`bK8Mz)N+qY`ch$)oJ05) zq)QoIlwTazm3~EwY6vFWFX)(5LBKCEQvdnHSaqqQ;Kqcr)g*^ZVBYzDdQ=U6VVT;=T+OfeiPXCn0ko>P@t}9ebP77su=51&WD8D5%xVGe3&8=-y!i z{R{?%8N&^mi%?fv-Q9PnZj4>s#oATc_yOms8oo)Pcy}#3&ET<~W^KBQ)>PZ!4>|X5 zDYB=VaQb$aRCd8!RGjTE)$MT9HqS&gW^?Rp^89hSn&v#Q_QU--S!C9J#nX(( zYp<(XX!+8$!HDy_2tF08PMMj$-2MA0djX&$12)w^<%?2+mBbF^Qj~Pge8A`2T8ML+gdj+n~9v&TyE?+~T6Nt%t{O7emjg$|6m7t5AdD3q$Ighe? z&f+$XJ4@1TY$sE36=Uu+ybU$hK_jCjstdADJF3O8!EdxqRGaP8dw5$=cde zs^H1eD0DpFUw(R<(?J(a%^}&%!s_d@>R<-6pmamdhCVH~bTf$O{>n6Rv%4cjaV>UOTc{kSEE`>nx`-X^z$jU6MoqT zn|KqQ`M7tlqx4X!R=CP#!btuz*vB9K1c z!{@dJCw{jdKjUw3{nu@rr)iE$H!gCIi@@Y>=Jq$i`kRORpCkz04p}w<^7r{i0M>!5 z@b4ehf788x!q@*Mg8zBpZ+iZ}Z~OaQ|7Ps}WRHyQ7y079zXLyU@sW${^Is(5f4=ko zm16$Ag^LSJ3HzV7{r8$LjPt(<>cwqd{J)D^$+`H*Mb3Br+5cwc|2&9`hxHc_LD^Bl zSOmNRX8RO}yp@Es(q+4w|Kc7U6%o`<RM;{39o`ebhgvdm!m+ODtOpn0DgVINlbPx?>-2jt8hp01%^#TF~ zDINs@0V+F*BTRtkfW+>>Do5GwX?AutKv1ChcL2HTF-#W<1ll&VOq>ff-)J>{6YXC4c3Jqi)KD>$n;ugJM+uGVVq#?s}g7HTmVntmJ z;ZHFa2M%Zz&};Ndgbu&)!MB%umeDr)zQM`%FlR$_wQeU(C$k;M>%$r;HW4K z05wH~g~|NG+)B^QLNFujtEIt9eXy(?)37RjxRgIraM>6wbAIvVd{xH))EEEW)Ky9s=~CwKdJ{>=fXgSQwB* zgq>+EXsBNH(gMZAgn&Jz6ndwvOijohw)OP5j%lgU5f9*9z6{Uc1CZ!#6^~5yR8&ZE zFuIxL!x4J!nRMokj)JQRrSOK(TgsR{rBoXVuyCz+c5d9f>HicZX9OlPReSZ^lCi+0-~o(RLQ(O< z>!7-qY*#j)vt1lT#=kbLHkm@i(Kbf#wQdyvqJT-LwQ)gnOC5$o?ZG_drOUyXg{+8b z<6at2$1))}{OWvQt*fgG3fAnz(@?R%G`! znJ)irTobZIk0_v!XBhV<%iNIttOh+rzzpBkiV?YU`NF6NGyV;Aw#;Cg!6c^H=g?{d zhV4@<$Z=y`6D5y_Gp(MhSpjLL6&uemY?TL`P)k7VGjIF^NSlnJ;{5qgn7TC>d=`i3 z%V$8Zg-#QC318@9vPyvJl19e@;iTdmbaBLs1){5UKWDTTATGp-#|F`FA}?jQJ;fLV zmSoSL1PQwR#3qZG$FFx(s+}vwm2Sc+F=#K#|PR?UM`2>TI-+T!1SRxc4>=l$4 zEL@6#TrHtR8_=fI+)3MKVFnH7KAik7p-@TS78Z`yeoreV3hhxsN-NNYV?;M(14ua7 zK{(XsXfc!_G$WI;QB0Vnr`EyTg0K=`JRit7J((W4IXfT2`iYzA=;%1U0uUJ6Ok#N` zu%_=xCj%8&2(F;2x;eEZ0L7uN;JFqG5*94#HO3D-O1_nxvT@Y-0yP*f1H#<8wqB=0 z#?Z(EpuFBClViEl>KlK33Al~JFQ~;O8y~@vlyk7L^)xqQ+yk3Ka{AoT65qC=YH}WW zqoGASUPhM!3_=mpO<;oCsc|qeHudzp1RLh7sn-e`S%VqMSX`L#8MyhF9ngF;e6Uuv z4m*6Xz8)ptSGi(wMl@wl8Qr(v+6n}c2Vfr7!_;mbUtD~Ad^M5{!wkfXXMy4M?R4<=o)xTNG{(hHvjEM5W(dUU^4T+x?S077R%ua}#kG^Q*A&;Hr# zK3{TfqkP&k`Z&fGaLi3W<`FleQImZT4~*I-aLlyOAr?q2Wh^RN$3bLBzI4M zl4+x$_Z9l}w1R+tedu`%q9b^+SSHlyMzlXX(dOGK$|5he+^$KQ>(a1i1a>%_StRzm zU{mFw<_qvgV0B?kRAxmhj=a}avy13|?5Ygn#QHKS4X3_$QPDQ(*-a&-UI{-uLt|so zq#9W2u2_*OC|l&Hmuzir-UR@7)2#O2i-6FlS;A-B!)T%NjDZ2;epGF@@+-s68R{`_M835fpB)&A#G|L1BiuI=J`{xSUjxyOs3_y)WgZuUHk z`g?2tz1oY%_3yL&&()rhA!|<$f4hafeRy`de#Yh=>Nrh@w)kk? zbik-q5Kq*|M|{z&u%3g~qfiNaTG8ZL8?(FJ zkZi*ISFbyOoMetp?J1Oaz#4Ho?wDBjgGRFG$&Rvh<4li;&AdEZS)0Lu=L&d_)8DnH zFTglHxi>vrVPg!!b3o5Q+9>yH?kq)hgQaeQ@FtSqa&2-FNjg$ySqt*uk7JgIq@g^q zHa-0~OC<*aAE5O(vmCj zT%kb~JR>NWeVMGo0m{Zr#$PZA@*eKXD{%DAJz_BJ2vm4ie^rHT+&d=c zHG2YMreJbiM4J%_p+opLF`>3TT{8~xd2nu$_pZx>@&s7h(ey_Qjs?&KI2oPy<@mA?X+Q*e+B^wlSSMxmO?xajBW3tAo_!r3U1S3#Y= z1ZMN(?*qMv={ljd*wIQ6Of-s;5BdDL7<5Ce$2c8N5qd`?suc1%4Uo0l&Mm%Cc{U_N@ z*!||j`|s5JG@+t5Aj1$uEhObr^=6$<&L1l1@_q?zsY7oabsam91oMe2V$S>doOt?T zq_DKaX?=N6l(Gqcsxh}VPNJ*%eaB&MN(lY3)e1aswaCW_yqrxAv%Qn6g*c#v_VKi}u z`yK9>^ycj2CDz`yws`fQKM;ZnA~3HN7SU9glTk=03bjkt-vOdis3H5me?QC8w+)rX zI!ClUNCOzhVgz1!-h#=vmV)=2@&|gC-jT91Im5uJ*at`DqlA)Q0B+20NPDs4IjNfw z7kahiaBp4Aa$J}KX}}Qcy!%6ya!YcSzKZ~#|K(U)zyR8BcCAXHZGuSumZlrZ`1K!e zHdTa2D&JH}gZk(_P6~N{CVkb6$3D%M<%lZ4s??gfGhP0q?w@Z!;rSd=HI+PVg$8#F zSfryCN=qCv4um@s4@6gC<>n|Rs4hUsXj7ZbghL6OAWx+HPzi!(bl@2j=OLX4KBJsbA|xp4^%|R-AgBhNG)L8B+P__%^*ME+Tdn=SCVU{-$0p#xkIPYHbB;d zGB8<$Dv_KK?YB@N(RumxR_Q}2{u}Liac-w~Rr$%xOuoFn2QHfP6DW?)UddM<+}It| zDKq=g(bB?n^QH-e!Emy&bZSLc6T)GZN=yv3N}>-8&gKjVmBe8C3<`vhMGi=@B-Ve% z$H9pfapJ2u_@SzEgTcBIs01op?Xk|o5I}Nx%7dz76;QQM&U4_wUYy~ zv9sl`+c5HFsViAGcDgv5a#M-KTbJjBs*xm!%6vjcH=rSRH}ahx?Yi|f@|iqBkBuWB zT!_S-iS4eSAeRm#Tc`;HJnF?JC`;-zI7uwn$f#k3Y=#=nPfDf_oEZ;$MUi^;hYjklzk+uDwqNSe0-r<&Kvbu zR9yV~OP6k6nREh~P;{G0Q1E%SOpo-~H3|@gxg?-afa(9`u2-|a?lHt=#pc7oZFnzIY$+GKlZ9#G$<%i|?zDbQO-Au&woC(J-#=%CipS(5FS5}JGM%THPw3e| zxq;AxDU-kod>1Q1Vdo6W$d~$j!!ypHzL|q!qUUh0nu0Xls>qF!>f^;Z!6&swlu zvu%({%sYifHc7EX2)s&!`O*;t6qwW9BERs(j`JZY!LW%c*+MDaW#w@X=TXUga;u*# zgP#@&aD52su>%C6K=KIA zDaHyP2)HwKqIRhHN|42QxHAJK)YJ+r_HdulmMnXa+Lgk5gPb-NkqNWVyFHTU!VEmn zk7tbAmroInZPVzFPWv+g@{INGY53))C?@t4jz)qUCm9~(=t7XDuoB7bPuot@{GNhS zO>`)O(_!_9=}to9)!Jbt|D+hrOa6Dt(7nTQ)7RnIxXs;VDn4^E;&W2VDkuyTtv6O8 z5bg*WN%1EX?WkTRQbf~t0*m&LAQuy@tUObzQ-l0mb5i9{SMWYvtDy z@V(j!Zpa*#pz%B;BPHcG?xDKQKsH$G=7wMcu)H0iaNX9%2GnZK6pyt&r@_nvuYg&6 zY@~9k`EVh4TrN{1NMw{SP5Q~kR&;S?FgRUl`<|F7{r$@G?Gr*=(lgn|KdPbxI-D676?nFqCo375Zu_K!X+VxFXxp1nW{jUx(tvLjEUUr zXGZYY#5H8Uj`;e@Vfw0^idhkY$52-itq=W*JAtz^6_T*Y1hXhy5XR=!*TWoE?q-@Y zQRAPD9Zks0lj};L(QT{}Da~Ml*lcQ&QNAm3-I_q=l9yxvFlf|iX6A1y$I78)BU$R3 zwiYqANnPifK!92iMzSYcpd*<89I>qt8pDiFFk^wCG7XwQiK5r+Q}}b9qF%r5c?DEF zdlDd&(51Q=A^Nx%{3+ZkssF-G8-mBK#bib4!2?}hJS3sSIf^mu-7({ORs2il-_UjY zo#Z`#VL)9VnLXNBvVl2^aI+8iVYDRwgtHAGdz15!2g=BSV@OKgWuRh2OT5c#l@8Bt z{u&;eq?yN;1iPCdSyyA)Z$v*(Nm6HFtX9u%X+4H}o z1|Mf{e*TS`2iKG!q*)BeYqE?Ts3XG=*~WAdJ6bM-OM{6ofYa3?Fe_&jM`eznaHPXP z{kg*Y`V&z%0~I9cfGVJqA|2Se6B8e&$Rc=5gPdhDpNo|mnv`E*O`CbWJU|~*$mF0n zAxtNuB+*Z23ZklnZUC2ET_8kIEb|6gy{F)P8`0JSP*C`RK8ne4w10h*p3y6|p584o z67Ql2CVAYA0!V@L8Mr^uNr-ge>0cvyaiIR5mbe*k9h}F^hu4tdFRYluxX%QxqxL;0 zt=CD!b@6_s@DZgZV(ByI8*~$5RlhBbViWOoe_@{R+=5tRPNaLH*THq1L9VkohK@(> zi&JZBJ;VhGz-P=d>A6AlnO(;f!iyo4I@YI|M0PWkB<`u$YF=ASB-!<^fuS_{{QBjB z4=wJUl13_*fK-YR&nf!u3=AQm2CS*HR9#Q8$AfcKYZl|Q-LnZuHC%rnEMhni-D6{# z4L8+rLJN$+ilV=G9PeOMH@4_Q)PA!`MgI`UD(zI!in+_9W$eS>gazfPm<2ieB5j!vg^A=jkZW|9I2eiNF0p5BGf5c7JB}WaJ?65amsCa zb(~Al`hXz}$U8DgKMsGSQE7Y2zMg(0+mxX?n(K^0pNU4;{$mNPjo*FV&R+mU5KyT} za4B?=J|vLhGauqG>LQ!J4}^{b)hzk8(uqgDp zBl5!Np8|QWSWp@lFxD|;_XiXpIW{_2VlK5Dl4tUjm2vueyl!Am@Jdn($O>0_5sq0j zUMK0zaOJ8@7Xy_fpW?G}EBZ~p65OOZM?n ztkT^zGkI&ac!$RU8xi>7gQyO&?j*?d!+3HQGm(AddT|Hy_9d)Y?k1f)%$?-!F^-e*aEZ&QoM&dPZeSWe*hG9Clvny z{Lrc4+nrLNC<7fc=unzfy?F5zZJZlD*Ti+`hWN9i;|7W&T#)+Ncq!zv%k&b*hXe;B z`?WBLXlP!>K5=ybk{-ryW@ctFm#5GYfrUo}v(Ud>eDl9~@_*Cmi^TpsXTAV9E|TZ} zuLL_e?8yXBpaIB)1Rcyr7e1SMjo;@$NHdri86l5`X$HTXCB&S-2LMgh4W_t^-~A00 z5BAa%vBTyJA{dD!3Nj=h%uwU^iPX%zZ5CQ`rdT?~4;c7@N3s^!PT)0}< zOD#}f0lQx~(cZq)6hH_wQ9$PaL@|J#-e|15!@N zB)<>2O57P`xv;4=W@}u<0UV`4o(IDf$R)p_w(_Af!zBC2d|g<=RiJj{xtVs zcNm6JfEC zISrYxy;ZI2$;IAc`bldAPd*?IsqKyePAF)Jyc=n_JJsj(yjRh zG89O=!R*x7Ea2V5&pR*K{d+GQYoMnO!Wl3tQ~!^9KBUetn9DanT#21M54t66F0FYx z!juaTE(4r)4W{RWglt2Ga#u*`oLG4;g(LUfyH0TGz*a@iLG6Y+(<(J5R|CzMw=kl( zycDW5fWbidXT`fY2PHDnlwj=Z^H#iN6bArIDk07RQf>bOHVQaIN=3D>HY>t5^EkxT z^baZ_v4ygV-?v-6Lh<-AMOex6>aa;Sc6W&Z9sGN2Vr3DWYM1BF=RS>5fe`tSAe8gq z5s(pKB2j(nqa~%NK zfHVm|aHfIo|A7Dcb$wU}5NM@M*Z5#wzpgh>sLpc$Y5u2LwR&ArGSP=e8FH zyhXm*-_RqAJfBBy0lSrkh6YRuX1pH`IqD}HlW&RlKQF|{(rp3S77H3#kRm|sP7^j6 z6@K=AmqaNAilJ*@Y_D?g2t!Z-5c|2%FC2#XgLxol!V?Ci>yJrLMuf%U#}|Q7!xGgS zY|>zshHStg!YL@ibx@{cJgIfqE6RvIfrZDchq*!^W&0wK^T1``_$uKa87c4j)D6GT?F^38gz zJUzg<7bfrmUF>JS2a$PaTU$9yqpit%2|1P}jQ@t|=U<|tUYxkojYbE97}$;yt|-iq zpr`jptr7m&C|it$2DMxLkb7>uIB1VCpR3b0JFe6R-#=Trfhh=ZC!C5~skG4zw)PG1 zB;QYvj9?$yvn&0S?pCF+xx(;x$0D@^u&(C478a&vTWC`%7tGbLI*fY7snhhx>bliU zA)zXNw~e5802Oe95|ED7ZSC9-_1L)ILa%!3bxY>c5S<#-?kYVXc@`BBg>Du-Ygm+4 zK-_)XL`JSDpGqICGwqD-j^^jkS#`zBiH!DZ^M?248PY2T1#M=b{pK>`8hdX zfsFu#opUR?vjb9M@jkFU+&-s4tyX>zh+*Iv&MoD{#2Mrq$;E)@K$W!kdF)XB@tynM z{F_Q)W5IUSuY-ym$l{Z**QT7ny2GR-OUPN)!GJ*ht^T<}1Cx+7kug{eLTNx8!|pt5 z;F#iut8Du5%Ji!X>_6Mun@ik0Jm<6NblgsO8(|7hL}Hw%QpW!N{@?0ufBAWj!63|3 z$l}Hdz=3*RxuEOI>n$LrcZta;;T{T=Ym@)A^Qt`E+rn<3D`IVtWsuP_*w~VtD*92&RU;g1M9&G4IbuQ}mAmPy4|S*u0vbg2D5x z1Fi_=cNsGDMbZ~i6dTQ?I=Y09iK-bS2rrteCPEt99IP<4=(hT+?!P0^^8;s$_SlvM zk~G81?Ljpf6Gn7DCCXqfM!2YpAdxk*5?BS;G;F{8Fv%|YUM2*y7H9(?X+GHR29a-} zeU>#G9_TIf4QvjcojyK3tnGw5P`PZc00TTr0Gg7F7P`i4>YAEi*hcgpA%UU%s#GL1f!mMqxkAXs<3nfkvuvR1uu8!gCDR` zMpoP1@nM?Tm|mWqR?If|_@vFf3HIQ@QeXIGu|5dC-WIO~0+6 zhWNB3%%Eh>5s*$^6+x!2N$m)O5(({Qew7PK1jGb#KbKJPDi>UT|M@^4H}}KIUD-Ze zd?TO%=s}eZI5^aCx7hgnd`<6O?P@^tYhqpEec zlznuw;-7~>yb%R&pL^=sbUh%ailS5UjuUp+f}=mlXV&<>JpGk!10J8TfB=S{@jy6U zcGwV8DyP~!G-(MBUWXDnAxd8Q!vxe<0VFj;{vO*r79*dL;^lSS*BKa0&tR4U2ZI#1 zn7WKEqyD5{GL6X*@NOWYG>XvLP{cKAswThGfGw5eg>#kDd#X+@fRZd?WP=|SgU~xG zU%U$+4{yGIRzNdGjFV!Nj67&gDOU(GjE*SFAie3U8g+XBKyVrSoVf|7fz)}AXut%H zh_3D=L<2G(=XJ*CrC(nD86@=W=g(3)WLj!!u)@j7NyG*S*3l0JKXciVe1fqH^#|#? z>pj8kP&<)F%~ejrGj(5unD1KZFgQ$%i15k#5$8BregJ!pMj2C6-EG)~H=3@>f}b8f zYdEVm#O*U87H@*_h!_=8AE+)PqjuY8);K?UsA2Y9lXGW!N$|KpOx|5wphF*xB!T6R zZabLFf(?$;{Gy`9{H7OO{`KvjaZp-}ckfY>JyVe!%7F|X0<-#;IJQV6LZM*`-Il`i z`SZs3Str>vGL+&1%5}BT&Q5Ip)R}zMgx@GQJ!Td_Vk#$$Xdh`D4qB25qEVO!sZ?4C zWbAXIp%%^ZbPy`^$0Vt%F%am&Yn?2fRSdqFMW;Bs3}{>Ojd-+>HY%S-;f4k;v%6Q~ z$yn1ex;~Jeo`u9=^^vHyI&or`>NAUYCSGy8mMZ{pMn@gFeGD&B0!@IKcX9}ae4E-A zjZ5vDN9PkfD?($d*Es%JjYt93+r}g*&;`{H{vJ;{gG6=VW=X+T>yH94=ILecYb~Fz z>G=}4k=0DkE&oJV{K>!5G~#~iq*=85RZUfAAM4botAVe=*X?A8M|0G=fH8LNEW5) z@;5cK-+>U)oweE;q4|M2S(^2AV&8_%d`uLN6)7@$3M-<8*)e?4eRq@_tooeq&DR1Z zSBYlcLQcXUlt0q?{{9*IX9y5uZ3#z?X+NqV(doIbT~d>%g4wW+j)gpBpLy^Z)d-Xl z5DKZYlGi;C#7KQIJmQfYH=HFZ-LWHr=&)-L!w@>s^=&UE_51x-H@h)-n+mz*4Bxe$rh1;7OZ+MLk2*_D}5sK6MI$-gPyH%eO$;K9zZmd9!F|5k{=h=qy(JR>uN&g}RES7hLx+ds zLsX9legij+?OCCBa5N1GiIL0$0~gG9JVuJ*jCbD_EbmWzO(V6krwdxSR5rEDfwJS? z3eA>4^}n`&UT8`jKtj?wK)BWPD40LI!JWdr<+Ulx929V#eC^V{8ZWZ}T&=oGm`l3P z86 z{K-f{^p!$Ce{TH|ajC218iQ0gWp1_xnf`|lJ%oB#TktrwHwzua-ID!FOTInGuER7#O+|EOnL+^XM8GkjE zKZHb-+v!rIWHp3(D8>Nqq2$x)Hgtf;I{dRLcUp8z*+UV3nM4x3LE(Io09IW=T!hkt z>;6b?-Rog=R)i1Jy(J{IQ7aFqwLGyOg*#qi`^*EtH{;v-_++F_XC@sopg{~1 zN&q`;ecWS8X}TDP)4zXXFFvPzEQ*flX?2MqJWj3gPH4IU?mU`AxIRQu=r5b_>1oJ65DT|)eLrNoK^eHkNf1g+O zce>O+*?VS1jV3~%hbE~;G}W>K^Nc-tk=#N`sgD^#I9O~X|nCL zEU$5={oc277x>I8)oc%$07SWp^-eF(*T#_#ehx&!uu&qozl9Wg{T!`bbhh-|Rp7o=F;jcFxdv12s>Ags4B^_WHPe z#eU@CEC;+}uE61KmIAp!^h>Bz2_G@owxgXJ(Tx(hOalVI@&Y~duAVIxkcA4}Z(9f7pJj$|U#r>F6pOmI*> zFt*Z|(Xm`xgN9T|6K>j9Nn`;Y8@SKzc@IczA*P323(EL8u&KR^+ivfCYQyIFcv{;{ z_>0-$xyDDlsC-;2v=Re4Yq4w5Z4Wx?Rg$HyGY!96Jci;b;UcFHefSNmsV_V&G~0TT z*@4t*Dfz~LbcTaad8Ad)5?K{Mv|f0Oe7^6;WGn`{Xz0e>=?s7aBZHRDhbFs-wf-O! zN9EO~Mj>ei#zw;mqZ5t~GnwS%O|5Z=^Vs~lsi~;w&>k~XGu;373?xzh*j!nxtHe92 z-|PzHS#Bcg!r1WCymZ6`rU{ZQ12cCb9IB^NE%TI^@gi)JoDVS<4DO_(DBFA{c^sQ8 z$7#(xXm}?*M4wEIhv&15H#2EoRNwseWdH}t1>fo232L2Qp8r4{YOvHW68;6J)2J=n zL{}lpmi9ZR4hP;8>asqXa|FcWBrQKYbU#cVf0P8g!Rj?CMy{fFeKjZhtOwPGiP2s=w?>Sznk`4ZJ8M_3g0~6j(6{&)#Z`8j?g3=HqsjkSF?2X{ho#g zWT$M7gAW`X87ZtcU_>S3?l+(2E|$%1;o4{Z7Ih-|=s5rXq3o^0s%pP(VOmOBI;267 zmM)c)20;*{OF&XU5b5sj5)h=6Qt3`fDM3O?Km_ToGq=z4zQ6Z7*ZHpVeg6S#@6BFo z-D}=+jycAdPW539SyHd@uK+j}|~oLAG2J)tf_LGP0Z@_DJUZCOCFW9b0&t$n7{A1llayIJN|Wrh2>TNzS$g9k3@U>TK~uN2NSS3imqlFO7a;Eg*JUlgBmT!{2KX z<0puvBpRt)`s(>}S;cRqs+P?MM;a7-PY!jGhb)skKVGbTy`O%ud-h!7_YLbkPs0s! zangzQpIn$_14U{-+Uso?Zg@$EWS5Zf*E5@toHDybryrv~o%I-6 z%9t{F5I%+D;_^GM^Xs_x*h<%19Es`756xGl@M@PggXhb6TQ=P< zmPy?-cd9*`I4o%Bc9jGC6N~?gm~<8$jzBx+J`*_?aw z&B-^-SUQ*$(((pE_J=miRUemBis$B}DCu;+fZ!kJ00e392I}jhSvSCXKn#gMAzvMF zdOx8X=*J2gYLNuXKxi)bN{$7JY5TXw~ClY6dMcGKd80d ztbjd2L4N*nJsU#I0DTwsm@T4aBp(1RzKM@>LJoW? zzyXHPKMniw-Ht7(fLI=4JHIcaFbeWakkInT&M;An1Npv-o(kS%ubs(XC~Y-#Y+ zpZaJ+ww03U-YP}1KGaXyZ~-VISb+?J#^Jrn(Uez4i6J=pW|vUeuM!AU%tdRd0_Wq# z4Re|(jymrv2}j%?UG0z+hkOL!X_9ShJk<~|>YQRja;c?hz{Fctj#;TrOwil1q;VOr z_}V-e)FOCHSl`>d#x@No+n%#;I$3dikKnKYr;ASH#TrcL9*Y>DIPzoeDxDK@>Z;^^@m~a^Ft*mnmh?XFs`iZQXbXFeK~)*}={RH0+HB+}%F< zj?gmDuN%g_-*GdTG{O|Y-hA|ef$atGaREw)jsZ;9wuaIaFsD{b2AZirNr>>CuU`X1 zeoMQw#%@-`?M?c zVN*NKV)4JYf&HuWkavlL1yhuQB?w8od@Qup7~ST9E~_H$pQWX}pUH-&XJ=K{C)JH5 zsH;PYx&w7S>ul*KvuYQ2lI1w&1M>*jLPWRA@BHaLf156J%gm>Q*x2C!aH^%%h#sIq zM@I7yCUF_6^iyyfA(>hh#T@$&EX4T{qPFuI=)8hfMMmEZ3PWg2KYCX}tOw5p%$R%D z7=IVBZ80z~gmFoY{(y$a{5q%t07z*Y2k5!s`LuwNT!)d%SLlBMDbr&)v==1ldLwJ z5&gbm<$Y41^ooJRuzt34c~Gf3LYnTc0$SbaXxn}K#O{E3fe#b z6@va~bnzWIopixcpaGrla$%u=g_*#S@O!MN3jFkU1(6f1z>*%OmnxYs)=SIC@JvIW;C^ZK zpjmz$v?AkMV3xs~c3A90Az0WLa+ug~4KSnu8j;yxG5It7`2*~k(0E(EW6k&n>_2dH z==})%kgeojL(WXX67!WiDv3jvh}!J!Wd`XF8Khw!4&>w}VugY0D)QljL}@&afDI!g z9nrOc+X0JoC{t8!5qVc>L&ANR>F1Pv1=5?OZRItlWAMz6>L(UzE9WHu`Bh5hDR5Ej z?d?_HrvXfbV6T)}O(;h6C9rGyV08tC3?cELjY*m4z<9O04z;S17YWW&oJ~mAMMgEx zhi81JYaAKH1J!i-hVPP-<(mriICF60K}31YY1kA|V!0CP{Az#A_;ovx&C(n?4npo zD_Y`mP|hjwi{(Sf@bU}VHLXyh$wTPtDVd!Po_D^ep`OF^btHM*jVHuJg-gnY^wEge zBH(~1?IwUen&sRDv_Myq#v?RgUZeP($Yn_T_!{rL>GysPeGN?jagO>^2gVn>%N_LNlCWqSC_|=46ZI`RnLrR z0+Th|C_cj>V&fl`5|%1rW}K6h3@q+ z#v6aml;s|E6m*rSSHz}{^i?V=B)^w2H^5m9b;#_x|Lw)M95)@sWW%s27?Z-$uCJgk zN3v>0bjWS9nhm7#1;^Ba` zQ1foc#4wx2TBN*#J@wXp5XBqcu^zFZYr`4jBqY)=d=BMR&~QxHe0`oTvX9cDEjXcB z<>DeK4?lxqBQ^S>L*Mx*ZBqJOARi`=+FelvtPLjfHo6Yo?c9)GC&I|pZ^`!GIj;`t z1c}KNC@3j;&eWzvd8(dvmy`AZn(W{55NWpD81?)*c6o3$+O?R9d9k3l8l&l8GoTL)qAIo)8uAhnQw=LtKIEtQYO_1+)~#(8{){3k1z ziQKxB(C!U{>#wZfi4NTz73sn|{x&-nKAH|sGVpN@d)=^^ca{OLTPTsX zSMvQ0Xpi0s-kZ>ie!@juR7z6PE-53CMZR$BMXbaJ8T#lxEaeU^+Dj3P^lNKh^ZHS+ z3HEFF$xNP_xjKnVU|f>pb)7m!hmeR%RZW_r5bkF;HaZj&PbWg%v?#RJ z$SoJU&&gJ$!^j_Pz{@r&b6`E882h!C1|pzlp^{mc$4VmQfx)F(Vhse_3uvasDcaiF zws7Grs}pe;Q7B6vvjdTneq*lQ4WF86JhnaM3Yu5a#)Fj@YtRP+@POE%7R?dA&(y_GFQUJ7E9{MNsJt;K!jwoUQH}X%T4F)jB>wt`AuS;5#P+&j z#__91cA+_m)e=O`zbR$PS!bIi*XEJ1>}0`upohxei_$lYt5WFI z6-Ag|{u8Kx-M`SRO@sy|0c;?Y|+}-trXXaOZlL#q?ADNz<86$ zU4febX#we`1q!hOf--Pcxy?+FA@MrxdRs7h&h^`CM`%76W1!?yVM6E1d0^Fr zGQ2_okN0Q~Joo+Sg3&da9k>^^$Y}=snWD+n%HCK-FqEFVBox|ZxzUqJ5Iw>v;uuP0 zF&KVBZc-isS2mG*)0bsL|c7v9)Xd^6I;N6ffRb9SeGPO1} zHT@%{hYHdW!IVCQp_}(?Z)_Q`JRu#gO(WA1_3k4(ku!eewk4#8F zWC%JM=6#H+*%_;V)s)5@j9~dY0Wy2fn&HW6K~|%AW_z$c-XX2Y%fZnv_C_(V-?|O$ z8Gan&cPCE5`0W9ez1rJ*k!U10`TfmpV+cVbsPYnJKF77~wo4jiPizMlB?d;U8@mFU zyH_-^n7t3nh_U7UI}@f_QaViPx_@Nj-{-7Df09tA_m3LK8*1X%?-ls31+(ukJ~UzI zM|UvNA?2RjgI<~s?_KUv=m^`IN^wMAV4dP(`v+e1x-Z`vM4>B+DQ|+|Wg6&Zxy)m| z^(S~}rOMB5vS_8^zb4i6=?XXEU8{)Yhv8D6N=e}a+bE2Rq(`r2A3Gzlv;XgVX)NOz&Ca0eH1ef&3nA+dq~^J zuH}E&{r#AE0#7oKWb=^T_e3D&rAi-*;e~cOIHgccSL(OwKkD1AXr&GEdC)bUdUrBj z=UQM(EJ=aN1F8tJkLvi$1B@GEhDykkLvL`p#vS=^!`V3m-9i_1ZnQBL`+wpPD8C`l z(L)};Fdt(qy%Bv|1L-kq(0t=Y$H`OSI@A;DZy%`Ploo}tflG}WOjg6X8$M(e0V|ob zsY9bn*-Gb~?%(U(xV&Tr8T4YCg#!AF*<7=Rh%?QFCR>g|WT;K3Y__A2i(;gt$-CP! z2KQT~S2W*{2>jrCV9L#SKpY|;G8wNWzKT$Pj(m`poJafE(;q{39LP%M@N=fcU~lI=A<|9d-H{_gpJ zWGLPqL=K(hHt%lvviMZRwT^?=0Qi3mNRnqCJ&iV;JCdT}141())na2q6^r{E8Nvzk ztw{lx>6Mq4Z*On={lL8K7nzut^b6vFGLga7!u%C9We!T_0iuLhc?R#PYY!i&oK6Ak1SBe{wEEPJ*gNOf$Z#u0_y=N+l|c3zA= zoFd&ibt1X`-J7U#;0a6dJ6HVN2t403Q+aq|{dzaF})u-vV4uELFg;Kn`Vx9nOXewfQgvEt6VcxfVxD0>$iWWRqp3`z`n9ZH;eX0`YI zeso!@RlGI+t}MG9&ET3BAM2m(L;3li& zF)>Pk6KDUU*W;95FnMX7x#Xf-b#-Izxd%an>JMWJ%=_PL<#D3>-F_)IED$$L?)<5F zHsmq(Dfm^Qtc3Wc`=+GpleI4&i|)~XFc6QAV?3h9O*ywMKEtDQDbud{Js@&&ooeVa z^5{zEQ`aHSU8bP|!NY#^hE%n!h>Yd&nnt-<($Ci-UXq?HPpMyve%sx-YVM2kq#Kk{`9StHy^-HG4^z&Q$Cd8V^ z_Frpj|B;SMjP3n(KS?yzsAeVw$v|1;{PnusWY&q$-Dnn`+qcDUv3O3~-HyyUc=EL5 z>!fF2=P}btY_%N9ol9{h{k{EnqvyNHT2ANnQ?>SS(T3WxvaM?@9vx}BZCR($=}8`v z%V+)lGY#$NN*N2ztKQ5M^21r#uxaX8!;50lP9+c8Pci*tV{7f*6^*N-gB%{z4qQXBS+OZ3 zwKjAI#)BD;-#NRDE$efB)){>#_&%W|k&#?U#B3@}U-RV0VRgv*{u7t_jAtB%XMcJW zQrX!@XtAn~-JaTFc|GVhxL7s!OPAixO;p|6jxYNmH8L>$`PCHs;}yxWpu6d^88xL$9(n7w|=e?hXwe7E7|K`o7xSOzpwH)XwMSOi0#QouwNeiiAr_) zxG1UH^y+T?@QoiocaEI$CBv>8Y{i(tJ!Wk^7#Pv9RsY|p;ZM4MZ1rbO_vHT$8~-Y@g) zH5bO{5=k$**Byzq7wxv}kNezOLh`93KIAw=M&dmEI5BZ=_Lg9cpfi1mn?du`+X7iS zmqzQo+pQ%(tgjeHW2C~p8C4l_&mnlcw|HthGq<~I|M)8K8s8;Ei89f}kL#|n zoYxG;^;mN8sXsQV&a4Ztwa)Twoef9NRaI5y;?vegWjzsJ7eU(W(-ilhCUJ7x7%TZ{ zb9IrbpBVI}Wa8~=upqibs2544?ZC%0<+6PYrL08C)0B^ra6 z^y{)R6mPe`T4o;ihCT@|GZn}VnLkGFd#AxA;UDmg$44-8%293i(fUYRmt5Xr0$gWpL!n?t5gY z#r@0}|6)T9o{J=R7P1B_^hBj56?C%QPA-y}5~rlDJr;EUcgOS7_QLQl%JT@FEj^_z|-qpVQf# z&8p7h|w{M%7jlmxYYH&Wm@ohUC8S?$nC@3)$Mv*xdzt5iIH zTXHhH>}9Q`pz3{iq@zW3vQN3YfnRR$EOT3EevT#>>(4kPx_AAq!lM_+^7ao~#Al*H zNve%>pS+YdAPL_uT+qwvor%P2%og_=d~Cm8$Wr^Y;&bv-(T7~6{B$x|8*a%(US+O) z@0-pK(vzyM;ivY~eoU9V{bctr+jgb2W|6q<^QJ}dkM$ABxdBG+#Kma+A&Yz=B(IVq0?cWp`fzHwCd0cK3td4r{uuDqq{@+G9M* z8*V$++OigN8A$AIDCgZuSh{}_$S$LU3=5?zcJARMT~kVl-Ee00ewsg2a60kS>oCuB zNWVXic&VGg`!LFH&+9?g%{-+o&rcJx4UtFJ2K3T~vO>+w-v){2*wylRdRkAmKc-f+ zV0pG*MMuo% z)(3X5c06@eypGq0(;V@Nnna2FK}1hI5v05&6q#AE?;fPl|Az}i!g5^ztw}s z$Cr_G+7Zjk(p%~7C*DIRJNyy~hmBNQu4OILilC=8d!#f|@-kZ^i`eSh=&B9Bwe#;! zhT6zpBxz(HKJ{;1Js`?*qdkh-_DkMt_>p&>e&{J~oZu>x<^7oGEe}31#f{mbZ}aZq zG|a<9R`dY`_c7^xlcaF|{)APH_5>60lY9>IFdTrNQ11AW{{4x_3ac9N;(!<_#sACE zs|TrK&aJ0+M2X|$62kPC-u?c?UPU^4&|LUvnbH`)Jy}R(>-($i#PjA4R~6Kf?yiHm zHw@-JBVOvI))SZbDa?M zWl^>pwTIDxZ#~*4?rwg!E?!+}{vJ84Wj|c35`PG1cTtiqKCU~S7^Z*LU93^Ba9BL2 zRO0czpwXZTk7$h!?X=DO2rdy%w8;UN+vUgK0!F+2=)On(h&Oz;-Ymcq$H;=GxTxo& zH|;ljJX&GLOOx(;XyQ)o@;ya*K^^JNu8``EX~Z$+cp<1 z>0Oi_^Ij33#prY~cRyNf*|+5%Hyo)+*_UYEDBmgURy@D0IE?eyzHI$!F>=?k@N`cz z4Yjvc+XYsoh~(^NKR-WEPdtW>{j<@6&hF-U%?!XSK?be|17$=-MVnNzCB(r_0u{hY z5SHr#3jjFv+9lcuZ59AzT>5IDvJ%AuloB5YrwYRy)YJ&B1uzz$t9n2S6$Fwl|Hx(B zb-B2YPfx-4?<3f&fWXD(Yi=i~JwSqD>t@6ZAioVjtv$lEn=m2*JOp7>28gW?O3AQE zl@(~}IzexqY`qhRB@~&^g2@zfANlS}Zv%oNIAA|hW4zg$14eF5;7|aRl9OL=CB3JBhl`Z)QCQq(*$(ErzP5VnmA$j4Rmj9C0`wK^fGM%`_D{` zoQuJ*8FKNCxOrKG5 z65wS}IshF1ax&6?JUx67G&vKw3!9ryq2(jH3G3)n$tegU5CTbPZ>;{-Kt#_gDtQG3 zUC=Tni~ETJmqaf5^3wZ(!_jvzoU$8O40-DK6=qsz*o<^`zJ2j)gri%3K?f71`+{9a zD224#82n6ZW~;aN_st?*9UP2;5&n+M%&D8yeWYH?fBx9ny_%#ikpe>lkO~Y9ROU=9 z_Mj0(WK>W(^%-Z%~gfdm-(qs|F{VFIuxQf$Cq5GTd$&Hca_;XxC-&$#^c7bipB2$rMU zFiNU(!B1cofdIDeu@WUcOQdIC0uCmo(97+4SRl2Lj=%4BrgVAsmAwZQsnP~~%_U=K zwl#Si+;W)(<_f|fk6zeGCWu0sk#j~ufrTvjgmA#?e5d=-J!k;)YG%H=Be4S_KDXZ! zkvn%LQIt(~ZV^T4^k++E%LK{zPz>5egoe_`lk_XOa9#ta#TDTq3uT8FFU(M?Ot?w) z$u`>WOV2s+50dhYe>IQl#2I)b`C^Tr;68FMMPwx(^&>SU7i*dK67OSSofQS7c7Z@P zYJf#TBCGb{lO852-8u@wiR%{Zn&RBtF~vb>yyK}0Fgu7L5*MwV2jtuFjdSTK|PiCXr9iej~`iZ~#KC1@=oXiKyl zT&0Jga5J4UZsKA6YGZvv+kPzdyZecCO!F1Z{3B(xR6z50_?vys2YS2Sl;}rgt-qiBAp=Ya;bE~1q#dDZ%k#-%v=t>Baa%UB%otf%LnpdW#J;UF?(uL>FJCx@m z==)@fMiP7_u~lFY()9CF#|7>)n0Gz}R#))tv4)n-lAQER&VY0#oZ&hdzX38 zB$~>5*AafF(jAn^;K6ycIh|ZPAmtFsa|%zStfp0V1=}P_8}DdDKR5l?nyo5tWSPs8B;o$FevZ3aqekM!K0Ua- zma|6s@?_3Lin2LHMJiwK*L8LQ`lGtX@85KsCIh8uTOoY*r=0>p2 z?mVsIPc=qn?zOdCOZhj<^24P&Go_uu5k`N-`-NkMDf`SuULN!4 z@r)g8VH@pb?{l!L(d}BrBg}Zal~hXUG9_Nu0c(wEYAnF&1zecgibz`VI@oDv$Oi0Y z*kK2Z!1*q5z4Hcl8*ICLjkT|K()_MD@@vxg$_iPf>ZL)eqdjaLxIqzpVNMlrLuXmB z`lXb@aWc|oFsq;YDmx))hRjBvVnnQkOUbBmCadBiq+s>a-1+o88wYvQbiI=1vxS}8 zuaQ9KSk+J54%Jl;WX|_~h#~M!`>M~6F&I-*84Mo?55Y38-+{mN;1^4iibT z;Y5z#3kX2_QwFq(xA&&g(TP*g7$!*0p^a#PT*&Us$Zg**>M)PwZt?VfYi34yG4y=?q$u2-G}P5l zUjcMmXa}+M&NN0yfw?e-?s8J^_b(##=6$3I?;%q|*U3>(lowzcn2CfW?wP%PG_uZF z4BC5?u`gn6p2yqtXWZih(-!0?d-aq~@`I`GC-fwzdS1Lezpb8&rkR<+IMX1@`_5SC zZj=zn^Ks+IEI#d_h#KcZDAWu#Dj}IRvjBs!H4cn^5ul*E-gZHVX;GF>z!E`!Ym6Qk za*$}z@DblI`jjr{UrBWX^V#RSL_~dd%!)~7KaGo*h!byV-O|X%Btu_}l8UC_3vwSM za9D?PigTBflT*svgfmE0n7Zz1uElzi@Dqr)QaMHaMfAGUJoa6g2iTDU z%}(qssyV&%feg+O{e1C``WkWqrLX(*%rV-;%2wCs|MZF{#AW5_cn!$3W+pIHL7Tj( zN%H*h+>91HY<-ytTA8;)Y@LlD3(?I?>6%fU%Kk7Wqa&(Ce875Em7|d3P5(?a>!rsJ z>;^gxU()73EMx=(KC%wF8rL*uG0@Q~JT#c|Y%s5lwA|1En_2J*uE=ugG{FFh;3{dS zUteN`4z+}5is`e~zP`RwMp{xr?Ku#0ewSEnZ*wFerA&IVA1IULs^cM)luqZSO}Y?1pY96F#^iQ3-0q?@=?aMiKCh7bZayK5xLN}Zbw@cQiE12YrnErjA#+sv!`p-B-waD*qb&z%)bi>qiV93X_N)wynd;nvj@ zXJ+nf&`WSFZAQ*%g&GH!N&pv~A7vnfTVzZ|pG0XlBqYS|&hU${z6@K#-XNH&>9FZs zXi0PPhHVEfLA^nv2mKVJXJ$plxw(U8(L9SpQt0G3E7lNeAYPkY(Dpy#P4I>ZgpmgZ zlo)=b4w!?TwySZ#GE&)vv4Y+Gg$`|s*q@~0(e^&=0&(1_QIl5lWeoF|`;}ivetKBa zvGA3MXl6>sQs71e>@gmiVIH%WAZup6P0^Y?y>97$s11oKEki6kkHC$P^0US zrLT$B>KM9SPV>+u>ccSWTiDtMU~&#gedLYA0JWCwvXgopP|B626{!sBj`!l<7Opi( zVQbJe14o@NMAFJ_WuvD!*aaerXnWb_9Uc;$_Yt<`{AmR>9pnHl zdk`$VUapU^hh2+nISsG)>Xtw8^(*135#bDYnfjYZjNn`uB`#wq@U;6WqMI;>-A(IE zlZ|nju9(-MPSaE?NttGnG^f+{BsG+c?}t(?mbnJW+kpJnPk}D!w-)4J-vHUnS5Zx; z9KxC02v4=t7@dSDy_-}6k1VChQI^ z3HnW01%d(tcPRarU$8`=G>@sR@I0kJiBrmzZv(!wJV&U>N8{pnJu%!!i@G#bDzma2 zUSkf2aM(zu-ZPP$&d)F-VG#^C%{odL)i0rIstBT=@rY8pM4n0xmp~XHX{S`*HH1Q~U=K1py(XK$>yk*m=UPn6m zRa}PWPV^}yE0D5*{eV#j#Y0!4{GJ90N>!9QfAPG4e^ETle-S%^MP}?5pql^#dI&aK zo14!Y$+jVP1XUFn;|q9i@0_o$dS_pJx`5&(62c02ZGd;z%2sxN;S-R>nvK0v=dWAt1}Q4vaSwleS_B0aw1Q$iq!3|xC4 z0D*g;3=qeo@Ous-vawJkws!!;OL$g3G+&B{*kmL& z43|J2oh{_J2qb1O0)m*wQ0R#spPgMDeDJ>5%f9Mw(Xnp?|KXLVSJP4(>8X9C1bt~RTFMpvZvjVeBUJ|R5$g2M?14G+s_$;KUUY94U1rjZvU}mrk z@baNBA*H0W4hsciVi*bW2XaNIRP828Q%|T-!+fxaSth5Z#0~Y-@?r?Xh$yf2M^#^Y zdcr=5RtGRo##;z;E6!$C!Af`S=Q8#Ir}=l;#mcqpu&8DMHUmX5tS*qyw3;61GVU;#WOIG}3H$--=JYG1@#$$Q`VNe2vM~h3cML8UsAR0NMksv|R_zta zQM}odFF;BU$&u-HE$c9biv8_j(iaiCxj=45EK+v|6M=@DEGf^?k&*r^C^FARvo8bY zHDs&pf6RTCfTj7(;ME=Q$p2){l!B{Wq=BwnIE}DfyjyU5v-!m(1l5n$4~t^;WlM&P=^WRa>)u~%KF5ZBbfh8OM@IjZyt!5XYHh+7B` z$`wW$*bpW!xtfmKT3hLZy+Nh|3$OR75 z%@m!4-AoN&9;;Zsc(}OKU_cH0Wyh=4k1havVOAgX&>SJGRsp4(g?b2rC{=I46`);Q zTEz(`A=1AmVS6V5FE%KZ;qHJ>udwL(roA`c);>3ivhe-1mkX<_-#$Ip+artsR^>lX zWJm+j_n>3&Sov!q;uY{~uIY6>loy~&Zo?S3mbw%SfuQ3jXntOTeacdHh#{C7f%~!5 ztCvbjN?-zcwO=Ty1&Ite>{jm$if-}S%?RDP^#yYuKmy@%$atY1WeEbu7qESH0bK5b zK@L!*iXRpE&FWnpB+%bV->SN*k7SSu0}IJR5OKf+uYF>w0=!AsZP2fsrY9swTgu-y z_uXXO*7S>Mf>n+%?y?YA<1J9hh9ya0&NvKHSm9n4foJdlYz-h~{qMyX5*e9!*M_wC zaC_lFAFHOO=E)&cj>mH;Jf^VXL2wjG07GvmsOG>+1|AZy9G5idz)UzrAb81~cMO{v z>g(&rIqq!CHP!>x2y$$m+z@AmZkS1{S78!JXcdsIx$*sD2|UKtpMWXYz;lF_b~+Es z34E$HU|#fTbn$K*z=R?q-&#suYU)GkXrTQj>V~}{}^#i&B<2sV78Zlr1Ey2dJlrf+6RNi?`3*|O$@d*yWkC4aj)Y8+%$0c zQUrGs2-$9bs3(HiDjOjs1^BT_!ZgwXMpKu|;ZAFx3@hSwE5f*uU~vdNSr3587lcO> zmPd^AKBQbMWv9Oiey4e8DZZlPErL?L~RByJBQPDdMT?xP49V0p*U!^lJV) zN|WbUR9+zuOd2f4gM#|8~Tn^YhOq|L4HqjmpYO z5NLtrlm}P@fC<&v@i8DBsdR4(3&G3J9oX@8br2USrs47RHm(Bw0^lA(VB+>10}V~g z>pX?HG2$4iA5e$=Sz9Bgq)baqJyARirhY}F2jjMp4f_Fu6a^bpE_S1v^oU&<77-Ln zQ*bLmnpp1j4*;#Vk@c|l7K1_#wovj+32_=i@RhwEzO%I@chdq zj0T$DhERdfZoOSe%7}^a3D{u}5lMU1Fh$229vS%>tl0GRGaPxnNa*N>YSobOIzC4{ zs$li4V4aRUf^=caA0Jp>RjgcAquQPWgVnZpt9me69rS9jBL|qYe2*-E9!EXEJf5e& z3JpDXIz_h{_P`Mvd}nY6n8{t+m7h^eRhrxsSTbnfM$UU(ZE7%j&wz?gz^ltQz}RHv zZrCT|b_`Wj)qaf73~pmTaAHZk{{4l61DoEb4QM++q<3BC88?)%*B!UO31GG$8T1J@0ggBNB`n|^Mr!?z+ZvV@L6ks z+zzO@+JRX9b!w)_?C-MKqk|pE14g+)UJx=wfBWSK7(mvxf-d5i%lkwTHX=gJKLa!Q zkgR=`u&a|iGqMNusQ20Q4=8~HU2i^s+P22&C$^ha|3@H&gK;Z4XfFMFo?k-#TtlW; zV+q!3#Qs0Po7T&xbe;zSs_bvV9XQB{)`NT+{s)SGs=M7Dp`)FhfyXxE#do1$0HzP1 z=IaJuTmkSul-`DmS7`pd)pw2T!_{`M-x0da@#gkx8ueUO&|-Tx1h6B3x$nx8;ymgoEak zR0sm$qAkn~=w*GSB(I_z2Z<4kZV!iClJCbmS)nb+kWx|eGdE!f1w@cTFmzkv?afbD z3If9*aOU5I#Qb*W0))rZj4tz3uLMtNt23X}_SD>r%m{UC4XfDPmKG_DI#9Az_db?Q z;?zk-x=W=x`0zQ2#QxepIJF41MOWaJBM-`!Az{%z>Z@aF^Vw5XC_38#Rc%Rsu>Asyli$ z3Rjc$m^HOvPP?rwa2&kZev1izNXpYO51~)vv{g`6E;DF^<_TEy+4|I9elzrb4G0yu ziINbTcJ5R=Ez1Q%>HIg1^^ zva(?tM(K4hsi9?%-^0hmY^fHaF3M%^2;Vcom>AXOM|mY>%tQ6N@is<(o^&YDGO{vL z{)egsowVG0)u>u`GzJ-H(zNVw@EWW&*dM54d!VQhhx{Nueq{ z*Z~hI)iqS4y{v*4j2P`!*Vn@&3D;X}N%-DczvX zbS1DuBNP9*@rVU6ydpM+bOnD2(TI4I&JgfThn-WN`2mferbclrT3Q|0#NsJey@(N3 zVg$@kCX+%mD0J$+AUkTgYMgB&PpFMCw9C=1zz3GTsH01wT9tf1p?Z`t$mh+1^f7;t zDfP2Rd2&V%vkN?5qj!Gt=$r9^N||x~&!k@J+dx6W#Qi=BWSEPK{ns#(Lt#pX#$rCv zBn&NJ8R&nbu}Fe@eKa%@O*X0ge0W%`KRxxqo1O!`CQo-~EVb=niDaT%N>AA2+4xdQ z`nO@*cP@(+M=?ESqk5-G&TDd0MxQJ|`o@6oer$h+kYih&(%+cdiSOVq9gR>KhdF-h z#^-JWY4Sewd?#5o+Af4Z3&qFBCoD{1yhv&%?koGfh9X@qx+bmmTjeU)P|nQGf(7oB z6C}Ejl`c37WyEr4z~xqiPvts1aSB0ZP` z1k8i&S%@#hogK8pn6^m9=Bfd-d#0*PmG=cEg@YLly2`P9;!R`#eW~&?kCw{YIA$Xa zSsWET^&2G+7Ellqy6)O%w_!?I*vEDgBPchPv+cocM@DM&?zZBJ3~1NqYMSBFF~pmp zCgz{5;Tfk}W+c#A5o^dBeJbFEWNJ|#ZRM8I7OXh|tK! z(vtri;f;qKc=EJPT?#z%1`gH?_fjNI*BED_wnI{pCZB5vOwx+RSR0UW8DwDjI5M0< zZk93u5xL?v24+QG)*GfAx3!m#9+9ke_Vfz7-$ZJN^sMA-w4i&`?{S1<*Nw(!|Eu*A z)(u7`88EeIk5iB!?K`LMeL;NNoCWd{vW=6?&p(KWjEU(-Kf)#MC%FGp&u>|7AE$2_ zy_K&!BtR^lGwd4W0!I&v8@rJ9)y1L;Lmd&IM+XBc{CB=V)d>xMcqHx`axHyS~~ZB(218Q&w3a&X@}bh^>H| z$|I8d4af2uNUW#>#TE=?mnK8Lod0oaW_-5BIH7nM8sPKGHkLug>+0-B{B;qVbk`S2 z^4mt%^h&W`&^GfJ?(aIh5HNZCh`l=KDNhF`e`+DpI~c@p2A$gIcm^}5DQ=PKuQclvwdMg#=aQ!tNEr3n!Pzty^_lB$i`HU!bpd8I_z60Iq6Ux z8?z~JeMBSnnW}gC`A|BB95b<>&v}i**ZtM2c;?;qwc#{(k*FqTkpkqYs!|c4eeC)l z7Ik2z9AX^v$fYau=P&H$5-O-McRZl#b_94O2?{xFXX}%w~lgfzT@!rs#sE-1Zq&<%V>Ym zBe;#E7XCMky}D?JX{h_Arc$T5vs&$%m&IeL!{?wM>j64RaG+V0+M62&C+VeOe!r@^ z`uZgXc^z=Wf$~Q<(0wu@(BY<=2UGMbG1A|Qi+*3yS&gmOD1G(juK@yY=3R~Y;Qkrr zUGTdarD&Yj!^NVPvQ*6!3EU^X0n#QA`sRw2zg|>as`iAu7HlnY6E{pHynP&bFW)jM6woG0cVM4iY8Y~#2OfJI+`|{Hh zh$rM!=MAXue^4M)cEtX^+{OV4SU`uS53Bv%-FF~#Tge;|3Od`S`2($WG- z@VGk2vKBQU)3vmWlhU;9z&v>}nnq+5pHz>ywUxxiPr%{!6U1V4Jkxx0(j&NNSh?}&RXz55Fyd3Gt`lQ9sYCG4EY9z!c$$$}Oba^wn#VWq4yz}rvvR}$c-Ut_sk6^S@E;QqH z%4&6&dICeWlt=25N9fYz6coKSbqXI8FfSjsu@|*_}t2BI^QiVu~Nw`v7?s ztGY)p=LJ};vqPWtv#WW#&tnJz2m_imnzREONh;(Al#*WOa&)ivVNz!kj3dk^Gd`V? z#CWZs)(9wOR+C!5xgI4<_|wVNqnJ19W2cDdN^Qe-OwP`{Z9NH z8X$IvIUo-SpT0mWWYUvR(Esll>v>$geXcUhwU_GAT!tR^Y;IxT+v78{ubvxEA3b^$ zB{}uK8oSbXD%ZDLGHj9gn3Z{!X(uAeoD7Ky4P;iklS)#e($1W@GG{vI9H%lwA(h!l zGF9lD%wvSMl=)px|6l(P=lst5zJ2iJ(YBwxpZmVAYpr#ywfss_Qqs`GK&*d>d<%xMZduNV}rc z5(5_@DM`t$mQ6*RA|e*4sbhD(Mj3nlVV>?N#ORU7Cv2%x`rhl6PYkp0t((kEf{O@& z^i_%PzjGmJ8Lr<|o&Ch^L+>ualZkHgZos@fYHvctgMtlswu@;fgZpz9CZ-TvEi`~m zg^_*pW$YP_;-Sb#fgmvR6@WoxOMIE^rXdg^|1jHc#ge&@TsQ_U3_~6{bBhR~LhRP{ z!uiXY*$-!-egah`r4sc!{v#NbF6SFxm#@7o)05u$`gB>TQ#g(kFx ze1R)~fKb+Hd;M0)P!k!652_Y%T-rB>6NfN7!K63n1x(WPoFkD{v5{|UvuiP*^eJj1u=N1I|M)~!3a$NDBf$_&K`sIM1_(g@Efcg)^gFiD zK>r{%vjW{L!pCN5G$AY~93jD5nHU%{d4T3HRF?pqVP)M;pyTqBA3rX2ycSs)$I$~J z>v@MkA_eIUV%OYfl_Xgg2wfjuGS56IC@m*1PePS&jBPGBP!#ekG!n z09IsbqF1&wH!nHx8k*9}pl)#k5PeP&uz+QnL$0|1Tf2Fc`wJv^9&M)Q5YPr}zl0d^LHCyKIs$cm zQ+C=hF1TpAtFY$tGY|kV3i2HD{hh9&FrrAiNF3?g@x;#5hl90S8I28XpCm;OQ~+BiRDmDc=b@*20mI99k2Af)&;cWT32ick{d3A zMZyl%hIit{!8v3o9O2`<=IVlC-e&@s`4Hd{26bKNYGyl5av?3pNh9mQh<=S{n?&EENTOQAL{g9%KAhl_x+3TA?HDB4pA%}n|28NM zXhnY++0GT&m`^dym77SEXWf376wJ!;(-(8VLO9ob$J?b^>c|-xkSUZVb~&PAaui=oiQj< zRh6`aM@zU)%!hg}^a^(~%t^{9-PMl_)W!pWz0RtPCP}7tmQo0iTC#Ove+D7BvJmI} z^BpHr?xs#Rege6$3jD|E{@eJ$W=deG1<%IefYG!IF&Z=-{$ja<1^ijphx!%1kv|2^ z_4&dU?c2s4YOd(4p}r1@n=!h`i3-h_(Vu$DGc|}DaCo+4%;A-18Y>PL%c9Nelk-MQ zxqd%TV8?0k;$W};^iFaQ(wnCjo~|hDkpLUG)V<5om}+6!rWR%4Gi!yW^CQp z$Xu+)odOfd>s^i>)Kxz6n22oLexj4~*HC5ZQ#{3{gqG#QWtzoYg5pyY+XQJ8Fs?kS zE=v+;U^fxV#bsxv`Pw($ZVm>AnT4|5cI8>57;9qz8;CC~Hp$#oNk)5wgjCUm#2q~$ zb?tJ#qUyE1_iSWxZi{$^_p8)c9JGFq=$m4hV-dx13USXp+u0-{B(l;Ib^=$D;CINF zn*d_AWA74?;!9$lDs?Ai8?!{p85lG$M@Fgd858LV_i?^^H*itzCN24FkXhJfK#PDIQp~L$PpVo#c+Q>MRs)(C2=F*=RG~$##b;G22ZF+4*&t@jZ zmnX?4{(w7!KyJ9`X~>xd@*<@LKDGbCB9h~$aI8|(UrkHkjIXGqxTF{B?O89|6yC0o z#H0=8XXEy)g5@!}v1?Oy-S-!G$jYXY-R zfh?W~4FDa+z{CV zV{0R?ExTAvjAk7m=a>}yAUuH#lx*``p{|%wNf10=1@FhIhznWMtE;OC+=BgWrxLbb4i32adW*f z1(VgtA3}se`eHA&T$jV?hS;AC*UtFvx2W$*;^ij=sWQvNlj}(PjS|bY_oa<#f7o-7 zxmm}?DE?`075Hku^-5Z>KPyZKcq}oZWqNIY+K2GwbGUm+19=jCEyNt52PBdW2zLT_c<%8;N+ zfpJ_i^TS92eDTHJbZV8H4&UmooE)LhA!x6EMJwF*F-PJXJGBk9G1itUqu6c@O`FBV zasD|UjM*;k`bVs|yC9?0OOy{DL7eOX9!mg5D0e>*6Ize~$+iC3XB~oX92*~lmoWz? z=SNks+kNOOwmwWbv+rf5wE}em=f&l3g>GdKmS^p+8GE1r9miG4g znH;U)A^48gV?tU-hz8_vTy!daQ=IXzQ?3~(w?U&AY6<)$ z2QsJK%~};rV$CT+!^3$LvRb%tiojT3-*XnQ||f?&Ld+7mxpRmU?P)OtDOs z`I{Flu0q>nl6;cl)y8C%>x8mi=NS3Qts^Ct4E|y6)M83?O}zI~G8Nv5+}E>is6|sA zst>PMB_#C^e5fa4PFyqfgz0GlbIpxi=>lw3btC8~%)$d{TO2oje^3swYji+*BFwP1 zOSzTg^T(ou{?7}cf`TEc8Bee_pGy)$M&(2TRiXN+flZ-o`j_DV+SO$zSnGI(jU1sd zt7RL;c;*qbo3#xMg9*zCQ>ZNKfA^wpN!#hqyRBBolit6$Mc)Q&W8Kf_|GNSGp;l#L zWaJVGulFtWKEcZMI@{hn)O}WRQ5z!!krVS%?@=PI+nEjkUYR4Mr>|y;oPC*| z|C7Zg0kvAH$(2Hnf;*Ac>`tfs^z@9Ty-k9HJ@=5bB%hLB>OPih64NKg7I*L!{n0(H z%``A9E7DJdeb39;?(_cK)IS##&?woRyPJIHs@&~;uGJ4$h2yhB*T1WM(w#J94prZ$ z9OQA>Iebdo0@^j7G0mO_{drL`X@nc=}YtCV= zpCv|^oy}Bgcl9Qv8n!%1*^TMj7#b2AjId z_>d-Jt0u1n5^iqGrP@DAo-wn)J+_%&2bI}T! zN4`X#Dv2uk%X3?zgnMz`#n{+QQE zkdsr0@U2U7sOF*cgzVU5sn92PP6LLttT`ZOd{+~^+X{A z#sBe7S5IkcU0QT$BGbB)H< zzgn@BKEDOZ&L8q(tbF8(e*$HTMD?{{PCmX60U}Syqqj1`Y;kcBsgkYW0w03m`r><| zwY3$kL+nnPKmSpY?}Z_Ld`Hp0ONcNo`=8m`|3x+M|L>vsp}+8*>{7)+2^&jAaG-!u zQ2E+-|5+9AzPtM(lzS-ekCal9?%a8Yjv;VQ3cTFm=!a5uc_igDCces?UeMvT;YW1n zO#{@oxPK`rsb6w)OF{R+4i`WN8t-SBa#0&eq%L$HsQAZwKrI6sHF>El_Zg18U`NY5 z+Xm35NVdLv_Z1(N)m`03z9i)B<$;$ZC=r8smJ~N(uNfd3F0L|G>4aM}$m96bLuxyy z7Rt*jE8_+CO^Ch9Gxi;(J={^ufo13b$E<8$l!JE#VR*XAaeX_xE{MO-BHBYoL_Pyu z9yCtF@6lVW6)0vU;>*QcpC97Ap_9p_aL_%YVz4uoQN6saPaQKua+JsQ4G20z7mTV=6Qo%>ovcN2OV;&l& zNKP3ky^>wJ7k6*D_=>H>K)#Xrs745O@o3CGfwO}qO+bTUhw1<$6YjXdtf)=3u{>)}`wre=Se?RaWVlW;hj;2jsEgSY*UeIq9?IDQan-A54ZbWAp zdcH*?4lKSCPkT!Qi}PJisl0q}?%_XBw{qpyduZqOAWi<(_pmV{g$Yg4J5hnkS>LEK3|lSPJ<>r|b@+ zQ_p=ZQ5o&5%6j*`?9V66s}VU$B$<_-cr&r`wRgJnUi?)2(dGx{Norl2gUz0ei5v@U zZ`)lPyS0NS)+U}qX`|oD;`yFTzrK@@9BQ5ie1k!ag^LZ1JQQ?1mb2j6n41h-~58uroD(xA=8B`ap*!of%i-1c4j4 zvx!YYoak-L#KgpyEKe7m7notO{GqKa)-24}RPYOXQ)XVH>Qjx~`*mVI0KABm4H{zb z%F7phKWXsw;w%I&{S761MC!!ogA2I1&N*RIBwMn$rr!jgpFFO!*%pNEF?o8M?|n}+)Eu9 z_rurSxbCK%Ko{JYor%Ouc&@#_Z!MG<@+Se#?_N$<0We>T_PKml+|$ZRtY*_#6%;&Y zBP^WnQcFDBKmb0lo+1jDNg)kN2J3SkIVT)x>Su>c%SR?AEUN)&;F|lOZ7`ArT*6y3uDn~P}Uw(>~PkXQeAOa!*hvt#G zo^=mdr=kxc>0DlTpHs~BAE;#(ca7BC#=y>`Cfs?5^&?cw&)E{J5KK?6g0S22=joB6 z_e537xrKyYvat@}dnoiaW7}bBI>h1SF0o$SJHfb_qbDaB?$v5GE%SVI#L-6YP)Qea zbf^el90<A1_p_-hV&_H)wz{af2ng4I8Xg%~=Maoub?7|83@Y>4mY@^=BIF&on@m zlX$*>`r4-Q_-U5F+p^I`mcgK zJV1%JXmGJUt@+@2v%cc|$NKJz*W_B#cM82qOWWjYy7TJOde=Nwq^jP;`V-gj{yyk2m#06p}s7!%kpg@)_DC z*WWoD&DbyQ6s3Pkz}_ME-O#3$U$!j^U2*Hw(a~w0#iB+993Lt6IFe`g+uzBL7}l+0 zkl16m+cw|2hJPM43o}=y9wy##w2U)^6mJk%!Z%TP%=lNo*#Xv1&{N66H(j_3PiT2$ z=MzfTU2bcw#NG|9(gwy(krX(1Ym=emo~`any33;uyt17^;XBlSZ08uyDwW6CvWqL7 zN=<6Wud1qomL5>ea|u<+0@5S>f-Cb%!X;W@N#`FRV;O?{Hbu7DiG6Pf+d?NKJ>0*w zi7vb*v&U|mviHG~vp6{Omob6MF@{Z3SkwC|?~DxHZ_v8vhedD;#VulnEAwD&Z-Kpf zAl0y4^l3g4?kt-QE%X0%q{NeRwnI_giU~Te!GQr!futHWuNYAj7Lh)PO^#0Fd<4`V zV`Q~#%%#*XOOqZmm?g>B)+191&wS@3yKP4YJ8O5vkoRuUlJhl<`6jL(Ej74DeyP}&QO*A%Z`!-VyeZykrRc#l4))1eQtyrL$>~LG{iGGl;Hz)_gbMS7}H;npsUCaN5&0-77)9IVSGNDfVv zSj;(z&5VZ`hsPj~tnv*AHQi*bNzql=OOYZP&G(XY(Ar9su_`pV9#;3MpwL&`Ui8$ols|*W*DW5Dk!j+R9=|gT+N3RijNJ; zvf|=&kLqT+M(GxT(y61ldudwoIQw$(vo^# zjdq4CDW$nPHx1oZ@px#G{9Cz#b(*)UT20h1OaD^t@cqNP@l7Bm56};Sr~%u*syu#Z znd~=;T_+bD6M{)xzq+jzid;voc{3!0D{+n%M2Y`{!unsht$%3+{w_KG*B;>qo%WCB z;m3s^6FxuQ%a2$7@959pFZ@tB|Km^o>xCbj-{1e_|H=9NcrQO*`M)37-!J^*z5Mu- o|9au?@8!o`{`e|F9c$K@5`E9oIo5FNt;4@PMwW*6^__zL2Y;-%o&W#< literal 0 HcmV?d00001 diff --git a/docs/implementation.md b/docs/implementation.md index 937878d2655ca..0e150838090e5 100644 --- a/docs/implementation.md +++ b/docs/implementation.md @@ -22,24 +22,26 @@ Hoodie upsert/insert is merely a Spark DAG, that can be broken into two big piec ## Index Hoodie currently provides two choices for indexes : `BloomIndex` and `HBaseIndex` to map a record key into the file id to which it belongs to. This enables -us to speed up upserts significantly, without scanning over every record in the dataset. +us to speed up upserts significantly, without scanning over every record in the dataset. Hoodie Indices can be classified based on +their ability to lookup records across partition. A `global` index does not need partition information for finding the file-id for a record key +but a `non-global` does. -#### HBase Index +#### HBase Index (global) Here, we just use HBase in a straightforward way to store the mapping above. The challenge with using HBase (or any external key-value store for that matter) is performing rollback of a commit and handling partial index updates. Since the HBase table is indexed by record key and not commit Time, we would have to scan all the entries which will be prohibitively expensive. Insteead, we store the commit time with the value and discard its value if it does not belong to a valid commit. -#### Bloom Index +#### Bloom Index (non-global) This index is built by adding bloom filters with a very high false positive tolerance (e.g: 1/10^9), to the parquet file footers. The advantage of this index over HBase is the obvious removal of a big external dependency, and also nicer handling of rollbacks & partial updates since the index is part of the data file itself. -At runtime, checking the Bloom Index for a given set of record keys effectively ammonts to checking all the bloom filters within a given +At runtime, checking the Bloom Index for a given set of record keys effectively amounts to checking all the bloom filters within a given partition, against the incoming records, using a Spark join. Much of the engineering effort towards the Bloom index has gone into scaling this join -by caching the incoming RDD[HoodieRecord] to be able and dynamically tuning join parallelism, to avoid hitting Spark limitations like 2GB maximum +by caching the incoming RDD[HoodieRecord] and dynamically tuning join parallelism, to avoid hitting Spark limitations like 2GB maximum for partition size. As a result, Bloom Index implementation has been able to handle single upserts upto 5TB, in a reliable manner. @@ -61,9 +63,175 @@ records such that In this storage, index updation is a no-op, since the bloom filters are already written as a part of committing data. +In the case of Copy-On-Write, a single parquet file constitutes one `file slice` which contains one complete version of +the file + +{% include image.html file="hoodie_log_format_v2.png" alt="hoodie_log_format_v2.png" max-width="1000" %} + #### Merge On Read -Work in Progress .. .. .. .. .. +In the Merge-On-Read storage model, there are 2 logical components - one for ingesting data (both inserts/updates) into the dataset + and another for creating compacted views. The former is hereby referred to as `Writer` while the later + is referred as `Compactor`. + +##### Merge On Read Writer + + At a high level, Merge-On-Read Writer goes through same stages as Copy-On-Write writer in ingesting data. + The key difference here is that updates are appended to latest log (delta) file belonging to the latest file slice + without merging. For inserts, Hudi supports 2 modes: + + 1. Inserts to Log Files - This is done for datasets that have an indexable log files (for eg global index) + 2. Inserts to parquet files - This is done for datasets that do not have indexable log files, for eg bloom index + embedded in parquer files. Hudi treats writing new records in the same way as inserting to Copy-On-Write files. + +As in the case of Copy-On-Write, the input tagged records are partitioned such that all upserts destined to +a `file id` are grouped together. This upsert-batch is written as one or more log-blocks written to log-files. +Hudi allows clients to control log file sizes (See [Storage Configs](../configurations)) + +The WriteClient API is same for both Copy-On-Write and Merge-On-Read writers. + +With Merge-On-Read, several rounds of data-writes would have resulted in accumulation of one or more log-files. +All these log-files along with base-parquet (if exists) constitute a `file slice` which represents one complete version +of the file. + +#### Compactor + +Realtime Readers will perform in-situ merge of these delta log-files to provide the most recent (committed) view of +the dataset. To keep the query-performance in check and eventually achieve read-optimized performance, Hudi supports +compacting these log-files asynchronously to create read-optimized views. + +Asynchronous Compaction involves 2 steps: + + * `Compaction Schedule` : Hudi Write Client exposes API to create Compaction plans which contains the list of `file slice` + to be compacted atomically in a single compaction commit. Hudi allows pluggable strategies for choosing + file slices for each compaction runs. This step is typically done inline by Writer process as Hudi expects + only one schedule is being generated at a time which allows Hudi to enforce the constraint that pending compaction + plans do not step on each other file-slices. This constraint allows for multiple concurrent `Compactors` to run at + the same time. Some of the common strategies used for choosing `file slice` for compaction are: + * BoundedIO - Limit the number of file slices chosen for a compaction plan by expected total IO (read + write) + needed to complete compaction run + * Log File Size - Prefer file-slices with larger amounts of delta log data to be merged + * Day Based - Prefer file slice belonging to latest day partitions + ``` + API for scheduling compaction + /** + * Schedules a new compaction instant + * @param extraMetadata + * @return Compaction Instant timestamp if a new compaction plan is scheduled + */ + Optional scheduleCompaction(Optional> extraMetadata) throws IOException; + ``` + * `Compactor` : Hudi provides a separate API in Write Client to execute a compaction plan. The compaction + plan (just like a commit) is identified by a timestamp. Most of the design and implementation complexities for Async + Compaction is for guaranteeing snapshot isolation to readers and writer when + multiple concurrent compactors are running. Typical compactor deployment involves launching a separate + spark application which executes pending compactions when they become available. The core logic of compacting + file slices in the Compactor is very similar to that of merging updates in a Copy-On-Write table. The only + difference being in the case of compaction, there is an additional step of merging the records in delta log-files. + + Here are the main API to lookup and execute a compaction plan. + ``` + Main API in HoodieWriteClient for running Compaction: + /** + * Performs Compaction corresponding to instant-time + * @param compactionInstantTime Compaction Instant Time + * @return + * @throws IOException + */ + public JavaRDD compact(String compactionInstantTime) throws IOException; + + To lookup all pending compactions, use the API defined in HoodieReadClient + + /** + * Return all pending compactions with instant time for clients to decide what to compact next. + * @return + */ + public List> getPendingCompactions(); + ``` + +Refer to __hoodie-client/src/test/java/HoodieClientExample.java__ class for an example of how compaction +is scheduled and executed. + +##### Deployment Models + +These are typical Hoodie Writer and Compaction deployment models + + * `Inline Compaction` : At each round, a single spark application ingests new batch to dataset. It then optionally decides to schedule + a compaction run and executes it in sequence. + * `Single Dedicated Async Compactor` : The Spark application which brings in new changes to dataset (writer) periodically + schedules compaction. The Writer application does not run compaction inline. A separate spark applications periodically + probes for pending compaction and executes the compaction. + * ` Multi Async Compactors` : This mode is similar to `Single Dedicated Async Compactor` mode. The main difference being + now there can be more than one spark application picking different compactions and executing them in parallel. + In order to ensure compactors do not step on each other, they use coordination service like zookeeper to pickup unique + pending compaction instants and run them. + +The Compaction process requires one executor per file-slice in the compaction plan. So, the best resource allocation +strategy (both in terms of speed and resource usage) for clusters supporting dynamic allocation is to lookup the compaction +plan to be run to figure out the number of file slices being compacted and choose that many number of executors. + +## Async Compaction Design Deep-Dive (Optional) + +For the purpose of this section, it is important to distinguish between 2 types of commits as pertaining to the file-group: + +A commit which generates a merged and read-optimized file-slice is called `snapshot commit` (SC) with respect to that file-group. +A commit which merely appended the new/updated records assigned to the file-group into a new log block is called `delta commit` (DC) +with respect to that file-group. + +### Algorithm + +The algorithm is described with an illustration. Let us assume a scenario where there are commits SC1, DC2, DC3 that have +already completed on a data-set. Commit DC4 is currently ongoing with the writer (ingestion) process using it to upsert data. +Let us also imagine there are a set of file-groups (FG1 … FGn) in the data-set whose latest version (`File-Slice`) +contains the base file created by commit SC1 (snapshot-commit in columnar format) and a log file containing row-based +log blocks of 2 delta-commits (DC2 and DC3). + +{% include image.html file="async_compac_1.png" alt="async_compac_1.png" max-width="1000" %} + + * Writer (Ingestion) that is going to commit "DC4" starts. The record updates in this batch are grouped by file-groups + and appended in row formats to the corresponding log file as delta commit. Let us imagine a subset of file-groups has + this new log block (delta commit) DC4 added. + * Before the writer job completes, it runs the compaction strategy to decide which file-group to compact by compactor + and creates a new compaction-request commit SC5. This commit file is marked as “requested” with metadata denoting + which fileIds to compact (based on selection policy). Writer completes without running compaction (will be run async). + + {% include image.html file="async_compac_2.png" alt="async_compac_2.png" max-width="1000" %} + + * Writer job runs again ingesting next batch. It starts with commit DC6. It reads the earliest inflight compaction + request marker commit in timeline order and collects the (fileId, Compaction Commit Id “CcId” ) pairs from meta-data. + Ingestion DC6 ensures a new file-slice with base-commit “CcId” gets allocated for the file-group. + The Writer will simply append records in row-format to the first log-file (as delta-commit) assuming the + base-file (“Phantom-Base-File”) will be created eventually by the compactor. + + {% include image.html file="async_compac_3.png" alt="async_compac_3.png" max-width="1000" %} + + * Compactor runs at some time and commits at “Tc” (concurrently or before/after Ingestion DC6). It reads the commit-timeline + and finds the first unprocessed compaction request marker commit. Compactor reads the commit’s metadata finding the + file-slices to be compacted. It compacts the file-slice and creates the missing base-file (“Phantom-Base-File”) + with “CCId” as the commit-timestamp. Compactor then marks the compaction commit timestamp as completed. + It is important to realize that at data-set level, there could be different file-groups requesting compaction at + different commit timestamps. + + {% include image.html file="async_compac_4.png" alt="async_compac_4.png" max-width="1000" %} + + * Near Real-time reader interested in getting the latest snapshot will have 2 cases. Let us assume that the + incremental ingestion (writer at DC6) happened before the compaction (some time “Tc”’). + The below description is with regards to compaction from file-group perspective. + * `Reader querying at time between ingestion completion time for DC6 and compaction finish “Tc”`: + Hoodie’s implementation will be changed to become aware of file-groups currently waiting for compaction and + merge log-files corresponding to DC2-DC6 with the base-file corresponding to SC1. In essence, Hoodie will create + a pseudo file-slice by combining the 2 file-slices starting at base-commits SC1 and SC5 to one. + For file-groups not waiting for compaction, the reader behavior is essentially the same - read latest file-slice + and merge on the fly. + * `Reader querying at time after compaction finished (> “Tc”)` : In this case, reader will not find any pending + compactions in the timeline and will simply have the current behavior of reading the latest file-slice and + merging on-the-fly. + + * Read-Optimized View readers will query against the latest columnar base-file for each file-groups. + +The above algorithm explains Async compaction w.r.t a single compaction run on a single file-group. It is important +to note that multiple compaction plans can be run concurrently as they are essentially operating on different +file-groups. ## Performance From a745b9d636de0825f288186b50f983db0a4a44ee Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sat, 8 Sep 2018 12:26:54 -0700 Subject: [PATCH 092/374] Adding check for rolling stats not present to handle backwards compatibility of existing timeline --- .../main/java/com/uber/hoodie/HoodieWriteClient.java | 9 ++++++--- .../com/uber/hoodie/table/HoodieMergeOnReadTable.java | 11 +++++++---- .../com/uber/hoodie/table/TestMergeOnReadTable.java | 11 +++++++++-- 3 files changed, 22 insertions(+), 9 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 846db2b40ba2c..0e63ac167273f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -1286,9 +1286,12 @@ private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetada HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(table.getActiveTimeline().getInstantDetails(lastInstant .get()).get(), HoodieCommitMetadata.class); - rollingStatMetadata = rollingStatMetadata - .merge(HoodieCommitMetadata.fromBytes(commitMetadata.getExtraMetadata() - .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY).getBytes(), HoodieRollingStatMetadata.class)); + Optional lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + if (lastRollingStat.isPresent()) { + rollingStatMetadata = rollingStatMetadata + .merge(HoodieCommitMetadata.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class)); + } } metadata.addMetadata(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, rollingStatMetadata.toJsonString()); } catch (IOException io) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index b7d7a5080eeec..77e6ddf4fadde 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -307,10 +307,13 @@ protected HoodieRollingStatMetadata getRollingStats() { if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( this.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); - HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata - .fromBytes(commitMetadata.getExtraMetadata().get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY) - .getBytes(), HoodieRollingStatMetadata.class); - return rollingStatMetadata; + Optional lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + if (lastRollingStat.isPresent()) { + HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata + .fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class); + return rollingStatMetadata; + } } return null; } catch (IOException e) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 92c40e7e899cc..ed68e5857d741 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -911,7 +911,14 @@ public void testRollingStatsInMetadata() throws Exception { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - String commitTime = "000"; + // Create a commit without rolling stats in metadata to test backwards compatibility + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + String commitActionType = table.getMetaClient().getCommitActionType(); + HoodieInstant instant = new HoodieInstant(true, commitActionType, "000"); + activeTimeline.createInflight(instant); + activeTimeline.saveAsComplete(instant, Optional.empty()); + + String commitTime = "001"; client.startCommitWithTime(commitTime); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); @@ -936,7 +943,7 @@ public void testRollingStatsInMetadata() throws Exception { } Assert.assertEquals(inserts, 200); - commitTime = "001"; + commitTime = "002"; client.startCommitWithTime(commitTime); records = dataGen.generateUpdates(commitTime, records); writeRecords = jsc.parallelize(records, 1); From 07e33ea118c0af6aeaa139da58d02c1c34b9c73d Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Sun, 15 Jul 2018 22:34:02 -0700 Subject: [PATCH 093/374] Moving depedencies off cdh to apache + Hive2 support - Tests redone in the process - Main changes are to RealtimeRecordReader and how it treats maps/arrays - Make hive sync work with Hive 1/2 and CDH environments - Fixes to make corner cases for Hive queries - Spark Hive integration - Working version across Apache and CDH versions - Known Issue - https://github.com/uber/hudi/issues/439 --- docs/quickstart.md | 63 +++- docs/sql_queries.md | 6 +- hoodie-cli/pom.xml | 18 + hoodie-client/pom.xml | 48 ++- hoodie-common/pom.xml | 8 + .../common/model/HoodieCommitMetadata.java | 9 + .../table/log/HoodieLogFormatReader.java | 2 +- .../hoodie/common/util/HoodieAvroUtils.java | 24 +- hoodie-hadoop-mr/pom.xml | 99 +++-- .../uber/hoodie/hadoop/HoodieInputFormat.java | 74 ---- .../hadoop/HoodieROTablePathFilter.java | 2 +- .../AbstractRealtimeRecordReader.java | 112 +++++- .../realtime/HoodieRealtimeInputFormat.java | 22 +- .../RealtimeCompactedRecordReader.java | 11 +- .../hadoop/TestRecordReaderValueIterator.java | 14 +- .../HoodieRealtimeRecordReaderTest.java | 9 +- hoodie-hive/pom.xml | 139 ++++--- hoodie-hive/run_sync_tool.sh | 39 ++ .../com/uber/hoodie/hive/HiveSyncConfig.java | 16 + .../uber/hoodie/hive/HoodieHiveClient.java | 101 ++++- .../com/uber/hoodie/hive/util/SchemaUtil.java | 31 +- hoodie-spark/pom.xml | 75 +++- hoodie-spark/run_hoodie_app.sh | 24 ++ .../java/com/uber/hoodie/DataSourceUtils.java | 9 +- .../com/uber/hoodie/DataSourceOptions.scala | 25 ++ .../scala/com/uber/hoodie/DefaultSource.scala | 49 ++- hoodie-spark/src/test/java/HoodieJavaApp.java | 84 ++++- hoodie-utilities/pom.xml | 81 ++-- packaging/hoodie-hadoop-mr-bundle/pom.xml | 299 +++++++++++++++ packaging/hoodie-hive-bundle/pom.xml | 263 +++++++++++++ packaging/hoodie-spark-bundle/pom.xml | 345 ++++++++++++++++++ pom.xml | 291 +++++++++------ 32 files changed, 1984 insertions(+), 408 deletions(-) create mode 100644 hoodie-hive/run_sync_tool.sh create mode 100644 hoodie-spark/run_hoodie_app.sh create mode 100644 packaging/hoodie-hadoop-mr-bundle/pom.xml create mode 100644 packaging/hoodie-hive-bundle/pom.xml create mode 100644 packaging/hoodie-spark-bundle/pom.xml diff --git a/docs/quickstart.md b/docs/quickstart.md index f463c30756c2e..ea18c752fefdc 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -15,6 +15,11 @@ Check out code and pull it into Intellij as a normal maven project. Normally build the maven project, from command line ``` $ mvn clean install -DskipTests + +To work with older version of Hive (pre Hive-1.2.1), use + +$ mvn clean install -DskipTests -Dhive11 + ``` {% include callout.html content="You might want to add your spark jars folder to project dependencies under 'Module Setttings', to be able to run Spark from IDE" type="info" %} @@ -22,16 +27,45 @@ $ mvn clean install -DskipTests {% include note.html content="Setup your local hadoop/hive test environment, so you can play with entire ecosystem. See [this](http://www.bytearray.io/2016/05/setting-up-hadoopyarnsparkhive-on-mac.html) for reference" %} +## Supported Versions + +Hoodie requires Java 8 to be installed. Hoodie works with Spark-2.x versions. We have verified that hoodie works with the following combination of Hadoop/Hive/Spark. + +| 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 | +| 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" | + +If your environment has other versions of hadoop/hive/spark, please try out hoodie and let us know if there are any issues. We are limited by our bandwidth to certify other combinations. +It would be of great help if you can reach out to us with your setup and experience with hoodie. ## Generate a Hoodie Dataset +### Requirements & Environment Variable + +Please set the following environment variablies according to your setup. We have given an example setup with CDH version + +``` +export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64/jre/ +export HIVE_HOME=/var/hadoop/setup/apache-hive-1.1.0-cdh5.7.2-bin +export HADOOP_HOME=/var/hadoop/setup/hadoop-2.6.0-cdh5.7.2 +export HADOOP_INSTALL=/var/hadoop/setup/hadoop-2.6.0-cdh5.7.2 +export HADOOP_CONF_DIR=$HADOOP_INSTALL/etc/hadoop +export SPARK_HOME=/var/hadoop/setup/spark-2.3.1-bin-hadoop2.7 +export SPARK_INSTALL=$SPARK_HOME +export SPARK_CONF_DIR=$SPARK_HOME/conf +export PATH=$JAVA_HOME/bin:$HIVE_HOME/bin:$HADOOP_HOME/bin:$SPARK_INSTALL/bin:$PATH +``` ### DataSource API -Run __hoodie-spark/src/test/java/HoodieJavaApp.java__ class, to place a two commits (commit 1 => 100 inserts, commit 2 => 100 updates to previously inserted 100 records) onto your HDFS/local filesystem +Run __hoodie-spark/src/test/java/HoodieJavaApp.java__ class, to place a two commits (commit 1 => 100 inserts, commit 2 => 100 updates to previously inserted 100 records) onto your HDFS/local filesystem. Use the wrapper script +to run from command-line ``` - +cd hoodie-spark +./run_hoodie_app.sh --help Usage:

    [options] Options: --help, -h @@ -69,11 +103,12 @@ Now, lets see how we can publish this data into Hive. hdfs namenode # start name node hdfs datanode # start data node -bin/hive --service metastore -p 10000 # start metastore +bin/hive --service metastore # start metastore bin/hiveserver2 \ - --hiveconf hive.server2.thrift.port=10010 \ --hiveconf hive.root.logger=INFO,console \ - --hiveconf hive.aux.jars.path=hoodie/hoodie-hadoop-mr/target/hoodie-hadoop-mr-0.3.6-SNAPSHOT.jar + --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat \ + --hiveconf ive.stats.autogather=false \ + --hiveconf hive.aux.jars.path=hoodie/packaging/hoodie-hadoop-mr-bundle/target/hoodie-hadoop-mr-bundle-0.4.3-SNAPSHOT.jar ``` @@ -86,7 +121,8 @@ It uses an incremental approach by storing the last commit time synced in the TB This can be run as frequently as the ingestion pipeline to make sure new partitions and schema evolution changes are reflected immediately. ``` -{JAVA8}/bin/java -cp "/etc/hive/conf:./hoodie-hive-0.3.8-SNAPSHOT-jar-with-dependencies.jar:/opt/hadoop/lib/hadoop-mapreduce/*" com.uber.hoodie.hive.HiveSyncTool +cd hoodie-hive +./run_sync_tool.sh --user hive --pass hive --database default @@ -100,17 +136,19 @@ This can be run as frequently as the ingestion pipeline to make sure new partiti #### Manually via Beeline -Add in the hoodie-hadoop-mr jar so, Hive can read the Hoodie dataset and answer the query. +Add in the hoodie-hadoop-mr-bundler jar so, Hive can read the Hoodie dataset and answer the query. +Also, For reading hoodie tables using hive, the following configs needs to be setup ``` -hive> add jar file:///tmp/hoodie-hadoop-mr-0.2.7.jar; -Added [file:///tmp/hoodie-hadoop-mr-0.2.7.jar] to class path -Added resources: [file:///tmp/hoodie-hadoop-mr-0.2.7.jar] +hive> set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +hive> set hive.stats.autogather=false; +hive> add jar file:///tmp/hoodie-hadoop-mr-bundle-0.4.3.jar; +Added [file:///tmp/hoodie-hadoop-mr-bundle-0.4.3.jar] to class path +Added resources: [file:///tmp/hoodie-hadoop-mr-bundle-0.4.3.jar] ``` Then, you need to create a __ReadOptimized__ Hive table as below (only type supported as of now)and register the sample partitions - ``` drop table hoodie_test; CREATE EXTERNAL TABLE hoodie_test(`_row_key` string, @@ -200,8 +238,7 @@ Spark is super easy, once you get Hive working as above. Just spin up a Spark Sh ``` $ cd $SPARK_INSTALL -$ export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop -$ spark-shell --jars /tmp/hoodie-hadoop-mr-0.2.7.jar --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false +$ spark-shell --jars $HUDI_SRC/packaging/hoodie-spark-bundle/target/hoodie-spark-bundle-0.4.3-SNAPSHOT.jar --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false --packages com.databricks:spark-avro_2.11:4.0.0 scala> val sqlContext = new org.apache.spark.sql.SQLContext(sc) scala> sqlContext.sql("show tables").show(10000) diff --git a/docs/sql_queries.md b/docs/sql_queries.md index 45ebb4cf855f3..f133a85bb54b5 100644 --- a/docs/sql_queries.md +++ b/docs/sql_queries.md @@ -20,7 +20,7 @@ In the following sections, we cover the configs needed across different query en ## Hive For HiveServer2 access, [install](https://www.cloudera.com/documentation/enterprise/5-6-x/topics/cm_mc_hive_udf.html#concept_nc3_mms_lr) -the hoodie-hadoop-mr-x.y.z-SNAPSHOT.jar into the aux jars path and we should be able to recognize the Hoodie tables and query them correctly. +the hoodie-hadoop-mr-bundle-x.y.z-SNAPSHOT.jar into the aux jars path and we should be able to recognize the Hoodie tables and query them correctly. For beeline access, the `hive.input.format` variable needs to be set to the fully qualified path name of the inputformat `com.uber.hoodie.hadoop.HoodieInputFormat` For Tez, additionally the `hive.tez.input.format` needs to be set to `org.apache.hadoop.hive.ql.io.HiveInputFormat` @@ -39,7 +39,7 @@ However benchmarks have not revealed any real performance degradation with Hoodi Sample command is provided below to spin up Spark Shell ``` -$ spark-shell --jars hoodie-hadoop-mr-x.y.z-SNAPSHOT.jar --driver-class-path /etc/hive/conf --conf spark.sql.hive.convertMetastoreParquet=false --num-executors 10 --driver-memory 7g --executor-memory 2g --master yarn-client +$ spark-shell --jars hoodie-spark-bundle-x.y.z-SNAPSHOT.jar --driver-class-path /etc/hive/conf --packages com.databricks:spark-avro_2.11:4.0.0 --conf spark.sql.hive.convertMetastoreParquet=false --num-executors 10 --driver-memory 7g --executor-memory 2g --master yarn-client scala> sqlContext.sql("select count(*) from uber.trips where datestr = '2016-10-02'").show() @@ -62,7 +62,7 @@ spark.sparkContext.hadoopConfiguration.setClass("mapreduce.input.pathFilter.clas ## Presto -Presto requires a [patch](https://github.com/prestodb/presto/pull/7002) (until the PR is merged) and the hoodie-hadoop-mr jar to be placed +Presto requires a [patch](https://github.com/prestodb/presto/pull/7002) (until the PR is merged) and the hoodie-hadoop-mr-bundle jar to be placed into `/plugin/hive-hadoop2/`. {% include callout.html content="Get involved to improve this integration [here](https://github.com/uber/hoodie/issues/81)" type="info" %} diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index c696972f0bc71..515ea1d7f186f 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -167,11 +167,29 @@ log4j ${log4j.version} + + + com.uber.hoodie + hoodie-hive + ${project.version} + + com.uber.hoodie hoodie-client ${project.version} + + + org.apache.hadoop + hadoop-common + + + + org.apache.hadoop + hadoop-hdfs + + com.uber.hoodie hoodie-common diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index 7b0f7ab57c1d7..b8a316db89525 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -39,8 +39,12 @@ test-jar + test-compile + + false + org.apache.rat @@ -107,6 +111,7 @@ com.uber.hoodie hoodie-common ${project.version} + tests test-jar test @@ -183,12 +188,6 @@ ${project.version} test - - org.apache.hive - hive-exec - test - - org.apache.hbase @@ -218,5 +217,40 @@ - + + + + hive12 + + + !hive11 + + + + + ${hive12.groupid} + hive-exec + ${hive12.version} + test + + + + + hive11 + + + hive11 + + + + + ${hive11.groupid} + hive-exec + ${hive11.version} + test + + + + + diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 071ae7a636b6d..df28b11007faf 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -40,8 +40,12 @@ test-jar + test-compile + + false + org.apache.rat @@ -90,6 +94,10 @@ jackson-annotations ${fasterxml.version} + + com.fasterxml.jackson.core + jackson-databind + org.apache.parquet parquet-avro diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index c75c19d770d50..0aee2e86cbf21 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -333,4 +333,13 @@ protected static ObjectMapper getObjectMapper() { mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); return mapper; } + + @Override + public String toString() { + return "HoodieCommitMetadata{" + + "partitionToWriteStats=" + partitionToWriteStats + + ", compacted=" + compacted + + ", extraMetadataMap=" + extraMetadataMap + + '}'; + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index 8e8033b5b8b2a..8b49323e9c626 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -94,4 +94,4 @@ public HoodieLogFile getLogFile() { public void remove() { } -} \ No newline at end of file +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index 45b69e0b6a829..9c1dc2273e9e9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -27,9 +27,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; @@ -136,6 +138,26 @@ public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String re return record; } + /** + * Add null fields to passed in schema. Caller is responsible for ensuring there is no duplicates. + * As different query engines have varying constraints regarding treating the case-sensitivity of fields, its best + * to let caller determine that. + * @param schema Passed in schema + * @param newFieldNames Null Field names to be added + * @return + */ + public static Schema appendNullSchemaFields(Schema schema, List newFieldNames) { + List newFields = schema.getFields().stream().map(field -> { + return new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()); + }).collect(Collectors.toList()); + for (String newField : newFieldNames) { + newFields.add(new Schema.Field(newField, METADATA_FIELD_SCHEMA, "", null)); + } + Schema newSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError()); + newSchema.setFields(newFields); + return newSchema; + } + /** * Adds the Hoodie commit metadata into the provided Generic Record. */ @@ -155,7 +177,7 @@ public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema for (Schema.Field f : record.getSchema().getFields()) { newRecord.put(f.name(), record.get(f.name())); } - if (!new GenericData().validate(newSchema, newRecord)) { + if (!GenericData.get().validate(newSchema, newRecord)) { throw new SchemaCompatabilityException( "Unable to validate the rewritten record " + record + " against schema " + newSchema); diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index fca84ad4d7ba8..004b8ddce99e1 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -35,6 +35,7 @@ com.uber.hoodie hoodie-common ${project.version} + tests test-jar test @@ -59,17 +60,13 @@ org.apache.hadoop hadoop-hdfs - - org.apache.hive - hive-exec - commons-logging commons-logging - org.apache.hive - hive-jdbc + org.apache.commons + commons-lang3 org.apache.parquet @@ -79,6 +76,10 @@ com.twitter parquet-avro + + com.twitter + parquet-hadoop-bundle + com.twitter.common objectsize @@ -93,6 +94,11 @@ kryo test + + junit + junit + test + @@ -101,33 +107,60 @@ org.apache.rat apache-rat-plugin - - org.apache.maven.plugins - maven-shade-plugin - 2.4 - - - package - - shade - - - ${project.build.directory}/dependency-reduced-pom.xml - - true - - - com.uber.hoodie:hoodie-common - com.twitter:parquet-avro - com.twitter.common:objectsize - - - - - - - - + + + hive12 + + + !hive11 + + + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + + + commons-logging + commons-logging + + + + + ${hive12.groupid} + hive-exec + ${hive12.version} + + + + + hive11 + + + hive11 + + + + + ${hive11.groupid} + hive-jdbc + ${hive11.version} + + + commons-logging + commons-logging + + + + + ${hive11.groupid} + hive-exec + ${hive11.version} + + + + diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index 84d7da39c1bb7..4c0d548d7d0dc 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -16,13 +16,8 @@ package com.uber.hoodie.hadoop; -import static parquet.filter2.predicate.FilterApi.and; -import static parquet.filter2.predicate.FilterApi.binaryColumn; -import static parquet.filter2.predicate.FilterApi.gt; - import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; -import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; @@ -43,26 +38,13 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; -import org.apache.hadoop.hive.ql.io.parquet.read.ParquetFilterPredicateConverter; -import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; -import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; -import org.apache.hadoop.hive.ql.plan.TableScanDesc; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.Job; -import parquet.filter2.predicate.FilterPredicate; -import parquet.filter2.predicate.Operators; -import parquet.hadoop.ParquetFileReader; -import parquet.hadoop.metadata.FileMetaData; -import parquet.hadoop.metadata.ParquetMetadata; -import parquet.io.api.Binary; /** * HoodieInputFormat which understands the Hoodie File Structure and filters files based on the @@ -219,62 +201,6 @@ public RecordReader getRecordReader(final InputSplit split, return super.getRecordReader(split, job, reporter); } - /** - * Clears out the filter expression (if this is not done, then ParquetReader will override the - * FilterPredicate set) - */ - private void clearOutExistingPredicate(JobConf job) { - job.unset(TableScanDesc.FILTER_EXPR_CONF_STR); - } - - /** - * Constructs the predicate to push down to parquet storage. This creates the predicate for - * `hoodie_commit_time` > 'start_commit_time' and ANDs with the existing predicate if one is - * present already. - */ - private FilterPredicate constructHoodiePredicate(JobConf job, String tableName, InputSplit split) - throws IOException { - FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName); - LOG.info("Commit time predicate - " + commitTimePushdown.toString()); - FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split); - LOG.info("Existing predicate - " + existingPushdown); - - FilterPredicate hoodiePredicate; - if (existingPushdown != null) { - hoodiePredicate = and(existingPushdown, commitTimePushdown); - } else { - hoodiePredicate = commitTimePushdown; - } - LOG.info("Hoodie Predicate - " + hoodiePredicate); - return hoodiePredicate; - } - - private FilterPredicate constructHQLPushdownPredicate(JobConf job, InputSplit split) - throws IOException { - String serializedPushdown = job.get(TableScanDesc.FILTER_EXPR_CONF_STR); - String columnNamesString = job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR); - if (serializedPushdown == null || columnNamesString == null || serializedPushdown.isEmpty() - || columnNamesString.isEmpty()) { - return null; - } else { - SearchArgument sarg = SearchArgumentFactory - .create(Utilities.deserializeExpression(serializedPushdown)); - final Path finalPath = ((FileSplit) split).getPath(); - final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(job, finalPath); - final FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); - return ParquetFilterPredicateConverter.toFilterPredicate(sarg, fileMetaData.getSchema()); - } - } - - private FilterPredicate constructCommitTimePushdownPredicate(JobConf job, String tableName) - throws IOException { - String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName); - Operators.BinaryColumn sequenceColumn = binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD); - FilterPredicate p = gt(sequenceColumn, Binary.fromString(lastIncrementalTs)); - LOG.info("Setting predicate in InputFormat " + p.toString()); - return p; - } - /** * Read the table metadata from a data path. This assumes certain hierarchy of files which should * be changed once a better way is figured out to pass in the hoodie meta directory diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index 158aa00391dba..ae55ea21b9ce1 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -126,7 +126,7 @@ public boolean accept(Path path) { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder)); List latestFiles = fsView.getLatestDataFiles() .collect(Collectors.toList()); diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java index 383ce0cd9832a..89c2fb355ca47 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -18,6 +18,13 @@ package com.uber.hoodie.hadoop.realtime; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.log.HoodieLogFormat; +import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader; +import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; +import com.uber.hoodie.common.table.log.block.HoodieLogBlock; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; @@ -29,12 +36,15 @@ import java.util.TreeMap; import java.util.stream.Collectors; import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericArray; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.io.DoubleWritable; @@ -44,7 +54,6 @@ import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; @@ -119,10 +128,15 @@ protected static String arrayWritableToString(ArrayWritable writable) { StringBuilder builder = new StringBuilder(); Writable[] values = writable.get(); - builder.append(String.format("Size: %s,", values.length)); + builder.append(String.format("(Size: %s)[", values.length)); for (Writable w : values) { - builder.append(w + " "); + if (w instanceof ArrayWritable) { + builder.append(arrayWritableToString((ArrayWritable) w) + " "); + } else { + builder.append(w + " "); + } } + builder.append("]"); return builder.toString(); } @@ -130,12 +144,9 @@ protected static String arrayWritableToString(ArrayWritable writable) { * Given a comma separated list of field names and positions at which they appear on Hive, return * a ordered list of field names, that can be passed onto storage. */ - public static List orderFields(String fieldNameCsv, String fieldOrderCsv, - String partitioningFieldsCsv) { + private static List orderFields(String fieldNameCsv, String fieldOrderCsv, List partitioningFields) { String[] fieldOrders = fieldOrderCsv.split(","); - Set partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) - .collect(Collectors.toSet()); List fieldNames = Arrays.stream(fieldNameCsv.split(",")) .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); @@ -157,17 +168,34 @@ public static List orderFields(String fieldNameCsv, String fieldOrderCsv * columns */ public static Schema generateProjectionSchema(Schema writeSchema, List fieldNames) { + /** + * Avro & Presto field names seems to be case sensitive (support fields differing only in case) + * whereas Hive/Impala/SparkSQL(default) are case-insensitive. Spark allows this to be configurable + * using spark.sql.caseSensitive=true + * + * For a RT table setup with no delta-files (for a latest file-slice) -> we translate parquet schema to Avro + * Here the field-name case is dependent on parquet schema. Hive (1.x/2.x/CDH) translate column projections + * to lower-cases + * + */ List projectedFields = new ArrayList<>(); + Map schemaFieldsMap = writeSchema.getFields().stream() + .map(r -> Pair.of(r.name().toLowerCase(), r)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); for (String fn : fieldNames) { - Schema.Field field = writeSchema.getField(fn); + Schema.Field field = schemaFieldsMap.get(fn.toLowerCase()); if (field == null) { - throw new HoodieException("Field " + fn + " not found log schema. Query cannot proceed!"); + throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! " + + "Derived Schema Fields: " + + schemaFieldsMap.keySet().stream().collect(Collectors.toList())); } projectedFields .add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); } - return Schema.createRecord(projectedFields); + Schema projectedSchema = Schema + .createRecord(writeSchema.getName(), writeSchema.getDoc(), writeSchema.getNamespace(), writeSchema.isError()); + projectedSchema.setFields(projectedFields); + return projectedSchema; } /** @@ -176,10 +204,16 @@ public static Schema generateProjectionSchema(Schema writeSchema, List f public static Writable avroToArrayWritable(Object value, Schema schema) { // if value is null, make a NullWritable + // Hive 2.x does not like NullWritable if (value == null) { - return NullWritable.get(); + + return null; + //return NullWritable.get(); } + + Writable[] wrapperWritable; + switch (schema.getType()) { case STRING: return new Text(value.toString()); @@ -196,7 +230,8 @@ public static Writable avroToArrayWritable(Object value, Schema schema) { case BOOLEAN: return new BooleanWritable((Boolean) value); case NULL: - return NullWritable.get(); + return null; + // return NullWritable.get(); case RECORD: GenericRecord record = (GenericRecord) value; Writable[] values1 = new Writable[schema.getFields().size()]; @@ -214,7 +249,8 @@ public static Writable avroToArrayWritable(Object value, Schema schema) { for (Object obj : arrayValue) { values2[index2++] = avroToArrayWritable(obj, schema.getElementType()); } - return new ArrayWritable(Writable.class, values2); + wrapperWritable = new Writable[]{new ArrayWritable(Writable.class, values2)}; + return new ArrayWritable(Writable.class, wrapperWritable); case MAP: Map mapValue = (Map) value; Writable[] values3 = new Writable[mapValue.size()]; @@ -226,7 +262,8 @@ public static Writable avroToArrayWritable(Object value, Schema schema) { mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType()); values3[index3++] = new ArrayWritable(Writable.class, mapValues); } - return new ArrayWritable(Writable.class, values3); + wrapperWritable = new Writable[]{new ArrayWritable(Writable.class, values3)}; + return new ArrayWritable(Writable.class, wrapperWritable); case UNION: List types = schema.getTypes(); if (types.size() != 2) { @@ -248,16 +285,61 @@ public static Writable avroToArrayWritable(Object value, Schema schema) { } } + public static Schema readSchemaFromLogFile(FileSystem fs, Path path) throws IOException { + Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); + HoodieAvroDataBlock lastBlock = null; + while (reader.hasNext()) { + HoodieLogBlock block = reader.next(); + if (block instanceof HoodieAvroDataBlock) { + lastBlock = (HoodieAvroDataBlock) block; + } + } + if (lastBlock != null) { + return lastBlock.getSchema(); + } + return null; + } + + /** + * Hive implementation of ParquetRecordReader results in partition columns not present in the original parquet file + * to also be part of the projected schema. Hive expects the record reader implementation to return the row in its + * entirety (with un-projected column having null values). As we use writerSchema for this, make sure writer schema + * also includes partition columns + * @param schema Schema to be changed + * @return + */ + private static Schema addPartitionFields(Schema schema, List partitioningFields) { + final Set firstLevelFieldNames = schema.getFields().stream().map(Field::name) + .map(String::toLowerCase).collect(Collectors.toSet()); + List fieldsToAdd = partitioningFields.stream().map(String::toLowerCase) + .filter(x -> !firstLevelFieldNames.contains(x)).collect(Collectors.toList()); + + return HoodieAvroUtils.appendNullSchemaFields(schema, fieldsToAdd); + } + /** * Goes through the log files and populates a map with latest version of each key logged, since * the base split was written. */ private void init() throws IOException { writerSchema = new AvroSchemaConverter().convert(baseFileSchema); + List fieldNames = writerSchema.getFields().stream().map(Field::name).collect(Collectors.toList()); + if (split.getDeltaFilePaths().size() > 0) { + String logPath = split.getDeltaFilePaths().get(split.getDeltaFilePaths().size() - 1); + FileSystem fs = FSUtils.getFs(logPath, jobConf); + writerSchema = readSchemaFromLogFile(fs, new Path(logPath)); + fieldNames = writerSchema.getFields().stream().map(Field::name).collect(Collectors.toList()); + } + + // Add partitioning fields to writer schema for resulting row to contain null values for these fields + List partitioningFields = Arrays.stream( + jobConf.get("partition_columns", "").split(",")).collect(Collectors.toList()); + writerSchema = addPartitionFields(writerSchema, partitioningFields); + List projectionFields = orderFields( jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), - jobConf.get("partition_columns", "")); + partitioningFields); // TODO(vc): In the future, the reader schema should be updated based on log files & be able // to null out fields not present before readerSchema = generateProjectionSchema(writerSchema, projectionFields); diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index e423bd01804ed..04351202c2cf9 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -203,21 +203,31 @@ private static Configuration addRequiredProjectionFields(Configuration configura @Override public RecordReader getRecordReader(final InputSplit split, final JobConf job, final Reporter reporter) throws IOException { + + LOG.info("Before adding Hoodie columns, Projections :" + job + .get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :" + + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); + + // Hive (across all versions) fails for queries like select count(`_hoodie_commit_time`) from table; + // In this case, the projection fields gets removed. Looking at HiveInputFormat implementation, in some cases + // hoodie additional projection columns are reset after calling setConf and only natural projections + // (one found in select queries) are set. things would break because of this. + // For e:g _hoodie_record_key would be missing and merge step would throw exceptions. + // TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction time. + this.conf = addRequiredProjectionFields(job); + LOG.info("Creating record reader with readCols :" + job - .get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); + .get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :" + + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); // sanity check Preconditions.checkArgument(split instanceof HoodieRealtimeFileSplit, "HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with " + split); + return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, job, super.getRecordReader(split, job, reporter)); } - @Override - public void setConf(Configuration conf) { - this.conf = addRequiredProjectionFields(conf); - } - @Override public Configuration getConf() { return conf; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java index c64933389f53c..266e0d64c3975 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -95,8 +95,15 @@ public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? Writable[] replaceValue = deltaRecordMap.get(key).get(); Writable[] originalValue = arrayWritable.get(); - System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); - arrayWritable.set(originalValue); + try { + System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); + arrayWritable.set(originalValue); + } catch (RuntimeException re) { + LOG.error("Got exception when doing array copy", re); + LOG.error("Base record :" + arrayWritableToString(arrayWritable)); + LOG.error("Log record :" + arrayWritableToString(deltaRecordMap.get(key))); + throw re; + } } return true; } diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java index 06fc41c99158f..95aa8f036ca79 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java @@ -18,11 +18,11 @@ package com.uber.hoodie.hadoop; -import groovy.lang.Tuple2; import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.RecordReader; @@ -41,8 +41,8 @@ public void testValueIterator() { "spark", "dataset", }; - List> entries = IntStream.range(0, values.length) - .boxed().map(idx -> new Tuple2<>(idx, values[idx])).collect(Collectors.toList()); + List> entries = IntStream.range(0, values.length) + .boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList()); TestRecordReader reader = new TestRecordReader(entries); RecordReaderValueIterator itr = new RecordReaderValueIterator(reader); for (int i = 0; i < values.length; i++) { @@ -58,10 +58,10 @@ public void testValueIterator() { */ private static class TestRecordReader implements RecordReader { - private final List> entries; + private final List> entries; private int currIndex = 0; - public TestRecordReader(List> entries) { + public TestRecordReader(List> entries) { this.entries = entries; } @@ -71,8 +71,8 @@ public boolean next(IntWritable key, Text value) throws IOException { if (currIndex >= entries.size()) { return false; } - key.set(entries.get(currIndex).getFirst()); - value.set(entries.get(currIndex).getSecond()); + key.set(entries.get(currIndex).getLeft()); + value.set(entries.get(currIndex).getRight()); currIndex++; return true; } diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index a889e1a6ec91c..b3b095592ab1a 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -104,7 +104,6 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); - long size = writer.getCurrentSize(); return writer; } @@ -348,7 +347,7 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { // Assert type MAP ArrayWritable mapItem = (ArrayWritable) values[12]; - Writable[] mapItemValues = mapItem.get(); + Writable[] mapItemValues = ((ArrayWritable) mapItem.get()[0]).get(); ArrayWritable mapItemValue1 = (ArrayWritable) mapItemValues[0]; ArrayWritable mapItemValue2 = (ArrayWritable) mapItemValues[1]; Assert.assertEquals("test value for field: tags", mapItemValue1.get()[0].toString(), @@ -381,10 +380,10 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { // Assert type ARRAY ArrayWritable arrayValue = (ArrayWritable) values[14]; - Writable[] arrayValues = arrayValue.get(); + Writable[] arrayValues = ((ArrayWritable) arrayValue.get()[0]).get(); for (int i = 0; i < arrayValues.length; i++) { - Assert.assertEquals("test value for field: stringArray", arrayValues[i].toString(), - "stringArray" + i + recordCommitTimeSuffix); + Assert.assertEquals("test value for field: stringArray", "stringArray" + i + recordCommitTimeSuffix, + arrayValues[i].toString()); } } } diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index cbaedb9fd5529..e9cff4750cc08 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -43,38 +43,32 @@ org.apache.hadoop hadoop-auth - - org.apache.hive - hive-common - - - org.apache.hive - hive-jdbc - com.google.guava guava - - org.apache.hive - hive-service - - - org.apache.hive - hive-metastore - org.apache.thrift libthrift 0.9.2 + + joda-time + joda-time + + commons-dbcp commons-dbcp + + commons-io + commons-io + + org.slf4j @@ -90,6 +84,16 @@ jcommander + + org.apache.httpcomponents + httpcore + + + + org.apache.httpcomponents + httpclient + + junit @@ -159,52 +163,79 @@ org.apache.maven.plugins - maven-assembly-plugin - 2.4.1 - - - src/assembly/src.xml - - - - com.uber.hoodie.hive.HiveSyncTool - - - - + maven-jar-plugin + 2.5 - make-assembly - - package - single + test-jar - - org.apache.maven.plugins - maven-dependency-plugin - 2.4 - - - copy-dependencies - package - - copy-dependencies - - - ${project.build.directory}/jars - false - false - true - - - - - + + + hive12 + + + !hive11 + + + + + ${hive12.groupid} + hive-service + ${hive12.version} + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + + + ${hive12.groupid} + hive-metastore + ${hive12.version} + + + ${hive12.groupid} + hive-common + ${hive12.version} + + + + + hive11 + + + hive11 + + + + + org.apache.hive + hive-service + ${hive11.version} + + + org.apache.hive + hive-jdbc + ${hive11.version} + + + org.apache.hive + hive-metastore + ${hive11.version} + + + org.apache.hive + hive-common + ${hive11.version} + + + + diff --git a/hoodie-hive/run_sync_tool.sh b/hoodie-hive/run_sync_tool.sh new file mode 100644 index 0000000000000..910b4c5d8640e --- /dev/null +++ b/hoodie-hive/run_sync_tool.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +function error_exit { + echo "$1" >&2 ## Send message to stderr. Exclude >&2 if you don't want it that way. + exit "${2:-1}" ## Return a code specified by $2 or 1 by default. +} + +if [ -z "${HADOOP_HOME}" ]; then + error_exit "Please make sure the environment variable HADOOP_HOME is setup" +fi + +if [ -z "${HIVE_HOME}" ]; then + error_exit "Please make sure the environment variable HIVE_HOME is setup" +fi + +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +#Ensure we pick the right jar even for hive11 builds +HOODIE_HIVE_UBER_JAR=`ls -c $DIR/../packaging/hoodie-hive-bundle/target/hoodie-hive-*.jar | head -1` + +if [ -z "$HADOOP_CONF_DIR" ]; then + echo "setting hadoop conf dir" + HADOOP_CONF_DIR="${HADOOP_HOME}/etc/hadoop" +fi + +## Include only specific packages from HIVE_HOME/lib to avoid version mismatches +HIVE_EXEC=`ls ${HIVE_HOME}/lib/hive-exec-*.jar` +HIVE_SERVICE=`ls ${HIVE_HOME}/lib/hive-service-*.jar | grep -v rpc` +HIVE_METASTORE=`ls ${HIVE_HOME}/lib/hive-metastore-*.jar` +# Hive 1.x/CDH has standalone jdbc jar which is no longer available in 2.x +HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*standalone*.jar` +if [ -z "${HIVE_JDBC}" ]; then + HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*.jar | grep -v handler` +fi +HIVE_JARS=$HIVE_METASTORE:$HIVE_SERVICE:$HIVE_EXEC:$HIVE_SERVICE:$HIVE_JDBC + +HADOOP_HIVE_JARS=${HIVE_JARS}:${HADOOP_HOME}/share/hadoop/common/*:${HADOOP_HOME}/share/hadoop/mapreduce/*:${HADOOP_HOME}/share/hadoop/hdfs/*:${HADOOP_HOME}/share/hadoop/common/lib/*:${HADOOP_HOME}/share/hadoop/hdfs/lib/* + +echo "Running Command : java -cp ${HADOOP_HIVE_JARS}:${HADOOP_CONF_DIR}:$HOODIE_HIVE_UBER_JAR com.uber.hoodie.hive.HiveSyncTool $@" +java -cp $HOODIE_HIVE_UBER_JAR:${HADOOP_HIVE_JARS}:${HADOOP_CONF_DIR} com.uber.hoodie.hive.HiveSyncTool "$@" diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java index dd9e701493ba8..5e81ad9af5a4a 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java @@ -69,4 +69,20 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; + + @Override + public String toString() { + return "HiveSyncConfig{" + + "databaseName='" + databaseName + '\'' + + ", tableName='" + tableName + '\'' + + ", hiveUser='" + hiveUser + '\'' + + ", hivePass='" + hivePass + '\'' + + ", jdbcUrl='" + jdbcUrl + '\'' + + ", basePath='" + basePath + '\'' + + ", partitionFields=" + partitionFields + + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + + ", assumeDatePartitioning=" + assumeDatePartitioning + + ", help=" + help + + '}'; + } } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 55d74d8f122cf..03268fe2dd371 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieFileFormat; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -34,6 +35,7 @@ import java.io.IOException; import java.sql.Connection; import java.sql.DatabaseMetaData; +import java.sql.Driver; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -43,6 +45,9 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.commons.dbcp.BasicDataSource; +import org.apache.commons.dbcp.ConnectionFactory; +import org.apache.commons.dbcp.DriverConnectionFactory; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; @@ -185,8 +190,7 @@ private List constructChangePartitions(List partitions) { String fullPartitionPath = new Path(syncConfig.basePath, partition).toString(); String changePartition = - alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '" - + "hdfs://nameservice1" + fullPartitionPath + "'"; + alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '" + fullPartitionPath + "'"; changePartitions.add(changePartition); } return changePartitions; @@ -234,7 +238,7 @@ List scanTablePartitions() throws TException { void updateTableDefinition(MessageType newSchema) { try { - String newSchemaStr = SchemaUtil.generateSchemaString(newSchema); + String newSchemaStr = SchemaUtil.generateSchemaString(newSchema, syncConfig.partitionFields); // Cascade clause should not be present for non-partitioned tables String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : ""; StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append("`") @@ -242,7 +246,7 @@ void updateTableDefinition(MessageType newSchema) { .append(syncConfig.tableName).append("`") .append(" REPLACE COLUMNS(").append(newSchemaStr).append(" )") .append(cascadeClause); - LOG.info("Creating table with " + sqlBuilder); + LOG.info("Updating table definition with " + sqlBuilder); updateHiveSQL(sqlBuilder.toString()); } catch (IOException e) { throw new HoodieHiveSyncException("Failed to update table for " + syncConfig.tableName, e); @@ -311,7 +315,8 @@ public MessageType getDataSchema() { String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for commit " + lastCommit - + ", could not get schema for dataset " + metaClient.getBasePath())); + + ", could not get schema for dataset " + metaClient.getBasePath() + + ", Metadata :" + commitMetadata)); return readSchemaFromDataFile(new Path(filePath)); case MERGE_ON_READ: // If this is MOR, depending on whether the latest commit is a delta commit or @@ -340,12 +345,31 @@ public MessageType getDataSchema() { // read from the log file wrote commitMetadata = HoodieCommitMetadata.fromBytes( activeTimeline.getInstantDetails(lastDeltaInstant).get(), HoodieCommitMetadata.class); - filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION)) - .findAny().orElseThrow(() -> new IllegalArgumentException( - "Could not find any data file written for commit " + lastDeltaInstant - + ", could not get schema for dataset " + metaClient.getBasePath())); - return readSchemaFromLogFile(lastCompactionCommit, new Path(filePath)); + Pair filePathWithFormat = + commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() + .stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION)) + .findAny().map(f -> Pair.of(f, HoodieFileFormat.HOODIE_LOG)) + .orElseGet(() -> { + // No Log files in Delta-Commit. Check if there are any parquet files + return commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream() + .filter(s -> s.contains((metaClient.getTableConfig().getROFileFormat().getFileExtension()))) + .findAny() + .map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() -> { + return new IllegalArgumentException( + "Could not find any data file written for commit " + lastDeltaInstant + + ", could not get schema for dataset " + metaClient.getBasePath() + + ", CommitMetadata :" + commitMetadata); + }); + }); + switch (filePathWithFormat.getRight()) { + case HOODIE_LOG: + return readSchemaFromLogFile(lastCompactionCommit, new Path(filePathWithFormat.getLeft())); + case PARQUET: + return readSchemaFromDataFile(new Path(filePathWithFormat.getLeft())); + default: + throw new IllegalArgumentException("Unknown file format :" + filePathWithFormat.getRight() + + " for file " + filePathWithFormat.getLeft()); + } } else { return readSchemaFromLastCompaction(lastCompactionCommit); } @@ -442,14 +466,15 @@ void updateHiveSQL(String s) { private void createHiveConnection() { if (connection == null) { - BasicDataSource ds = new BasicDataSource(); - ds.setDriverClassName(driverName); + BasicDataSource ds = new HiveDataSource(); + ds.setDriverClassName(HiveDriver.class.getCanonicalName()); ds.setUrl(getHiveJdbcUrlWithDefaultDBName()); ds.setUsername(syncConfig.hiveUser); ds.setPassword(syncConfig.hivePass); LOG.info("Getting Hive Connection from Datasource " + ds); try { this.connection = ds.getConnection(); + LOG.info("Successfully got Hive Connection from Datasource " + ds); } catch (SQLException e) { throw new HoodieHiveSyncException( "Cannot create hive connection " + getHiveJdbcUrlWithDefaultDBName(), e); @@ -589,4 +614,54 @@ static PartitionEvent newPartitionUpdateEvent(String storagePartition) { return new PartitionEvent(PartitionEventType.UPDATE, storagePartition); } } + + /** + * There is a bug in BasicDataSource implementation (dbcp-1.4) which does not allow custom version of Driver (needed + * to talk to older version of HiveServer2 including CDH-5x). This is fixed in dbcp-2x but we are using dbcp1.4. + * Adding a workaround here. TODO: varadarb We need to investigate moving to dbcp-2x + */ + protected class HiveDataSource extends BasicDataSource { + + protected ConnectionFactory createConnectionFactory() throws SQLException { + try { + Driver driver = HiveDriver.class.newInstance(); + // Can't test without a validationQuery + if (validationQuery == null) { + setTestOnBorrow(false); + setTestOnReturn(false); + setTestWhileIdle(false); + } + + // Set up the driver connection factory we will use + String user = username; + if (user != null) { + connectionProperties.put("user", user); + } else { + log("DBCP DataSource configured without a 'username'"); + } + + String pwd = password; + if (pwd != null) { + connectionProperties.put("password", pwd); + } else { + log("DBCP DataSource configured without a 'password'"); + } + + ConnectionFactory driverConnectionFactory = new DriverConnectionFactory(driver, url, connectionProperties); + return driverConnectionFactory; + } catch (Throwable x) { + LOG.warn("Got exception trying to instantiate connection factory. Trying default instantiation", x); + return super.createConnectionFactory(); + } + } + + @Override + public String toString() { + return "HiveDataSource{" + + "driverClassName='" + driverClassName + '\'' + + ", driverClassLoader=" + driverClassLoader + + ", url='" + url + '\'' + + '}'; + } + } } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index 098c01368fd1f..cb60ad2bd1ce5 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -27,9 +27,11 @@ import com.uber.hoodie.hive.HoodieHiveSyncException; import com.uber.hoodie.hive.SchemaDifference; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.slf4j.Logger; @@ -322,6 +324,14 @@ private static String tickSurround(String result) { return result; } + private static String removeSurroundingTick(String result) { + if (result.startsWith("`") && result.endsWith("`")) { + result = result.substring(1, result.length() - 1); + } + + return result; + } + /** * Create a 'Map' schema from Parquet map field */ @@ -372,11 +382,17 @@ public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) } public static String generateSchemaString(MessageType storageSchema) throws IOException { + return generateSchemaString(storageSchema, new ArrayList<>()); + } + + public static String generateSchemaString(MessageType storageSchema, List colsToSkip) throws IOException { Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema); StringBuilder columns = new StringBuilder(); for (Map.Entry hiveSchemaEntry : hiveSchema.entrySet()) { - columns.append(hiveSchemaEntry.getKey()).append(" "); - columns.append(hiveSchemaEntry.getValue()).append(", "); + if (!colsToSkip.contains(removeSurroundingTick(hiveSchemaEntry.getKey()))) { + columns.append(hiveSchemaEntry.getKey()).append(" "); + columns.append(hiveSchemaEntry.getValue()).append(", "); + } } // Remove the last ", " columns.delete(columns.length() - 2, columns.length()); @@ -386,19 +402,20 @@ public static String generateSchemaString(MessageType storageSchema) throws IOEx public static String generateCreateDDL(MessageType storageSchema, HiveSyncConfig config, String inputFormatClass, String outputFormatClass, String serdeClass) throws IOException { Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema); - String columns = generateSchemaString(storageSchema); + String columns = generateSchemaString(storageSchema, config.partitionFields); - StringBuilder partitionFields = new StringBuilder(); + List partitionFields = new ArrayList<>(); for (String partitionKey : config.partitionFields) { - partitionFields.append(partitionKey).append(" ") - .append(getPartitionKeyType(hiveSchema, partitionKey)); + partitionFields.add(new StringBuilder().append(partitionKey).append(" ") + .append(getPartitionKeyType(hiveSchema, partitionKey)).toString()); } + String paritionsStr = partitionFields.stream().collect(Collectors.joining(",")); StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS "); sb = sb.append(config.databaseName).append(".").append(config.tableName); sb = sb.append("( ").append(columns).append(")"); if (!config.partitionFields.isEmpty()) { - sb = sb.append(" PARTITIONED BY (").append(partitionFields).append(")"); + sb = sb.append(" PARTITIONED BY (").append(paritionsStr).append(")"); } sb = sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'"); sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'"); diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 5ed93b3b05189..55d373d6da83e 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -118,8 +118,11 @@ - + + org.apache.avro + avro + org.scala-lang scala-library @@ -200,17 +203,22 @@ hoodie-hadoop-mr ${project.version} + + com.uber.hoodie + hoodie-hive + ${project.version} + junit junit-dep ${junit.version} test - com.uber.hoodie hoodie-client ${project.version} + tests test-jar test @@ -218,9 +226,72 @@ com.uber.hoodie hoodie-common ${project.version} + tests test-jar test + + + hive12 + + + !hive11 + + + + + ${hive12.groupid} + hive-service + ${hive12.version} + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + + + ${hive12.groupid} + hive-metastore + ${hive12.version} + + + ${hive12.groupid} + hive-common + ${hive12.version} + + + + + hive11 + + + hive11 + + + + + ${hive11.groupid} + hive-service + ${hive11.version} + + + ${hive11.groupid} + hive-jdbc + ${hive11.version} + + + ${hive11.groupid} + hive-metastore + ${hive11.version} + + + ${hive11.groupid} + hive-common + ${hive11.version} + + + + diff --git a/hoodie-spark/run_hoodie_app.sh b/hoodie-spark/run_hoodie_app.sh new file mode 100644 index 0000000000000..ec82eddb912d5 --- /dev/null +++ b/hoodie-spark/run_hoodie_app.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +function error_exit { + echo "$1" >&2 ## Send message to stderr. Exclude >&2 if you don't want it that way. + exit "${2:-1}" ## Return a code specified by $2 or 1 by default. +} + +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +#Ensure we pick the right jar even for hive11 builds +HOODIE_JAR=`ls -c $DIR/../packaging/hoodie-spark-bundle/target/hoodie-spark-bundle-*.jar | head -1` + +if [ -z "$HADOOP_CONF_DIR" ]; then + echo "setting hadoop conf dir" + HADOOP_CONF_DIR="/etc/hadoop/conf" +fi + +if [ -z "$CLIENT_JAR" ]; then + echo "client jar location not set" +fi + +OTHER_JARS=`ls -1 $DIR/target/lib/*jar | grep -v '*avro*-1.' | tr '\n' ':'` +#TODO - Need to move TestDataGenerator and HoodieJavaApp out of tests +echo "Running command : java -cp $DIR/target/test-classes/:$DIR/../hoodie-client/target/test-classes/:${HADOOP_CONF_DIR}:$HOODIE_JAR:${CLIENT_JAR}:$OTHER_JARS HoodieJavaApp $@" +java -cp $DIR/target/test-classes/:$DIR/../hoodie-client/target/test-classes/:${HADOOP_CONF_DIR}:$HOODIE_JAR:${CLIENT_JAR}:$OTHER_JARS HoodieJavaApp "$@" diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index 551a1865d101e..5c0d27248daa7 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -32,6 +32,8 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -47,7 +49,8 @@ public class DataSourceUtils { public static String getNestedFieldValAsString(GenericRecord record, String fieldName) { String[] parts = fieldName.split("\\."); GenericRecord valueNode = record; - for (int i = 0; i < parts.length; i++) { + int i = 0; + for (;i < parts.length; i++) { String part = parts[i]; Object val = valueNode.get(part); if (val == null) { @@ -65,7 +68,9 @@ public static String getNestedFieldValAsString(GenericRecord record, String fiel valueNode = (GenericRecord) val; } } - throw new HoodieException(fieldName + " field not found in record"); + throw new HoodieException(fieldName + "(Part -" + parts[i] + ") field not found in record. " + + "Acceptable fields were :" + valueNode.getSchema().getFields() + .stream().map(Field::name).collect(Collectors.toList())); } /** diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala index c4e3307c98e09..26040337f6e03 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala @@ -19,6 +19,7 @@ package com.uber.hoodie import com.uber.hoodie.common.model.HoodieTableType +import com.uber.hoodie.hive.SlashEncodedDayPartitionValueExtractor /** * List of options that can be passed to the Hoodie datasource, @@ -143,4 +144,28 @@ object DataSourceWriteOptions { */ val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix" val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_" + + // HIVE SYNC SPECIFIC CONFIGS + //NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes + // unexpected issues with config getting reset + val HIVE_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.hive_sync.enable" + val HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database" + val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table" + val HIVE_USER_OPT_KEY = "hoodie.datasource.hive_sync.username" + val HIVE_PASS_OPT_KEY = "hoodie.datasource.hive_sync.password" + val HIVE_URL_OPT_KEY = "hoodie.datasource.hive_sync.jdbcUrl" + val HIVE_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.hive_sync.partition_fields" + val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.hive_sync.partition_extractor_class" + val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = "hoodie.datasource.hive_sync.assume_date_partitioning" + + // DEFAULT FOR HIVE SPECIFIC CONFIGS + val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false" + val DEFAULT_HIVE_DATABASE_OPT_VAL = "default" + val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown" + val DEFAULT_HIVE_USER_OPT_VAL = "hive" + val DEFAULT_HIVE_PASS_OPT_VAL = "hive" + val DEFAULT_HIVE_URL_OPT_VAL = "jdbc:hive2://localhost:10000" + val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = "" + val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName + val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false" } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 5a94f9e60e9b8..580529513d97c 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -25,11 +25,12 @@ import java.util.{Optional, Properties} import com.uber.hoodie.DataSourceReadOptions._ import com.uber.hoodie.DataSourceWriteOptions._ import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient} -import com.uber.hoodie.common.util.TypedProperties +import com.uber.hoodie.common.util.{FSUtils, TypedProperties} import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException import org.apache.avro.generic.GenericRecord -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.log4j.LogManager import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD @@ -39,6 +40,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} import scala.collection.JavaConversions._ +import scala.collection.mutable.ListBuffer /** * Hoodie Spark Datasource, for reading and writing hoodie datasets @@ -92,6 +94,7 @@ class DefaultSource extends RelationProvider classOf[com.uber.hoodie.hadoop.HoodieROTablePathFilter], classOf[org.apache.hadoop.fs.PathFilter]); + log.info("Constructing hoodie (as parquet) data source with options :" + parameters) // simply return as a regular parquet relation DataSource.apply( sparkSession = sqlContext.sparkSession, @@ -118,6 +121,15 @@ class DefaultSource extends RelationProvider defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL) defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_USER_OPT_KEY, DEFAULT_HIVE_USER_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_PASS_OPT_KEY, DEFAULT_HIVE_PASS_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_URL_OPT_KEY, DEFAULT_HIVE_URL_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_PARTITION_FIELDS_OPT_KEY, DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_ASSUME_DATE_PARTITION_OPT_KEY, DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL) mapAsScalaMap(defaultsMap) } @@ -200,7 +212,8 @@ class DefaultSource extends RelationProvider } // Create a HoodieWriteClient & issue the write. - val client = DataSourceUtils.createHoodieClient(new JavaSparkContext(sparkContext), + val jsc = new JavaSparkContext(sparkContext); + val client = DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get, tblName.get, @@ -228,6 +241,13 @@ class DefaultSource extends RelationProvider else { log.info("Commit " + commitTime + " failed!") } + + val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false) + if (hiveSyncEnabled) { + log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")") + val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration) + syncHive(basePath, fs, parameters) + } client.close } else { log.error(s"Upsert failed with ${errorCount} errors :"); @@ -247,5 +267,28 @@ class DefaultSource extends RelationProvider createRelation(sqlContext, parameters, df.schema) } + private def syncHive(basePath: Path, fs: FileSystem, parameters: Map[String, String]): Boolean = { + val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, parameters) + val hiveConf: HiveConf = new HiveConf() + hiveConf.addResource(fs.getConf) + new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable() + true + } + + private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = { + val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig() + hiveSyncConfig.basePath = basePath.toString + hiveSyncConfig.assumeDatePartitioning = + parameters.get(HIVE_ASSUME_DATE_PARTITION_OPT_KEY).exists(r => r.toBoolean) + hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY) + hiveSyncConfig.tableName = parameters(HIVE_TABLE_OPT_KEY) + hiveSyncConfig.hiveUser = parameters(HIVE_USER_OPT_KEY) + hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY) + hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY) + hiveSyncConfig.partitionFields = + ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).toList: _*) + hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) + hiveSyncConfig + } override def shortName(): String = "hoodie" } diff --git a/hoodie-spark/src/test/java/HoodieJavaApp.java b/hoodie-spark/src/test/java/HoodieJavaApp.java index bef67cfc82c9e..e7693183679e0 100644 --- a/hoodie-spark/src/test/java/HoodieJavaApp.java +++ b/hoodie-spark/src/test/java/HoodieJavaApp.java @@ -29,6 +29,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.DataFrameWriter; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; @@ -48,6 +49,24 @@ public class HoodieJavaApp { @Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ") private String tableType = HoodieTableType.COPY_ON_WRITE.name(); + @Parameter(names = {"--hive-sync", "-hv"}, description = "Enable syncing to hive") + private Boolean enableHiveSync = false; + + @Parameter(names = {"--hive-db", "-hd"}, description = "hive database") + private String hiveDB = "default"; + + @Parameter(names = {"--hive-table", "-ht"}, description = "hive table") + private String hiveTable = "hoodie_sample_test"; + + @Parameter(names = {"--hive-user", "-hu"}, description = "hive username") + private String hiveUser = "hive"; + + @Parameter(names = {"--hive-password", "-hp"}, description = "hive password") + private String hivePass = "hive"; + + @Parameter(names = {"--hive-url", "-hl"}, description = "hive JDBC URL") + private String hiveJdbcUrl = "jdbc:hive://localhost:10000"; + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -86,11 +105,12 @@ public void run() throws Exception { Dataset inputDF1 = spark.read().json(jssc.parallelize(records1, 2)); // Save as hoodie dataset (copy on write) - inputDF1.write().format("com.uber.hoodie") // specify the hoodie source + DataFrameWriter writer = inputDF1.write().format("com.uber.hoodie") // specify the hoodie source .option("hoodie.insert.shuffle.parallelism", "2") // any hoodie client config can be passed like this .option("hoodie.upsert.shuffle.parallelism", "2") // full list in HoodieWriteConfig & its package + .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) // insert .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), @@ -101,9 +121,11 @@ public void run() throws Exception { "timestamp") // use to combine duplicate records in input/with disk val .option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries .mode( - SaveMode.Overwrite) // This will remove any existing data at path below, and create a - // new dataset if needed - .save(tablePath); // ultimately where the dataset will be placed + SaveMode.Overwrite); // This will remove any existing data at path below, and create a + + updateHiveSyncConfig(writer); + // new dataset if needed + writer.save(tablePath); // ultimately where the dataset will be placed String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); logger.info("First commit at instant time :" + commitInstantTime1); @@ -113,12 +135,15 @@ public void run() throws Exception { List records2 = DataSourceTestUtils.convertToStringList( dataGen.generateUpdates("002"/* ignore */, 100)); Dataset inputDF2 = spark.read().json(jssc.parallelize(records2, 2)); - inputDF2.write().format("com.uber.hoodie").option("hoodie.insert.shuffle.parallelism", "2") + writer = inputDF2.write().format("com.uber.hoodie").option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.upsert.shuffle.parallelism", "2") + .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") - .option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append).save(tablePath); + .option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append); + updateHiveSyncConfig(writer); + writer.save(tablePath); String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); logger.info("Second commit at instant time :" + commitInstantTime1); @@ -135,18 +160,39 @@ public void run() throws Exception { spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 2.0") .show(); - /** - * Consume incrementally, only changes in commit 2 above. - */ - Dataset hoodieIncViewDF = spark.read().format("com.uber.hoodie") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), - DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), - commitInstantTime1) // Only changes in write 2 above - .load( - tablePath); // For incremental view, pass in the root/base path of dataset - - logger.info("You will only see records from : " + commitInstantTime2); - hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show(); + if (tableType.equals(HoodieTableType.COPY_ON_WRITE.name())) { + /** + * Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE + */ + Dataset hoodieIncViewDF = spark.read().format("com.uber.hoodie") + .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), + DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), + commitInstantTime1) // Only changes in write 2 above + .load( + tablePath); // For incremental view, pass in the root/base path of dataset + + logger.info("You will only see records from : " + commitInstantTime2); + hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show(); + } + } + + /** + * Setup configs for syncing to hive + * @param writer + * @return + */ + private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { + if (enableHiveSync) { + logger.info("Enabling Hive sync to " + hiveJdbcUrl); + writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable) + .option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB) + .option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl) + .option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr") + .option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser) + .option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass) + .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true"); + } + return writer; } } diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index e8159d54080f1..de5018989761a 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -118,6 +118,7 @@ com.uber.hoodie hoodie-common ${project.version} + tests test-jar test @@ -175,31 +176,11 @@ com.uber.hoodie hoodie-client ${project.version} + tests test-jar test - - - org.apache.hive - hive-jdbc - ${hive.version}-cdh${cdh.version} - standalone - - - org.slf4j - slf4j-api - - - javax.servlet - servlet-api - - - com.fasterxml.jackson.* - * - - - - + commons-dbcp commons-dbcp @@ -278,7 +259,7 @@ org.apache.avro avro-mapred - 1.7.6-cdh5.7.2 + 1.7.7 @@ -323,4 +304,58 @@ + + + hive12 + + + !hive11 + + + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + standalone + + + org.slf4j + slf4j-api + + + javax.servlet + servlet-api + + + + + + + hive11 + + + hive11 + + + + + org.apache.hive + hive-jdbc + ${hive11.version} + standalone + + + org.slf4j + slf4j-api + + + javax.servlet + servlet-api + + + + + + diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml new file mode 100644 index 0000000000000..c299cef7061dd --- /dev/null +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -0,0 +1,299 @@ + + + + + + hoodie + com.uber.hoodie + 0.4.3-SNAPSHOT + ../../pom.xml + + 4.0.0 + + hoodie-hadoop-mr-bundle + + + + com.uber.hoodie + hoodie-common + ${project.version} + + + + com.uber.hoodie + hoodie-hadoop-mr + ${project.version} + + + + com.uber.hoodie + * + + + + + + org.apache.hadoop + hadoop-common + + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + org.apache.hadoop + hadoop-mapreduce-client-common + + + + org.apache.hadoop + hadoop-auth + + + + org.apache.hadoop + hadoop-hdfs + + + + commons-logging + commons-logging + + + + org.apache.commons + commons-lang3 + + + + org.apache.parquet + parquet-avro + + + + com.twitter + parquet-avro + + + + com.twitter + parquet-hadoop-bundle + + + + com.twitter.common + objectsize + 0.0.12 + + + + org.apache.avro + avro + + + + com.esotericsoftware + kryo + test + + + + junit + junit + test + + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + 2.4 + + + package + + shade + + + ${project.build.directory}/dependency-reduced-pom.xml + + + + parquet.avro + com.uber.hoodie.parquet.avro + + + parquet.column + com.uber.hoodie.parquet.column + + + parquet.format. + com.uber.hoodie.parquet.format. + + + parquet.hadoop. + com.uber.hoodie.parquet.hadoop. + + + parquet.schema + com.uber.hoodie.parquet.schema + + + org.apache.commons + com.uber.hoodie.org.apache.commons + + + false + + + com.uber.hoodie:hoodie-common + com.uber.hoodie:hoodie-hadoop-mr + com.twitter:parquet-avro + com.twitter:parquet-hadoop-bundle + com.twitter.common:objectsize + commons-logging:commons-logging + org.apache.commons:commons-lang3 + + + ${project.artifactId}-${project.version}${hiveJarSuffix} + ${hiveJarSuffix} + + + + + + + + + hive12 + + + !hive11 + + + + + + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + + + commons-logging + commons-logging + + + + + ${hive12.groupid} + hive-exec + ${hive12.version} + + + ${hive12.groupid} + hive-service + ${hive12.version} + + + ${hive12.groupid} + hive-shims + ${hive12.version} + + + ${hive12.groupid} + hive-serde + ${hive12.version} + + + ${hive12.groupid} + hive-metastore + ${hive12.version} + + + ${hive12.groupid} + hive-common + ${hive12.version} + + + + + hive11 + + + hive11 + + + + .hive11 + + + + ${hive11.groupid} + hive-service + ${hive11.version} + + + ${hive11.groupid} + hive-shims + ${hive11.version} + + + ${hive11.groupid} + hive-jdbc + ${hive11.version} + + + commons-logging + commons-logging + + + + + ${hive11.groupid} + hive-serde + ${hive11.version} + + + ${hive11.groupid} + hive-metastore + ${hive11.version} + + + ${hive11.groupid} + hive-common + ${hive11.version} + + + ${hive11.groupid} + hive-exec + ${hive11.version} + + + + + + diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml new file mode 100644 index 0000000000000..a0d43c9cc08ff --- /dev/null +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -0,0 +1,263 @@ + + + + + + hoodie + com.uber.hoodie + 0.4.3-SNAPSHOT + ../../pom.xml + + 4.0.0 + + hoodie-hive-bundle + jar + + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-auth + + + com.google.guava + guava + + + org.apache.thrift + libthrift + 0.9.2 + + + + joda-time + joda-time + + + + + commons-dbcp + commons-dbcp + + + + commons-io + commons-io + + + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + com.beust + jcommander + + + + org.apache.httpcomponents + httpcore + + + + org.apache.httpcomponents + httpclient + + + + com.twitter + parquet-avro + + + + com.uber.hoodie + hoodie-hadoop-mr-bundle + ${project.version} + + + + com.uber.hoodie + hoodie-hive + ${project.version} + + + + com.uber.hoodie + * + + + + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + 2.4 + + + package + + shade + + + + + com.beust. + com.uber.hoodie.com.beust. + + + org.joda. + com.uber.hoodie.org.joda. + + + com.google. + com.uber.hoodie.com.google. + + + org.slf4j. + com.uber.hoodie.org.slf4j. + + + org.apache.commons. + com.uber.hoodie.org.apache.commons. + + + parquet.column + com.uber.hoodie.parquet.column + + + parquet.format. + com.uber.hoodie.parquet.format. + + + parquet.hadoop. + com.uber.hoodie.parquet.hadoop. + + + parquet.schema. + com.uber.hoodie.parquet.schema. + + + false + + + log4j:log4j + org.apache.hadoop:* + org.apache.hive:* + org.apache.derby:derby + + + ${project.artifactId}-${project.version}${hiveJarSuffix} + + + + + + + + + + hive12 + + + !hive11 + + + + + + + + ${hive12.groupid} + hive-service + ${hive12.version} + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + + + ${hive12.groupid} + hive-metastore + ${hive12.version} + + + ${hive12.groupid} + hive-common + ${hive12.version} + + + + + hive11 + + + hive11 + + + + .hive11 + + + + org.apache.hive + hive-service + ${hive11.version} + + + org.apache.hive + hive-jdbc + ${hive11.version} + + + org.apache.hive + hive-metastore + ${hive11.version} + + + org.apache.hive + hive-common + ${hive11.version} + + + + + diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml new file mode 100644 index 0000000000000..624b69bb97ac2 --- /dev/null +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -0,0 +1,345 @@ + + + + + + hoodie + com.uber.hoodie + 0.4.3-SNAPSHOT + ../../pom.xml + + 4.0.0 + + com.uber.hoodie + hoodie-spark-bundle + jar + + + 1.2.17 + 4.10 + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + prepare-package + + copy-dependencies + + + ${project.build.directory}/lib + true + true + true + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + 2.4 + + + package + + shade + + + + + com.beust. + com.uber.hoodie.com.beust. + + + org.joda. + com.uber.hoodie.org.joda. + + + com.google. + com.uber.hoodie.com.google. + + + org.slf4j. + com.uber.hoodie.org.slf4j. + + + org.apache. + com.uber.hoodie.org.apache. + + com.databricks.spark.** + org.apache.avro.** + org.apache.derby.** + org.apache.hadoop.** + org.apache.hive.** + org.apache.logging.log4j.** + org.apache.log4j.** + org.apache.spark.** + org.apache.thrift.** + + + + parquet.column + com.uber.hoodie.parquet.column + + + parquet.format. + com.uber.hoodie.parquet.format. + + + parquet.hadoop. + com.uber.hoodie.parquet.hadoop. + + + parquet.schema + com.uber.hoodie.parquet.schema + + + org.apache.hive.jdbc. + com.uber.hoodie.org.apache.hive.jdbc. + + + org.apache.hadoop.hive.metastore. + com.uber.hoodie.org.apache.hadoop_hive.metastore. + + + org.apache.hive.common. + com.uber.hoodie.org.apache.hive.common. + + + org.apache.hadoop.hive.common. + com.uber.hoodie.org.apache.hadoop_hive.common. + + + org.apache.hadoop.hive.conf. + com.uber.hoodie.org.apache.hadoop_hive.conf. + + + org.apache.hive.service. + com.uber.hoodie.org.apache.hive.service. + + + org.apache.hadoop.hive.service. + com.uber.hoodie.org.apache.hadoop_hive.service. + + + false + + + com.databricks:spark-avro_2.11 + log4j:* + org.apache.avro:* + org.apache.derby:derby + org.apache.hadoop:* + org.apache.hbase:* + + org.apache.hive:hive-exec + org.apache.hive:hive-serde + org.apache.hive:hive-shims + org.apache.spark:* + + + ${project.artifactId}-${project.version}${hiveJarSuffix} + + + + + + + + + + com.beust + jcommander + + + commons-dbcp + commons-dbcp + + + org.apache.avro + avro + + + org.scala-lang + scala-library + ${scala.version} + + + org.scalatest + scalatest_2.11 + 3.0.1 + test + + + org.apache.spark + spark-core_2.11 + + + org.apache.spark + spark-sql_2.11 + + + com.databricks + spark-avro_2.11 + 4.0.0 + + + com.fasterxml.jackson.core + jackson-annotations + + + org.apache.hadoop + hadoop-client + + + javax.servlet + * + + + provided + + + org.apache.hadoop + hadoop-common + provided + + + log4j + log4j + ${log4j.version} + + + org.apache.avro + avro + + + org.apache.commons + commons-lang3 + + + org.apache.commons + commons-configuration2 + + + com.uber.hoodie + hoodie-common + ${project.version} + + + com.uber.hoodie + hoodie-hadoop-mr + ${project.version} + + + com.uber.hoodie + hoodie-hive + ${project.version} + + + com.uber.hoodie + hoodie-client + ${project.version} + + + com.uber.hoodie + hoodie-spark + ${project.version} + + + + + + hive12 + + + !hive11 + + + + + + + + ${hive12.groupid} + hive-service + ${hive12.version} + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + + + ${hive12.groupid} + hive-metastore + ${hive12.version} + + + ${hive12.groupid} + hive-common + ${hive12.version} + + + + + hive11 + + + hive11 + + + + .hive11 + + + + ${hive11.groupid} + hive-service + ${hive11.version} + + + ${hive11.groupid} + hive-jdbc + ${hive11.version} + + + ${hive11.groupid} + hive-metastore + ${hive11.version} + + + ${hive11.groupid} + hive-common + ${hive11.version} + + + + + + diff --git a/pom.xml b/pom.xml index d321ddb15267d..d77f8f23cd374 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS - https://github.com/uber/hoodie + https://github.com/uber/hudi Hoodie @@ -36,6 +36,9 @@ hoodie-hive hoodie-utilities hoodie-spark + packaging/hoodie-hadoop-mr-bundle + packaging/hoodie-hive-bundle + packaging/hoodie-spark-bundle @@ -61,7 +64,7 @@ prasanna Prasanna Rajaperumal - Uber + Snowflake @@ -94,23 +97,14 @@ Nishith Agarwal Uber + + Balaji Varadharajan + Uber + 2015-2016 - - - com.google.code.gson - gson - 2.3.1 - test - - - junit - junit - ${junit.version} - test - - + 2.10 @@ -121,11 +115,15 @@ 4.11 1.9.5 1.2.17 - 5.7.2 - 2.6.0 - 1.1.0 + 2.9.9 + 2.7.3 + org.apache.hive + 1.2.1 + org.apache.hive + 1.1.1 3.1.1 2.1.0 + 1.7.7 2.11.8 2.11 @@ -278,32 +276,6 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - org.apache.rat @@ -337,7 +309,7 @@ org.apache.avro avro-maven-plugin - 1.7.6 + ${avro.version} generate-sources @@ -359,6 +331,19 @@ + + com.google.code.gson + gson + 2.3.1 + test + + + + junit + junit + ${junit.version} + test + com.beust @@ -372,10 +357,17 @@ ${log4j.version} + + + joda-time + joda-time + ${joda.version} + + org.apache.hadoop hadoop-client - ${hadoop.version}-cdh${cdh.version} + ${hadoop.version} provided @@ -404,7 +396,7 @@ org.apache.avro avro-mapred - 1.7.7 + ${avro.version} @@ -418,25 +410,19 @@ org.apache.hadoop hadoop-common - ${hadoop.version}-cdh${cdh.version} + ${hadoop.version} provided org.apache.hadoop hadoop-hdfs - ${hadoop.version}-cdh${cdh.version} + ${hadoop.version} provided org.apache.hadoop hadoop-auth - ${hadoop.version}-cdh${cdh.version} - provided - - - org.apache.hive - hive-common - ${hive.version}-cdh${cdh.version} + ${hadoop.version} provided @@ -448,19 +434,13 @@ org.apache.hadoop hadoop-mapreduce-client-core - ${hadoop.version}-cdh${cdh.version} + ${hadoop.version} provided org.apache.hadoop hadoop-mapreduce-client-common - 2.6.0-cdh5.7.2 - provided - - - org.apache.hive - hive-exec - 1.1.0-cdh5.7.2 + ${hadoop.version} provided @@ -468,30 +448,34 @@ commons-logging 1.2 - + + commons-io + commons-io + 2.6 + com.twitter parquet-hadoop-bundle - 1.5.0-cdh5.7.2 + 1.6.0 com.twitter parquet-hive-bundle - 1.5.0 + 1.6.0 com.twitter parquet-avro - 1.5.0-cdh5.7.2 + 1.6.0 org.apache.parquet parquet-hive-bundle - 1.8.1 + ${parquet.version} @@ -532,7 +516,7 @@ org.apache.avro avro - 1.7.6-cdh5.7.2 + ${avro.version} org.slf4j @@ -574,6 +558,11 @@ httpcore 4.3.2 + + org.apache.httpcomponents + httpclient + 4.3.2 + org.slf4j slf4j-api @@ -621,35 +610,17 @@ 1.9.13 + + com.fasterxml.jackson.core + jackson-databind + 2.6.0 + org.codehaus.jackson jackson-mapper-asl 1.9.13 - - org.apache.hive - hive-jdbc - ${hive.version}-cdh${cdh.version} - - - com.fasterxml.jackson.* - * - - - - - - org.apache.hive - hive-service - ${hive.version}-cdh${cdh.version} - - - org.apache.hive - hive-metastore - ${hive.version}-cdh${cdh.version} - - junit junit @@ -659,33 +630,13 @@ org.apache.hadoop hadoop-hdfs tests - ${hadoop.version}-cdh${cdh.version} - - - org.codehaus - * - - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - test + ${hadoop.version} org.apache.hadoop hadoop-common tests - ${hadoop.version}-cdh${cdh.version} + ${hadoop.version} org.mockito @@ -703,8 +654,11 @@ - + + Maven repository + https://central.maven.org/maven2/ + cloudera-repo-releases https://repository.cloudera.com/artifactory/public/ @@ -723,6 +677,109 @@ + + hive12 + + + !hive11 + + + + + ${hive12.groupid} + hive-service + ${hive12.version} + provided + + + ${hive12.groupid} + hive-shims + ${hive12.version} + provided + + + ${hive12.groupid} + hive-jdbc + ${hive12.version} + provided + + + ${hive12.groupid} + hive-serde + ${hive12.version} + provided + + + ${hive12.groupid} + hive-metastore + ${hive12.version} + provided + + + ${hive12.groupid} + hive-common + ${hive12.version} + provided + + + ${hive12.groupid} + hive-exec + ${hive12.version} + provided + + + + + hive11 + + + hive11 + + + + + org.apache.hive + hive-service + ${hive11.version} + + + org.apache.hive + hive-shims + ${hive11.version} + provided + + + org.apache.hive + hive-jdbc + ${hive11.version} + provided + + + org.apache.hive + hive-serde + ${hive11.version} + provided + + + org.apache.hive + hive-metastore + ${hive11.version} + provided + + + org.apache.hive + hive-common + ${hive11.version} + provided + + + org.apache.hive + hive-exec + ${hive11.version} + provided + + + release From 9c46e6ac6a5a90a8d36a5e7e1c4ed1ff72ee16d6 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Sat, 8 Sep 2018 03:52:49 +0800 Subject: [PATCH 094/374] Rebasing and fixing conflicts against master --- .../src/main/java/com/uber/hoodie/HoodieReadClient.java | 2 +- .../src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java | 4 ++-- .../main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java | 3 +-- .../java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java | 4 ++-- .../src/main/java/com/uber/hoodie/table/WorkloadStat.java | 2 +- .../hoodie/io/strategy/TestHoodieCompactionStrategy.java | 2 +- hoodie-hadoop-mr/pom.xml | 4 ---- .../hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java | 2 +- .../uber/hoodie/hadoop/TestRecordReaderValueIterator.java | 2 +- .../src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 5 +++++ packaging/hoodie-hive-bundle/pom.xml | 4 ++++ packaging/hoodie-spark-bundle/pom.xml | 1 + 13 files changed, 21 insertions(+), 16 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 416d2905c3340..94a48b166b39f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIndexException; @@ -35,7 +36,6 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index e7864d496645e..0d991642d7b4b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; @@ -36,7 +37,6 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.commons.lang3.tuple.Pair; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -260,4 +260,4 @@ private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) { } return false; } -} \ No newline at end of file +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index a65008b4c2a8a..a5dff6b90c145 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -56,7 +56,6 @@ import java.util.stream.Stream; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -315,7 +314,7 @@ private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter( com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper .convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class); // Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer - avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, StringUtils.EMPTY); + avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, ""); return avroMetaData; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 1f9ea2f05377d..d8a0f484b5459 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -35,6 +35,7 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; @@ -63,7 +64,6 @@ import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -863,4 +863,4 @@ public int getPartition(Object key) { protected HoodieRollingStatMetadata getRollingStats() { return null; } -} \ No newline at end of file +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java index a93ac55af9414..84d8c712a985b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java @@ -17,9 +17,9 @@ package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecordLocation; +import com.uber.hoodie.common.util.collection.Pair; import java.io.Serializable; import java.util.HashMap; -import org.apache.commons.lang3.tuple.Pair; /** * Wraps stats about a single partition path. diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index 44a3e6537f549..b041546cf2f8c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -25,6 +25,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.strategy.BoundedIOCompactionStrategy; @@ -37,7 +38,6 @@ import java.util.Optional; import java.util.Random; import java.util.stream.Collectors; -import org.apache.commons.lang3.tuple.Pair; import org.junit.Assert; import org.junit.Test; diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 004b8ddce99e1..3424009cc4b7f 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -64,10 +64,6 @@ commons-logging commons-logging - - org.apache.commons - commons-lang3 - org.apache.parquet parquet-avro diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java index 89c2fb355ca47..22dff95ff348a 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; @@ -40,7 +41,6 @@ import org.apache.avro.generic.GenericArray; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; -import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java index 95aa8f036ca79..1fb8239eab3a1 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java @@ -18,11 +18,11 @@ package com.uber.hoodie.hadoop; +import com.uber.hoodie.common.util.collection.Pair; import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.RecordReader; diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 03268fe2dd371..53c86526517dc 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -29,6 +29,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidDatasetException; import com.uber.hoodie.hive.util.SchemaUtil; @@ -47,7 +48,6 @@ import org.apache.commons.dbcp.BasicDataSource; import org.apache.commons.dbcp.ConnectionFactory; import org.apache.commons.dbcp.DriverConnectionFactory; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index c299cef7061dd..625605cb2cb03 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -185,6 +185,11 @@ + + + true + + hive12 diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index a0d43c9cc08ff..4878992aa65c9 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -192,6 +192,10 @@ + + true + + hive12 diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 624b69bb97ac2..ef80eac50ded3 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -33,6 +33,7 @@ 1.2.17 4.10 + true From 6899a58206723ac5e3ddec77bc4b433c7ef03d25 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Fri, 7 Sep 2018 18:30:49 -0700 Subject: [PATCH 095/374] Bump up versions in packaging modules and remove commons-lang3 dep --- .../main/scala/com/uber/hoodie/DefaultSource.scala | 1 + hoodie-spark/src/test/scala/DataSourceTest.scala | 11 ++--------- packaging/hoodie-hadoop-mr-bundle/pom.xml | 8 +------- packaging/hoodie-hive-bundle/pom.xml | 2 +- packaging/hoodie-spark-bundle/pom.xml | 6 +----- 5 files changed, 6 insertions(+), 22 deletions(-) diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 580529513d97c..1a79b352a6d58 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -28,6 +28,7 @@ import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient} import com.uber.hoodie.common.util.{FSUtils, TypedProperties} import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException +import com.uber.hoodie.hive.{HiveSyncConfig, HiveSyncTool} import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index 1bdc92777c1a1..821a7225a7b7f 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -124,14 +124,7 @@ class DataSourceTest extends AssertionsForJUnit { assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) // Read RO View - try { - val hoodieROViewDF1 = spark.read.format("com.uber.hoodie") - .load(basePath + "/*/*/*/*") - fail("we should error out, since no compaction has yet occurred.") - } catch { - case e: AnalysisException => { - // do nothing - } - }; + val hoodieROViewDF1 = spark.read.format("com.uber.hoodie").load(basePath + "/*/*/*/*") + assertEquals(100, hoodieROViewDF1.count()) // still 100, since we only updated } } diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 625605cb2cb03..9e20c1faf5b24 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.4-SNAPSHOT ../../pom.xml 4.0.0 @@ -76,11 +76,6 @@ commons-logging - - org.apache.commons - commons-lang3 - - org.apache.parquet parquet-avro @@ -174,7 +169,6 @@ com.twitter:parquet-hadoop-bundle com.twitter.common:objectsize commons-logging:commons-logging - org.apache.commons:commons-lang3 ${project.artifactId}-${project.version}${hiveJarSuffix} diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index 4878992aa65c9..c55666d1febb2 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.4-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index ef80eac50ded3..75bb572ac844d 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.3-SNAPSHOT + 0.4.4-SNAPSHOT ../../pom.xml 4.0.0 @@ -239,10 +239,6 @@ org.apache.avro avro - - org.apache.commons - commons-lang3 - org.apache.commons commons-configuration2 From fe5b887065f01bb328eb9965c6f1dd79ee6b9d8b Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Mon, 10 Sep 2018 10:59:59 -0700 Subject: [PATCH 096/374] Reduce minimum delta-commits required for compaction --- .../java/com/uber/hoodie/config/HoodieCompactionConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index d1584aac02649..cd84505336fee 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -99,7 +99,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { .name(); private static final String DEFAULT_AUTO_CLEAN = "true"; private static final String DEFAULT_INLINE_COMPACT = "false"; - private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10"; + private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "1"; private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3"; private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24"; private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128); From fb0818fe251add4c9d4eadb849e9ecab8ce34a34 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Mon, 10 Sep 2018 16:06:31 -0700 Subject: [PATCH 097/374] Use spark Master from environment if set --- .../src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java index 74fce58f56836..2dd446b8d2768 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java @@ -48,7 +48,10 @@ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntax public static JavaSparkContext initJavaSparkConf(String name) { SparkConf sparkConf = new SparkConf().setAppName(name); - sparkConf.setMaster(DEFUALT_SPARK_MASTER); + String defMasterFromEnv = sparkConf.get("spark.master"); + if ((null == defMasterFromEnv) || (defMasterFromEnv.isEmpty())) { + sparkConf.setMaster(DEFUALT_SPARK_MASTER); + } sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); sparkConf.set("spark.driver.maxResultSize", "2g"); sparkConf.set("spark.eventLog.overwrite", "true"); From 8d779253a701cb311ab7f3e7fb360ecc9eb7855e Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 12 Sep 2018 23:38:00 +0530 Subject: [PATCH 098/374] Release notes for 0.4.4 --- RELEASE_NOTES.md | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index 4fffd75b2002e..b7bc5c3c01d29 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -1,3 +1,31 @@ +Release 0.4.4 +------------------------------------ + +### Highlights + * Dependencies are now decoupled from CDH and based on apache versions! + * Support for Hive 2 is here!! Use -Dhive11 to build for older hive versions + * Deltastreamer tool reworked to make configs simpler, hardended tests, added Confluent Kafka support + * Removed dependency on commons lang3, to ease use with different hadoop/spark versions + * Better CLI support and docs for managing async compactions + * New CLI commands to manage datasets + +### Full PR List + + * **@saravsars** - Updated jcommander version to fix NPE in HoodieDeltaStreamer tool #443 + * **@n3nash** - Removing dependency on apache-commons lang 3, adding necessary classes as needed #444 + * **@n3nash** - Small file size handling for inserts into log files. #413 + * **@vinothchandar** - Update Gemfile.lock with higher ffi version + * **@bvaradar** - Simplify and fix CLI to schedule and run compactions #447 + * **@n3nash** - Fix a failing test case intermittenly in TestMergeOnRead due to incorrect prev commit time #448 + * **@bvaradar**- CLI to create and desc hoodie table #446 + * **@vinothchandar**- Reworking the deltastreamer tool #449 + * **@bvaradar**- Docs for describing async compaction and how to operate it #445 + * **@n3nash**- Adding check for rolling stats not present in existing timeline to handle backwards compatibility #451 + * **@bvaradar** **@vinothchandar** - Moving all dependencies off cdh and to apache #420 + * **@bvaradar**- Reduce minimum delta-commits required for compaction #452 + * **@bvaradar**- Use spark Master from environment if set #454 + + Release 0.4.3 ------------------------------------ From 03a1b1ad0274db3cedd29c6ebc2aa73ad2099ee0 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 12 Sep 2018 23:59:34 +0530 Subject: [PATCH 099/374] [maven-release-plugin] prepare release hoodie-0.4.4 --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 4 ++-- packaging/hoodie-hive-bundle/pom.xml | 4 ++-- packaging/hoodie-spark-bundle/pom.xml | 4 ++-- pom.xml | 4 ++-- 11 files changed, 15 insertions(+), 15 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 515ea1d7f186f..91e99552bcb77 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index b8a316db89525..ee7a570ba9cb0 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index df28b11007faf..72c27c03c52b8 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 3424009cc4b7f..3c7bba57e741f 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index e9cff4750cc08..c4b9ebfb18bdc 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 55d373d6da83e..856db0d4fa14e 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index de5018989761a..91bab7b10f765 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 9e20c1faf5b24..32e88d0a70eb6 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 ../../pom.xml 4.0.0 @@ -193,7 +193,7 @@ - + diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index c55666d1febb2..09082f2c56121 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 ../../pom.xml 4.0.0 @@ -205,7 +205,7 @@ - + diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 75bb572ac844d..c5e5d0407d138 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 ../../pom.xml 4.0.0 @@ -279,7 +279,7 @@ - + diff --git a/pom.xml b/pom.xml index d77f8f23cd374..a36214628038c 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.4-SNAPSHOT + 0.4.4 Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -132,7 +132,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - HEAD + hoodie-0.4.4 From dc6c21403990eefb5c192eeeab8fde63379bf362 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 12 Sep 2018 23:59:47 +0530 Subject: [PATCH 100/374] [maven-release-plugin] prepare for next development iteration --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 2 +- packaging/hoodie-hive-bundle/pom.xml | 2 +- packaging/hoodie-spark-bundle/pom.xml | 2 +- pom.xml | 4 ++-- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 91e99552bcb77..2920f81ccc899 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index ee7a570ba9cb0..eccfa802c58b8 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 72c27c03c52b8..e56d35279da68 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 3c7bba57e741f..fe1df4396a740 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index c4b9ebfb18bdc..f74e5ea8350b0 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 856db0d4fa14e..15fb3ae930b96 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 91bab7b10f765..8a9feac7f31ef 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 32e88d0a70eb6..d200de1feb382 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index 09082f2c56121..eafad25922921 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index c5e5d0407d138..95b4aedd42e54 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index a36214628038c..ec859cf6b8d8c 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.4 + 0.4.5-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -132,7 +132,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - hoodie-0.4.4 + HEAD From 11909cbc9504dd7edf5e120f4ac0ff0edfa511e9 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Thu, 13 Sep 2018 15:01:53 +0530 Subject: [PATCH 101/374] [maven-release-plugin] rollback the release of hoodie-0.4.4 --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 4 ++-- packaging/hoodie-hive-bundle/pom.xml | 4 ++-- packaging/hoodie-spark-bundle/pom.xml | 4 ++-- pom.xml | 2 +- 11 files changed, 14 insertions(+), 14 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 2920f81ccc899..515ea1d7f186f 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index eccfa802c58b8..b8a316db89525 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index e56d35279da68..df28b11007faf 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index fe1df4396a740..3424009cc4b7f 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index f74e5ea8350b0..e9cff4750cc08 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 15fb3ae930b96..55d373d6da83e 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 8a9feac7f31ef..de5018989761a 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index d200de1feb382..9e20c1faf5b24 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT ../../pom.xml 4.0.0 @@ -193,7 +193,7 @@ - + diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index eafad25922921..c55666d1febb2 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT ../../pom.xml 4.0.0 @@ -205,7 +205,7 @@ - + diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 95b4aedd42e54..75bb572ac844d 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT ../../pom.xml 4.0.0 @@ -279,7 +279,7 @@ - + diff --git a/pom.xml b/pom.xml index ec859cf6b8d8c..d77f8f23cd374 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.5-SNAPSHOT + 0.4.4-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS From cf7ccac4aab6250b8fc4f350b298d07905f109dc Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Fri, 14 Sep 2018 12:51:27 +0530 Subject: [PATCH 102/374] Fix bug with incrementally pulling older data --- .../com/uber/hoodie/IncrementalRelation.scala | 4 ++++ hoodie-spark/src/test/scala/DataSourceTest.scala | 16 ++++++++++++++-- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index b0014d3b2fe3e..019a76e235e18 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -88,8 +88,12 @@ class IncrementalRelation(val sqlContext: SQLContext, .get, classOf[HoodieCommitMetadata]) fileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap } + // unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view + // will filter out all the files incorrectly. + sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class"); val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) sqlContext.read.options(sOpts) + .schema(latestSchema) // avoid AnalysisException for empty input .parquet(fileIdToFullPath.values.toList: _*) .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)) .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)) diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index 821a7225a7b7f..f75192b352e6c 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -98,14 +98,26 @@ class DataSourceTest extends AssertionsForJUnit { // Read Incremental View + // we have 2 commits, try pulling the first commit (which is not the latest) + val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0); + val hoodieIncViewDF1 = spark.read.format("com.uber.hoodie") + .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit) + .load(basePath); + assertEquals(100, hoodieIncViewDF1.count()) // 100 initial inserts must be pulled + var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); + assertEquals(1, countsPerCommit.length) + assertEquals(firstCommit, countsPerCommit(0).get(0)) + + // pull the latest commit val hoodieIncViewDF2 = spark.read.format("com.uber.hoodie") .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) .load(basePath); - assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled - val countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); + countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); assertEquals(1, countsPerCommit.length) assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) } From d3011834b7ef50bc5809781579e41948feb26984 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Mon, 17 Sep 2018 00:25:24 -0700 Subject: [PATCH 103/374] Add dummy classes to dump all classes loaded as part of packaging modules to ensure javadoc and sources jars are getting created --- .../hoodie/common/util/ReflectionUtils.java | 17 +++++++++ packaging/hoodie-hadoop-mr-bundle/pom.xml | 1 - .../com/uber/hoodie/hadoop/bundle/Main.java | 36 +++++++++++++++++++ .../com/uber/hoodie/hive/bundle/Main.java | 36 +++++++++++++++++++ .../com/uber/hoodie/spark/bundle/Main.java | 36 +++++++++++++++++++ 5 files changed, 125 insertions(+), 1 deletion(-) create mode 100644 packaging/hoodie-hadoop-mr-bundle/src/main/java/com/uber/hoodie/hadoop/bundle/Main.java create mode 100644 packaging/hoodie-hive-bundle/src/main/java/com/uber/hoodie/hive/bundle/Main.java create mode 100644 packaging/hoodie-spark-bundle/src/main/java/com/uber/hoodie/spark/bundle/Main.java diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java index 7416067e0e600..54348d5f56c72 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java @@ -16,12 +16,16 @@ package com.uber.hoodie.common.util; +import com.google.common.reflect.ClassPath; +import com.google.common.reflect.ClassPath.ClassInfo; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.stream.Stream; public class ReflectionUtils { @@ -82,4 +86,17 @@ public static Object loadClass(String clazz, Object... constructorArgs) { .map(arg -> arg.getClass()).toArray(Class[]::new); return loadClass(clazz, constructorArgTypes, constructorArgs); } + + /** + * Return stream of top level class names in the same class path as passed-in class + * @param clazz + */ + public static Stream getTopLevelClassesInClasspath(Class clazz) { + try { + ClassPath classPath = ClassPath.from(clazz.getClassLoader()); + return classPath.getTopLevelClasses().stream().map(ClassInfo::getName); + } catch (IOException e) { + throw new RuntimeException("Got exception while dumping top level classes", e); + } + } } diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 9e20c1faf5b24..121f72fbcd3be 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -172,7 +172,6 @@ ${project.artifactId}-${project.version}${hiveJarSuffix} - ${hiveJarSuffix} diff --git a/packaging/hoodie-hadoop-mr-bundle/src/main/java/com/uber/hoodie/hadoop/bundle/Main.java b/packaging/hoodie-hadoop-mr-bundle/src/main/java/com/uber/hoodie/hadoop/bundle/Main.java new file mode 100644 index 0000000000000..749de725ea88c --- /dev/null +++ b/packaging/hoodie-hadoop-mr-bundle/src/main/java/com/uber/hoodie/hadoop/bundle/Main.java @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.hadoop.bundle; + +import com.uber.hoodie.common.util.ReflectionUtils; + +/** + * A simple main class to dump all classes loaded in current classpath + * + * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating + * javadoc and sources plugins do not generate corresponding jars if there are no source files. + * + * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy. + */ +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} diff --git a/packaging/hoodie-hive-bundle/src/main/java/com/uber/hoodie/hive/bundle/Main.java b/packaging/hoodie-hive-bundle/src/main/java/com/uber/hoodie/hive/bundle/Main.java new file mode 100644 index 0000000000000..dad7e3af74f2c --- /dev/null +++ b/packaging/hoodie-hive-bundle/src/main/java/com/uber/hoodie/hive/bundle/Main.java @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.hive.bundle; + +import com.uber.hoodie.common.util.ReflectionUtils; + +/** + * A simple main class to dump all classes loaded in current classpath. + * + * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating + * javadoc and sources plugins do not generate corresponding jars if there are no source files. + * + * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy. + */ +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} diff --git a/packaging/hoodie-spark-bundle/src/main/java/com/uber/hoodie/spark/bundle/Main.java b/packaging/hoodie-spark-bundle/src/main/java/com/uber/hoodie/spark/bundle/Main.java new file mode 100644 index 0000000000000..9daead7d77269 --- /dev/null +++ b/packaging/hoodie-spark-bundle/src/main/java/com/uber/hoodie/spark/bundle/Main.java @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.spark.bundle; + +import com.uber.hoodie.common.util.ReflectionUtils; + +/** + * A simple main class to dump all classes loaded in current classpath + * + * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating + * javadoc and sources plugins do not generate corresponding jars if there are no source files. + * + * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy. + */ +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} From 957d2c77c8ccf810aaa031470154763b5edde9c1 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 19 Sep 2018 13:13:04 -0700 Subject: [PATCH 104/374] Explicitly release resources in LogFileReader and TestHoodieClientBase --- .../cli/commands/ArchivedCommitsCommand.java | 1 + .../cli/commands/HoodieLogFileCommand.java | 2 ++ .../com/uber/hoodie/TestAsyncCompaction.java | 5 +++ .../java/com/uber/hoodie/TestCleaner.java | 5 +++ .../com/uber/hoodie/TestClientRollback.java | 6 ++++ .../com/uber/hoodie/TestHoodieClientBase.java | 36 +++++++++++++++++-- .../TestHoodieClientOnCopyOnWriteStorage.java | 6 ++++ .../com/uber/hoodie/TestHoodieReadClient.java | 6 ++++ .../hoodie/io/TestHoodieCommitArchiveLog.java | 1 + .../log/AbstractHoodieLogRecordScanner.java | 13 ++++++- .../common/table/log/HoodieLogFileReader.java | 10 ++++-- .../table/log/HoodieLogFormatReader.java | 23 ++++++++++++ .../common/table/log/HoodieLogFormatTest.java | 10 +++++- .../AbstractRealtimeRecordReader.java | 1 + .../com/uber/hoodie/hive/util/SchemaUtil.java | 1 + 15 files changed, 119 insertions(+), 7 deletions(-) diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 893e03fcf76d8..51545d842dbaa 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -77,6 +77,7 @@ public String showCommits( List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r)) .collect(Collectors.toList()); allCommits.addAll(readCommits); + reader.close(); } TableHeader header = new TableHeader().addTableHeaderField("CommitTime") diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 8533d7afb2aee..23b8154319c01 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -131,6 +131,7 @@ public String showLogFileCommits( totalEntries++; } } + reader.close(); } List rows = new ArrayList<>(); int i = 0; @@ -221,6 +222,7 @@ public String showLogFileRecords(@CliOption(key = { } } } + reader.close(); if (allRecords.size() >= limit) { break; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java index 26fddc17a6065..c13c0c43e6364 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -73,6 +73,11 @@ private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } + @Override + public void tearDown() throws IOException { + super.tearDown(); + } + @Test public void testRollbackInflightIngestionWithPendingCompaction() throws Exception { // Rollback inflight ingestion when there is pending compaction diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 6b5440b3d5c26..2d2417ccaa0d8 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -85,6 +85,11 @@ public class TestCleaner extends TestHoodieClientBase { private static final int BIG_BATCH_INSERT_SIZE = 500; private static Logger logger = LogManager.getLogger(TestHoodieClientBase.class); + @Override + public void tearDown() throws IOException { + super.tearDown(); + } + /** * Helper method to do first batch of insert for clean by versions/commits tests * diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java index 2c88602196218..175318653edcb 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java @@ -37,6 +37,7 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; import java.io.File; +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.spark.api.java.JavaRDD; @@ -47,6 +48,11 @@ */ public class TestClientRollback extends TestHoodieClientBase { + @Override + public void tearDown() throws IOException { + super.tearDown(); + } + /** * Test case for rollback-savepoint interaction */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java index 5545992faa305..d33697bafbf0c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -51,6 +51,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -63,10 +65,13 @@ */ public class TestHoodieClientBase implements Serializable { + protected static Logger logger = LogManager.getLogger(TestHoodieClientBase.class); + protected transient JavaSparkContext jsc = null; protected transient SQLContext sqlContext; protected transient FileSystem fs; protected String basePath = null; + protected TemporaryFolder folder = null; protected transient HoodieTestDataGenerator dataGen = null; @Before @@ -78,10 +83,10 @@ public void init() throws IOException { //SQLContext stuff sqlContext = new SQLContext(jsc); - // Create a temp folder as the base path - TemporaryFolder folder = new TemporaryFolder(); + folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); if (fs instanceof LocalFileSystem) { LocalFileSystem lfs = (LocalFileSystem) fs; @@ -94,6 +99,33 @@ public void init() throws IOException { dataGen = new HoodieTestDataGenerator(); } + @After + /** + * Properly release resources at end of each test + */ + public void tearDown() throws IOException { + if (null != sqlContext) { + logger.info("Clearing sql context cache of spark-session used in previous test-case"); + sqlContext.clearCache(); + } + + if (null != jsc) { + logger.info("Closing spark context used in previous test-case"); + jsc.close(); + } + + // Create a temp folder as the base path + if (null != folder) { + logger.info("Explicitly removing workspace used in previously run test-case"); + folder.delete(); + } + + if (null != fs) { + logger.warn("Closing file-system instance used in previous test-run"); + fs.close(); + } + } + /** * Get Default HoodieWriteConfig for tests * diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 09b8795ab4309..684e9ec20d226 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -43,6 +43,7 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; import java.io.FileInputStream; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -64,6 +65,11 @@ @SuppressWarnings("unchecked") public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { + @Override + public void tearDown() throws IOException { + super.tearDown(); + } + /** * Test Auto Commit behavior for HoodieWriteClient insert API */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java index 9f60cc37078cb..8070be795860a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java @@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.config.HoodieWriteConfig; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -35,6 +36,11 @@ */ public class TestHoodieReadClient extends TestHoodieClientBase { + @Override + public void tearDown() throws IOException { + super.tearDown(); + } + /** * Test ReadFilter API after writing new records using HoodieWriteClient.insert */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index 6f03691803e8e..76c36411587d4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -271,6 +271,7 @@ public void testArchiveDatasetWithArchival() throws IOException { // verify in-flight instants after archive verifyInflightInstants(metaClient, 3); + reader.close(); } @Test diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index 427adda65dd09..56271904da727 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; import java.util.Deque; @@ -115,9 +116,10 @@ public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List new HoodieLogFile(new Path(logFile))) .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize); @@ -239,6 +241,15 @@ public void scan() { } catch (Exception e) { log.error("Got exception when reading log file", e); throw new HoodieIOException("IOException when reading log file "); + } finally { + try { + if (null != logFormatReaderWrapper) { + logFormatReaderWrapper.close(); + } + } catch (IOException ioe) { + // Eat exception as we do not want to mask the original exception that can happen + log.error("Unable to close log format reader", ioe); + } } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java index dd0348b909679..836870e6e330e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -62,6 +62,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { private long reverseLogFilePosition; private long lastReverseLogFilePosition; private boolean reverseReader; + private boolean closed = false; HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, boolean readBlockLazily, boolean reverseReader) throws IOException { @@ -95,13 +96,13 @@ public HoodieLogFile getLogFile() { } /** - * Close the inputstream when the JVM exits + * Close the inputstream if not closed when the JVM exits */ private void addShutDownHook() { Runtime.getRuntime().addShutdownHook(new Thread() { public void run() { try { - inputStream.close(); + close(); } catch (Exception e) { log.warn("unable to close input stream for log file " + logFile, e); // fail silently for any sort of exception @@ -277,7 +278,10 @@ private long scanForNextAvailableBlockOffset() throws IOException { @Override public void close() throws IOException { - this.inputStream.close(); + if (!closed) { + this.inputStream.close(); + closed = true; + } } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index 8b49323e9c626..b2ddede010b7d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -20,6 +20,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; @@ -29,6 +30,8 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final List logFiles; + // Readers for previously scanned log-files that are still open + private final List prevReadersInOpenState; private HoodieLogFileReader currentReader; private final FileSystem fs; private final Schema readerSchema; @@ -46,6 +49,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { this.readBlocksLazily = readBlocksLazily; this.reverseLogReader = reverseLogReader; this.bufferSize = bufferSize; + this.prevReadersInOpenState = new ArrayList<>(); if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false); @@ -53,7 +57,20 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { } @Override + /** + * Note : In lazy mode, clients must ensure close() should be called only after processing + * all log-blocks as the underlying inputstream will be closed. + * TODO: We can introduce invalidate() API at HoodieLogBlock and this object can call invalidate on + * all returned log-blocks so that we check this scenario specifically in HoodieLogBlock + */ public void close() throws IOException { + + for (HoodieLogFileReader reader : prevReadersInOpenState) { + reader.close(); + } + + prevReadersInOpenState.clear(); + if (currentReader != null) { currentReader.close(); } @@ -69,6 +86,12 @@ public boolean hasNext() { } else if (logFiles.size() > 0) { try { HoodieLogFile nextLogFile = logFiles.remove(0); + // First close previous reader only if readBlockLazily is true + if (!readBlocksLazily) { + this.currentReader.close(); + } else { + this.prevReadersInOpenState.add(currentReader); + } this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false); } catch (IOException io) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index dcc2c4996cc62..3c57b55f25758 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -304,6 +304,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords, dataBlockRead.getRecords()); + reader.close(); } @SuppressWarnings("unchecked") @@ -370,6 +371,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, dataBlockRead.getRecords()); + reader.close(); } @SuppressWarnings("unchecked") @@ -454,6 +456,8 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep //assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes())); assertFalse("There should be no more block left", reader.hasNext()); + reader.close(); + // Simulate another failure back to back outputStream = fs.append(writer.getLogFile().getPath()); // create a block with @@ -493,6 +497,7 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep assertTrue("We should get the last block next", reader.hasNext()); reader.next(); assertFalse("We should have no more blocks left", reader.hasNext()); + reader.close(); } @@ -1097,7 +1102,7 @@ public void testMagicAndLogVersionsBackwardsCompatibility() assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK); dBlock = (HoodieAvroDataBlock) block; assertEquals(dBlock.getRecords().size(), 100); - + reader.close(); } @SuppressWarnings("unchecked") @@ -1167,6 +1172,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect dataBlockRead.getRecords()); assertFalse(reader.hasPrev()); + reader.close(); } @Test @@ -1224,6 +1230,7 @@ public void testAppendAndReadOnCorruptedLogInReverse() throws IOException, URISy e.printStackTrace(); // We should have corrupted block } + reader.close(); } @SuppressWarnings("unchecked") @@ -1283,5 +1290,6 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect dataBlockRead.getRecords()); assertFalse(reader.hasPrev()); + reader.close(); } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java index 22dff95ff348a..e2bd10b8ddbe1 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -294,6 +294,7 @@ public static Schema readSchemaFromLogFile(FileSystem fs, Path path) throws IOEx lastBlock = (HoodieAvroDataBlock) block; } } + reader.close(); if (lastBlock != null) { return lastBlock.getSchema(); } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index cb60ad2bd1ce5..c410273bd7bf1 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -447,6 +447,7 @@ public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws lastBlock = (HoodieAvroDataBlock) block; } } + reader.close(); if (lastBlock != null) { return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema()); } From 88536be550155a8856824324110b0d8cad358b9e Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Sun, 16 Sep 2018 08:06:30 -0700 Subject: [PATCH 105/374] Hive Sync handling must work for datasets with multi-partition keys --- .../com/uber/hoodie/hive/HiveSyncConfig.java | 14 +++++ .../uber/hoodie/hive/HoodieHiveClient.java | 51 +++++++++---------- .../uber/hoodie/hive/HiveSyncToolTest.java | 31 +++++++++++ .../java/com/uber/hoodie/hive/TestUtil.java | 4 ++ .../util/MultiPartKeysValueExtractor.java | 33 ++++++++++++ 5 files changed, 106 insertions(+), 27 deletions(-) create mode 100644 hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java index 5e81ad9af5a4a..d040a5e79abdf 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java @@ -85,4 +85,18 @@ public String toString() { + ", help=" + help + '}'; } + + public static HiveSyncConfig copy(HiveSyncConfig cfg) { + HiveSyncConfig newConfig = new HiveSyncConfig(); + newConfig.basePath = cfg.basePath; + newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; + newConfig.databaseName = cfg.databaseName; + newConfig.hivePass = cfg.hivePass; + newConfig.hiveUser = cfg.hiveUser; + newConfig.partitionFields = cfg.partitionFields; + newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; + newConfig.jdbcUrl = cfg.jdbcUrl; + newConfig.tableName = cfg.tableName; + return newConfig; + } } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 53c86526517dc..de7c7aed35f15 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -40,6 +40,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -152,45 +153,41 @@ private String constructAddPartitions(List partitions) { alterSQL.append(syncConfig.databaseName).append(".").append(syncConfig.tableName) .append(" ADD IF NOT EXISTS "); for (String partition : partitions) { - - StringBuilder partBuilder = new StringBuilder(); - List partitionValues = partitionValueExtractor - .extractPartitionValuesInPath(partition); - Preconditions.checkArgument(syncConfig.partitionFields.size() == partitionValues.size(), - "Partition key parts " + syncConfig.partitionFields - + " does not match with partition values " + partitionValues - + ". Check partition strategy. "); - for (int i = 0; i < syncConfig.partitionFields.size(); i++) { - partBuilder.append(syncConfig.partitionFields.get(i)).append("=").append("'") - .append(partitionValues.get(i)).append("'"); - } - + String partitionClause = getPartitionClause(partition); String fullPartitionPath = new Path(syncConfig.basePath, partition).toString(); - alterSQL.append(" PARTITION (").append(partBuilder.toString()).append(") LOCATION '") + alterSQL.append(" PARTITION (").append(partitionClause).append(") LOCATION '") .append(fullPartitionPath).append("' "); } return alterSQL.toString(); } + /** + * Generate Hive Partition from partition values + * @param partition Partition path + * @return + */ + private String getPartitionClause(String partition) { + List partitionValues = partitionValueExtractor + .extractPartitionValuesInPath(partition); + Preconditions.checkArgument(syncConfig.partitionFields.size() == partitionValues.size(), + "Partition key parts " + syncConfig.partitionFields + + " does not match with partition values " + partitionValues + + ". Check partition strategy. "); + List partBuilder = new ArrayList<>(); + for (int i = 0; i < syncConfig.partitionFields.size(); i++) { + partBuilder.add(syncConfig.partitionFields.get(i) + "=" + "'" + partitionValues.get(i) + "'"); + } + return partBuilder.stream().collect(Collectors.joining(",")); + } + private List constructChangePartitions(List partitions) { List changePartitions = Lists.newArrayList(); String alterTable = "ALTER TABLE " + syncConfig.databaseName + "." + syncConfig.tableName; for (String partition : partitions) { - StringBuilder partBuilder = new StringBuilder(); - List partitionValues = partitionValueExtractor - .extractPartitionValuesInPath(partition); - Preconditions.checkArgument(syncConfig.partitionFields.size() == partitionValues.size(), - "Partition key parts " + syncConfig.partitionFields - + " does not match with partition values " + partitionValues - + ". Check partition strategy. "); - for (int i = 0; i < syncConfig.partitionFields.size(); i++) { - partBuilder.append(syncConfig.partitionFields.get(i)).append("=").append("'") - .append(partitionValues.get(i)).append("'"); - } - + String partitionClause = getPartitionClause(partition); String fullPartitionPath = new Path(syncConfig.basePath, partition).toString(); String changePartition = - alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '" + fullPartitionPath + "'"; + alterTable + " PARTITION (" + partitionClause + ") SET LOCATION '" + fullPartitionPath + "'"; changePartitions.add(changePartition); } return changePartitions; diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java index 844b6c3641659..be5a04e5d4664 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java @@ -22,9 +22,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent; import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType; +import com.uber.hoodie.hive.util.MultiPartKeysValueExtractor; import com.uber.hoodie.hive.util.SchemaUtil; import java.io.IOException; import java.net.URISyntaxException; @@ -356,4 +358,33 @@ public void testSyncMergeOnReadRT() TestUtil.hiveSyncConfig.tableName = roTablename; } + @Test + public void testMultiPartitionKeySync() + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { + String commitTime = "100"; + TestUtil.createCOWDataset(commitTime, 5); + + HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(TestUtil.hiveSyncConfig); + hiveSyncConfig.partitionValueExtractorClass = MultiPartKeysValueExtractor.class.getCanonicalName(); + hiveSyncConfig.tableName = "multi_part_key"; + hiveSyncConfig.partitionFields = Lists.newArrayList("year", "month", "day"); + TestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + + HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, + TestUtil.getHiveConf(), TestUtil.fileSystem); + assertFalse("Table " + hiveSyncConfig.tableName + " should not exist initially", + hiveClient.doesTableExist()); + // Lets do the sync + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); + tool.syncHoodieTable(); + assertTrue("Table " + hiveSyncConfig.tableName + " should exist after sync completes", + hiveClient.doesTableExist()); + assertEquals("Hive Schema should match the dataset schema + partition fields", + hiveClient.getTableSchema().size(), hiveClient.getDataSchema().getColumns().size() + 3); + assertEquals("Table partitions should match the number of partitions we wrote", 5, + hiveClient.scanTablePartitions().size()); + assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", + commitTime, hiveClient.getLastCommitTimeSynced().get()); + } } diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java index a2a3ab2bb0a50..7093bcf9c7d7c 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java @@ -358,4 +358,8 @@ private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetada fsout.write(bytes); fsout.close(); } + + public static Set getCreatedTablesSet() { + return createdTablesSet; + } } diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java new file mode 100644 index 0000000000000..610126c46298e --- /dev/null +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java @@ -0,0 +1,33 @@ +/* + * 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.hive.util; + +import com.uber.hoodie.hive.PartitionValueExtractor; +import java.util.Arrays; +import java.util.List; + +/** + * Partition Key extractor treating each value delimited by slash as separate key. + */ +public class MultiPartKeysValueExtractor implements PartitionValueExtractor { + + @Override + public List extractPartitionValuesInPath(String partitionPath) { + String[] splits = partitionPath.split("/"); + return Arrays.asList(splits); + } +} \ No newline at end of file From ba62c9bb50e01e9de6288121c214730968dc23a8 Mon Sep 17 00:00:00 2001 From: Yishuang Lu Date: Sun, 23 Sep 2018 17:23:35 -0700 Subject: [PATCH 106/374] Fix the name of avro schema file in Test Fixed the name of avro schema file in Test Signed-off-by: Yishuang Lu --- .../java/com/uber/hoodie/common/util/SchemaTestUtil.java | 6 +++--- ...{complex-test-evolved.avro => complex-test-evolved.avsc} | 0 .../{simple-test-evolved.avro => simple-test-evolved.avsc} | 0 .../test/resources/{simple-test.avro => simple-test.avsc} | 0 .../java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java | 2 +- .../src/test/resources/{sample1.avro => sample1.avsc} | 0 pom.xml | 2 +- 7 files changed, 5 insertions(+), 5 deletions(-) rename hoodie-common/src/test/resources/{complex-test-evolved.avro => complex-test-evolved.avsc} (100%) rename hoodie-common/src/test/resources/{simple-test-evolved.avro => simple-test-evolved.avsc} (100%) rename hoodie-common/src/test/resources/{simple-test.avro => simple-test.avsc} (100%) rename hoodie-hadoop-mr/src/test/resources/{sample1.avro => sample1.avsc} (100%) diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java index 02c8933cbefd2..c096391630094 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java @@ -47,7 +47,7 @@ public class SchemaTestUtil { public static Schema getSimpleSchema() throws IOException { - return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avsc")); } public static List generateTestRecords(int from, int limit) throws IOException, URISyntaxException { @@ -163,7 +163,7 @@ public static List updateHoodieTestRecordsWithoutHoodieMetadata(Li } public static Schema getEvolvedSchema() throws IOException { - return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avsc")); } public static List generateEvolvedTestRecords(int from, int limit) @@ -172,7 +172,7 @@ public static List generateEvolvedTestRecords(int from, int limit } public static Schema getComplexEvolvedSchema() throws IOException { - return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avsc")); } public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, String commitTime, diff --git a/hoodie-common/src/test/resources/complex-test-evolved.avro b/hoodie-common/src/test/resources/complex-test-evolved.avsc similarity index 100% rename from hoodie-common/src/test/resources/complex-test-evolved.avro rename to hoodie-common/src/test/resources/complex-test-evolved.avsc diff --git a/hoodie-common/src/test/resources/simple-test-evolved.avro b/hoodie-common/src/test/resources/simple-test-evolved.avsc similarity index 100% rename from hoodie-common/src/test/resources/simple-test-evolved.avro rename to hoodie-common/src/test/resources/simple-test-evolved.avsc diff --git a/hoodie-common/src/test/resources/simple-test.avro b/hoodie-common/src/test/resources/simple-test.avsc similarity index 100% rename from hoodie-common/src/test/resources/simple-test.avro rename to hoodie-common/src/test/resources/simple-test.avsc diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java index 779184e0a2caa..0ba1c5a82da52 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/HoodieInputFormatTest.java @@ -176,7 +176,7 @@ public void testIncrementalWithMultipleCommits() throws IOException { //TODO enable this after enabling predicate pushdown public void testPredicatePushDown() throws IOException { // initial commit - Schema schema = InputFormatTestUtil.readSchema("/sample1.avro"); + Schema schema = InputFormatTestUtil.readSchema("/sample1.avsc"); String commit1 = "20160628071126"; File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1); InputFormatTestUtil.commit(basePath, commit1); diff --git a/hoodie-hadoop-mr/src/test/resources/sample1.avro b/hoodie-hadoop-mr/src/test/resources/sample1.avsc similarity index 100% rename from hoodie-hadoop-mr/src/test/resources/sample1.avro rename to hoodie-hadoop-mr/src/test/resources/sample1.avsc diff --git a/pom.xml b/pom.xml index d77f8f23cd374..a3731096cafb2 100644 --- a/pom.xml +++ b/pom.xml @@ -288,7 +288,7 @@ **/*.sh **/*.log **/dependency-reduced-pom.xml - **/test/resources/*.avro + **/test/resources/*.avsc **/test/resources/*.data **/test/resources/*.schema **/test/resources/*.csv From 90c5b590e83b12971ea35e6b77b28153231b8d0d Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Fri, 21 Sep 2018 17:09:51 -0700 Subject: [PATCH 107/374] Travis CI tests needs to be run in quieter mode (WARN log level) to avoid max log-size errors --- .travis.yml | 4 ++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ .../resources/log4j-surefire-quiet.properties | 21 +++++++++++++++++++ pom.xml | 14 ++++++++++++- 9 files changed, 164 insertions(+), 1 deletion(-) create mode 100644 hoodie-client/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hoodie-common/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hoodie-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hoodie-hive/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hoodie-integ-test/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hoodie-spark/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hoodie-utilities/src/test/resources/log4j-surefire-quiet.properties diff --git a/.travis.yml b/.travis.yml index c6205dc4e0365..d3f5d7b8d6dff 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,3 +2,7 @@ language: java jdk: - oraclejdk8 sudo: required +env: + - HUDI_QUIETER_LOGGING=1 +services: + - docker diff --git a/hoodie-client/src/test/resources/log4j-surefire-quiet.properties b/hoodie-client/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-client/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/hoodie-common/src/test/resources/log4j-surefire-quiet.properties b/hoodie-common/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-common/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/hoodie-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties b/hoodie-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-hadoop-mr/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/hoodie-hive/src/test/resources/log4j-surefire-quiet.properties b/hoodie-hive/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-hive/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/hoodie-integ-test/src/test/resources/log4j-surefire-quiet.properties b/hoodie-integ-test/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-integ-test/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/hoodie-spark/src/test/resources/log4j-surefire-quiet.properties b/hoodie-spark/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-spark/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/hoodie-utilities/src/test/resources/log4j-surefire-quiet.properties b/hoodie-utilities/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b687ef3a69ce6 --- /dev/null +++ b/hoodie-utilities/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,21 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n diff --git a/pom.xml b/pom.xml index a3731096cafb2..c86adfa11026f 100644 --- a/pom.xml +++ b/pom.xml @@ -126,6 +126,7 @@ 1.7.7 2.11.8 2.11 + file://${project.basedir}/src/test/resources/log4j-surefire.properties @@ -212,7 +213,7 @@ ${surefireArgLine} - file:${project.build.testOutputDirectory}/log4j-surefire.properties + ${surefire-log4j.file} @@ -847,6 +848,17 @@ + + warn-log + + + env.HUDI_QUIETER_LOGGING + + + + file://${project.basedir}/src/test/resources/log4j-surefire-quiet.properties + + From 1257cf78e518c3c20a6db512c789bf9f5a324120 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Thu, 20 Sep 2018 17:50:27 +0530 Subject: [PATCH 108/374] Perform consistency checks during write finalize - Check to ensure written files are listable on storage - Docs reflected to capture how this helps with s3 storage - Unit tests added, corrections to existing tests - Fix DeltaStreamer to manage archived commits in a separate folder --- docs/configurations.md | 2 + .../com/uber/hoodie/HoodieWriteClient.java | 51 ++++---- .../uber/hoodie/config/HoodieWriteConfig.java | 13 ++ .../com/uber/hoodie/io/ConsistencyCheck.java | 112 ++++++++++++++++++ .../hoodie/table/HoodieCopyOnWriteTable.java | 67 +++++------ .../hoodie/table/HoodieMergeOnReadTable.java | 8 +- .../com/uber/hoodie/table/HoodieTable.java | 33 ++++-- .../java/com/uber/hoodie/TestCleaner.java | 7 +- .../com/uber/hoodie/TestHoodieClientBase.java | 2 + .../TestHoodieClientOnCopyOnWriteStorage.java | 39 ++++++ .../com/uber/hoodie/index/TestHbaseIndex.java | 5 +- .../uber/hoodie/io/TestConsistencyCheck.java | 91 ++++++++++++++ .../common/table/HoodieTableMetaClient.java | 14 +++ .../scala/com/uber/hoodie/DefaultSource.scala | 11 +- .../deltastreamer/HoodieDeltaStreamer.java | 12 +- .../utilities/TestHoodieDeltaStreamer.java | 1 + pom.xml | 2 +- 17 files changed, 379 insertions(+), 91 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/io/TestConsistencyCheck.java diff --git a/docs/configurations.md b/docs/configurations.md index 4639fb2e52378..74d3608548a63 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -25,6 +25,8 @@ summary: "Here we list all possible configurations and what they mean" Should HoodieWriteClient autoCommit after insert and upsert. The client can choose to turn off auto-commit and commit on a "defined success condition" - [withAssumeDatePartitioning](#withAssumeDatePartitioning) (assumeDatePartitioning = false)
    Should HoodieWriteClient assume the data is partitioned by dates, i.e three levels from base path. This is a stop-gap to support tables created by versions < 0.3.1. Will be removed eventually + - [withConsistencyCheckEnabled](#withConsistencyCheckEnabled) (enabled = false)
    + Should HoodieWriteClient perform additional checks to ensure written files' are listable on the underlying filesystem/storage. Set this to true, to workaround S3's eventual consistency model and ensure all data written as a part of a commit is faithfully available for queries. - [withIndexConfig](#withIndexConfig) (HoodieIndexConfig)
    Hoodie uses a index to help find the FileID which contains an incoming record key. This is pluggable to have a external index (HBase) or use the default bloom filter stored in the Parquet files diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 0e63ac167273f..147ba56a0f6f4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -497,24 +497,25 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - - List> stats = writeStatuses.mapToPair( - (PairFunction) writeStatus -> new Tuple2<>( - writeStatus.getPartitionPath(), writeStatus.getStat())).collect(); - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - updateMetadataAndRollingStats(actionType, metadata, stats); + List writeStatusList = writeStatuses.collect(); + updateMetadataAndRollingStats(actionType, metadata, writeStatusList); // Finalize write final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); - final Optional result = table.finalizeWrite(jsc, stats); - if (finalizeCtx != null && result.isPresent()) { - Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); - durationInMs.ifPresent(duration -> { - logger.info("Finalize write elapsed time (milliseconds): " + duration); - metrics.updateFinalizeWriteMetrics(duration, result.get()); - }); + try { + table.finalizeWrite(jsc, writeStatusList); + if (finalizeCtx != null) { + Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); + durationInMs.ifPresent(duration -> { + logger.info("Finalize write elapsed time (milliseconds): " + duration); + metrics.updateFinalizeWriteMetrics(duration, writeStatusList.size()); + }); + } + } catch (HoodieIOException ioe) { + throw new HoodieCommitException( + "Failed to complete commit " + commitTime + " due to finalize errors.", ioe); } // add in extra metadata @@ -555,11 +556,11 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, logger.info("Committed " + commitTime); } catch (IOException e) { throw new HoodieCommitException( - "Failed to commit " + config.getBasePath() + " at time " + commitTime, e); + "Failed to complete commit " + config.getBasePath() + " at time " + commitTime, e); } catch (ParseException e) { throw new HoodieCommitException( - "Commit time is not of valid format.Failed to commit " + config.getBasePath() - + " at time " + commitTime, e); + "Failed to complete commit " + config.getBasePath() + " at time " + commitTime + + "Instant time is not of valid format", e); } return true; } @@ -1258,8 +1259,8 @@ private Optional forceCompact(Optional> extraMetadat return compactionInstantTimeOpt; } - private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetadata metadata, List> stats) { + private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetadata metadata, + List writeStatusList) { // TODO : make sure we cannot rollback / archive last commit file try { // Create a Hoodie table which encapsulated the commits and files visible @@ -1272,12 +1273,14 @@ private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetada // 2. Now, first read the existing rolling stats and merge with the result of current metadata. // Need to do this on every commit (delta or commit) to support COW and MOR. - for (Tuple2 stat : stats) { - metadata.addWriteStat(stat._1(), stat._2()); - HoodieRollingStat hoodieRollingStat = new HoodieRollingStat(stat._2().getFileId(), - stat._2().getNumWrites() - (stat._2().getNumUpdateWrites() - stat._2.getNumDeletes()), - stat._2().getNumUpdateWrites(), stat._2.getNumDeletes(), stat._2().getTotalWriteBytes()); - rollingStatMetadata.addRollingStat(stat._1, hoodieRollingStat); + for (WriteStatus status : writeStatusList) { + HoodieWriteStat stat = status.getStat(); + //TODO: why is stat.getPartitionPath() null at times here. + metadata.addWriteStat(status.getPartitionPath(), stat); + HoodieRollingStat hoodieRollingStat = new HoodieRollingStat(stat.getFileId(), + stat.getNumWrites() - (stat.getNumUpdateWrites() - stat.getNumDeletes()), + stat.getNumUpdateWrites(), stat.getNumDeletes(), stat.getTotalWriteBytes()); + rollingStatMetadata.addRollingStat(status.getPartitionPath(), hoodieRollingStat); } // The last rolling stat should be present in the completed timeline Optional lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index 2349860b1aa94..07ddc9673ec21 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -68,6 +68,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "false"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; + private static final String CONSISTENCY_CHECK_ENABLED = "hoodie.consistency.check.enabled"; + private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false"; private HoodieWriteConfig(Properties props) { super(props); @@ -150,6 +152,10 @@ public int getFinalizeWriteParallelism() { return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM)); } + public boolean isConsistencyCheckEnabled() { + return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED)); + } + /** * compaction properties **/ @@ -551,6 +557,11 @@ public Builder withFinalizeWriteParallelism(int parallelism) { return this; } + public Builder withConsistencyCheckEnabled(boolean enabled) { + props.setProperty(CONSISTENCY_CHECK_ENABLED, String.valueOf(enabled)); + return this; + } + public HoodieWriteConfig build() { HoodieWriteConfig config = new HoodieWriteConfig(props); // Check for mandatory properties @@ -581,6 +592,8 @@ public HoodieWriteConfig build() { DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE); setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED), + CONSISTENCY_CHECK_ENABLED, DEFAULT_CONSISTENCY_CHECK_ENABLED); // Make sure the props is propagated setDefaultOnCondition(props, !isIndexConfigSet, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java b/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java new file mode 100644 index 0000000000000..43913089e73d5 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java @@ -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.io; + +import com.google.common.annotations.VisibleForTesting; +import com.uber.hoodie.common.SerializableConfiguration; +import com.uber.hoodie.common.util.FSUtils; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +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; + +/** + * Checks if all the written paths have their metadata consistent on storage and thus be listable to + * queries. This is important for cloud, stores like AWS S3 which are eventually consistent with + * their metadata. Without such checks, we may proceed to commit the written data, without the + * written data being made available to queries. In cases like incremental pull this can lead to + * downstream readers failing to ever see some data. + */ +public class ConsistencyCheck implements Serializable { + + private static final transient Logger log = LogManager.getLogger(ConsistencyCheck.class); + + private String basePath; + + private List relPaths; + + private transient JavaSparkContext jsc; + + private SerializableConfiguration hadoopConf; + + private int parallelism; + + public ConsistencyCheck(String basePath, List relPaths, JavaSparkContext jsc, + int parallelism) { + this.basePath = basePath; + this.relPaths = relPaths; + this.jsc = jsc; + this.hadoopConf = new SerializableConfiguration(jsc.hadoopConfiguration()); + this.parallelism = parallelism; + } + + @VisibleForTesting + void sleepSafe(long waitMs) { + try { + Thread.sleep(waitMs); + } catch (InterruptedException e) { + // ignore & continue next attempt + } + } + + /** + * Repeatedly lists the filesystem on the paths, with exponential backoff and marks paths found as + * passing the check. + * + * @return list of (relative) paths failing the check + */ + public List check(int maxAttempts, long initalDelayMs) { + long waitMs = initalDelayMs; + int attempt = 0; + + List remainingPaths = new ArrayList<>(relPaths); + while (attempt++ < maxAttempts) { + remainingPaths = jsc.parallelize(remainingPaths, parallelism) + .groupBy(p -> new Path(basePath, p).getParent()) // list by partition + .map(pair -> { + FileSystem fs = FSUtils.getFs(basePath, hadoopConf.get()); + // list the partition path and obtain all file paths present + Set fileNames = Arrays.stream(fs.listStatus(pair._1())) + .map(s -> s.getPath().getName()) + .collect(Collectors.toSet()); + + // only return paths that can't be found + return StreamSupport.stream(pair._2().spliterator(), false) + .filter(p -> !fileNames.contains(new Path(basePath, p).getName())) + .collect(Collectors.toList()); + }) + .flatMap(itr -> itr.iterator()).collect(); + if (remainingPaths.size() == 0) { + break; // we are done. + } + + log.info("Consistency check, waiting for " + waitMs + " ms , after attempt :" + attempt); + sleepSafe(waitMs); + waitMs = waitMs * 2; // double check interval every attempt + } + + return remainingPaths; + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index d8a0f484b5459..81efc83d94553 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -29,7 +29,6 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRollingStatMetadata; -import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; @@ -382,44 +381,40 @@ public List rollback(JavaSparkContext jsc, List comm */ @Override @SuppressWarnings("unchecked") - public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { - if (!config.shouldUseTempFolderForCopyOnWrite()) { - return Optional.empty(); - } - - // This is to rename each data file from temporary path to its final location - List> results = jsc - .parallelize(writeStatuses, config.getFinalizeWriteParallelism()).map(writeStatus -> { - Tuple2 writeStatTuple2 = (Tuple2) - writeStatus; - HoodieWriteStat writeStat = writeStatTuple2._2(); - final FileSystem fs = getMetaClient().getFs(); - final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); - - if (writeStat.getTempPath() != null) { - final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); - boolean success; - try { - logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); - success = fs.rename(tempPath, finalPath); - } catch (IOException e) { - throw new HoodieIOException( - "Failed to rename file: " + tempPath + " to " + finalPath); - } + public void finalizeWrite(JavaSparkContext jsc, List writeStatuses) + throws HoodieIOException { + + super.finalizeWrite(jsc, writeStatuses); + + if (config.shouldUseTempFolderForCopyOnWrite()) { + // This is to rename each data file from temporary path to its final location + jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism()) + .map(status -> status.getStat()) + .foreach(writeStat -> { + final FileSystem fs = getMetaClient().getFs(); + final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); + + if (writeStat.getTempPath() != null) { + final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); + boolean success; + try { + logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); + success = fs.rename(tempPath, finalPath); + } catch (IOException e) { + throw new HoodieIOException( + "Failed to rename file: " + tempPath + " to " + finalPath); + } - if (!success) { - throw new HoodieIOException( - "Failed to rename file: " + tempPath + " to " + finalPath); + if (!success) { + throw new HoodieIOException( + "Failed to rename file: " + tempPath + " to " + finalPath); + } } - } + }); - return new Tuple2<>(writeStat.getPath(), true); - }).collect(); - - // clean temporary data files - cleanTemporaryDataFiles(jsc); - - return Optional.of(results.size()); + // clean temporary data files + cleanTemporaryDataFiles(jsc); + } } /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 77e6ddf4fadde..9968d72d40b89 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -43,6 +43,7 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.func.MergeOnReadLazyInsertIterable; @@ -294,9 +295,10 @@ public List rollback(JavaSparkContext jsc, List comm } @Override - public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { - // do nothing for MOR tables - return Optional.empty(); + public void finalizeWrite(JavaSparkContext jsc, List writeStatuses) + throws HoodieIOException { + // delegate to base class for MOR tables + super.finalizeWrite(jsc, writeStatuses); } @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 4ef33f9b1767e..28de32b902ae1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -23,7 +23,6 @@ import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; -import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; @@ -33,26 +32,31 @@ import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieSavepointException; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.io.ConsistencyCheck; import java.io.IOException; import java.io.Serializable; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import scala.Tuple2; /** * Abstract implementation of a HoodieTable */ public abstract class HoodieTable implements Serializable { + // time between successive attempts to ensure written data's metadata is consistent on storage + private static long INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L; + // maximum number of checks, for consistency of written data. Will wait upto 256 Secs + private static int MAX_CONSISTENCY_CHECKS = 7; + protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; protected final HoodieIndex index; @@ -245,11 +249,26 @@ public abstract List rollback(JavaSparkContext jsc, List finalizeWrite(JavaSparkContext jsc, - List> writeStatuses); + public void finalizeWrite(JavaSparkContext jsc, List writeStatuses) + throws HoodieIOException { + if (config.isConsistencyCheckEnabled()) { + List pathsToCheck = writeStatuses.stream() + .map(ws -> ws.getStat().getTempPath() != null + ? ws.getStat().getTempPath() : ws.getStat().getPath()) + .collect(Collectors.toList()); + + List failingPaths = new ConsistencyCheck(config.getBasePath(), pathsToCheck, jsc, + config.getFinalizeWriteParallelism()) + .check(MAX_CONSISTENCY_CHECKS, INITIAL_CONSISTENCY_CHECK_INTERVAL_MS); + if (failingPaths.size() > 0) { + throw new HoodieIOException("Could not verify consistency of paths : " + failingPaths); + } + } + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 2d2417ccaa0d8..03f895b27e09a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -192,6 +192,7 @@ private void testInsertAndCleanByVersions( HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) .retainFileVersions(maxVersions).build()) .withParallelism(1, 1).withBulkInsertParallelism(1) + .withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true) .build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); @@ -271,9 +272,6 @@ private void testInsertAndCleanByVersions( for (HoodieFileGroup fileGroup : fileGroups) { if (selectedFileIdForCompaction.containsKey(fileGroup.getId())) { // Ensure latest file-slice selected for compaction is retained - String oldestCommitRetained = - fileGroup.getAllDataFiles().map(HoodieDataFile::getCommitTime).sorted().findFirst().get(); - Optional dataFileForCompactionPresent = fileGroup.getAllDataFiles().filter(df -> { return compactionFileIdToLatestFileSlice.get(fileGroup.getId()) @@ -357,7 +355,8 @@ private void testInsertAndCleanByCommits( HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build()) - .withParallelism(1, 1).withBulkInsertParallelism(1).build(); + .withParallelism(1, 1).withBulkInsertParallelism(1) + .withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); final Function2, String, Integer> recordInsertGenWrappedFunction = diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java index d33697bafbf0c..282601d4daaa5 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -143,6 +143,8 @@ protected HoodieWriteConfig getConfig() { HoodieWriteConfig.Builder getConfigBuilder() { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2) + .withBulkInsertParallelism(2).withFinalizeWriteParallelism(2) + .withConsistencyCheckEnabled(true) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) .forTable("test-trip-table") diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 684e9ec20d226..427b6b03e6e61 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -40,6 +41,8 @@ import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieCommitException; +import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.table.HoodieTable; import java.io.FileInputStream; @@ -665,6 +668,42 @@ public void testRollingStatsInMetadata() throws Exception { } + /** + * Tests behavior of committing only when consistency is verified + */ + @Test + public void testConsistencyCheckDuringFinalize() throws Exception { + HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + basePath); + + String commitTime = "000"; + client.startCommitWithTime(commitTime); + JavaRDD writeRecords = jsc + .parallelize(dataGen.generateInserts(commitTime, 200), 1); + JavaRDD result = client.bulkInsert(writeRecords, commitTime); + + // move one of the files & commit should fail + WriteStatus status = result.take(1).get(0); + Path origPath = new Path(basePath + "/" + status.getStat().getPath()); + Path hidePath = new Path(basePath + "/" + status.getStat().getPath() + "_hide"); + metaClient.getFs().rename(origPath, hidePath); + + try { + client.commit(commitTime, result); + fail("Commit should fail due to consistency check"); + } catch (HoodieCommitException cme) { + assertTrue(cme.getCause() instanceof HoodieIOException); + } + + // Re-introduce & commit should succeed + metaClient.getFs().rename(hidePath, origPath); + assertTrue("Commit should succeed", client.commit(commitTime, result)); + assertTrue("After explicit commit, commit file should be created", + HoodieTestUtils.doesCommitExist(basePath, commitTime)); + } + /** * Build Hoodie Write Config for small data file sizes */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java index bde2bf6ad1f73..f92466963a82e 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -24,6 +24,7 @@ import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTestUtils; @@ -50,7 +51,6 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.junit.After; @@ -101,8 +101,7 @@ public static void init() throws Exception { hbaseConfig = utility.getConnection().getConfiguration(); utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s")); // Initialize a local spark env - SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]"); - jsc = new JavaSparkContext(sparkConf); + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHbaseIndex")); jsc.hadoopConfiguration().addResource(utility.getConfiguration()); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestConsistencyCheck.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestConsistencyCheck.java new file mode 100644 index 0000000000000..9a1c4d5b372b0 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestConsistencyCheck.java @@ -0,0 +1,91 @@ +/* + * 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.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import com.uber.hoodie.common.HoodieClientTestUtils; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestConsistencyCheck { + + private String basePath; + private JavaSparkContext jsc; + + @Before + public void setup() throws IOException { + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("ConsistencyCheckTest")); + TemporaryFolder testFolder = new TemporaryFolder(); + testFolder.create(); + basePath = testFolder.getRoot().getAbsolutePath(); + } + + @After + public void teardown() { + if (jsc != null) { + jsc.stop(); + } + File testFolderPath = new File(basePath); + if (testFolderPath.exists()) { + testFolderPath.delete(); + } + } + + @Test + public void testExponentialBackoff() throws Exception { + HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); + JavaSparkContext jscSpy = spy(jsc); + + ConsistencyCheck failing = new ConsistencyCheck(basePath, + Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f2_0_000.parquet"), + jscSpy, 2); + long startMs = System.currentTimeMillis(); + assertEquals(1, failing.check(5, 10).size()); + assertTrue((System.currentTimeMillis() - startMs) > (10 + 20 + 40 + 80)); + verify(jscSpy, times(5)).parallelize(anyList(), anyInt()); + } + + @Test + public void testCheckPassingAndFailing() throws Exception { + HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); + HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2"); + HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3"); + + ConsistencyCheck passing = new ConsistencyCheck(basePath, + Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f2_0_000.parquet"), + jsc, 2); + assertEquals(0, passing.check(1, 1000).size()); + + ConsistencyCheck failing = new ConsistencyCheck(basePath, + Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f4_0_000.parquet"), + jsc, 2); + assertEquals(1, failing.check(1, 1000).size()); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 12649c5271dde..a2c99062170ac 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -207,6 +207,20 @@ public synchronized HoodieArchivedTimeline getArchivedTimeline() { return archivedTimeline; } + + /** + * Helper method to initialize a dataset, with given basePath, tableType, name, archiveFolder + */ + public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, + String tableType, String tableName, String archiveLogFolder) throws IOException { + HoodieTableType type = HoodieTableType.valueOf(tableType); + Properties properties = new Properties(); + properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName); + properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, type.name()); + properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder); + return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); + } + /** * Helper method to initialize a given path, as a given storage type and table name */ diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 1a79b352a6d58..68ac868e87e2f 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -19,12 +19,12 @@ package com.uber.hoodie import java.util +import java.util.Optional import java.util.concurrent.ConcurrentHashMap -import java.util.{Optional, Properties} import com.uber.hoodie.DataSourceReadOptions._ import com.uber.hoodie.DataSourceWriteOptions._ -import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import com.uber.hoodie.common.table.HoodieTableMetaClient import com.uber.hoodie.common.util.{FSUtils, TypedProperties} import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException @@ -205,11 +205,8 @@ class DefaultSource extends RelationProvider // Create the dataset if not present (APPEND mode) if (!exists) { - val properties = new Properties(); - properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tblName.get); - properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, storageType); - properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived"); - HoodieTableMetaClient.initializePathAsHoodieDataset(sparkContext.hadoopConfiguration, path.get, properties); + HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType, + tblName.get, "archived") } // Create a HoodieWriteClient & issue the write. diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index 211a171e0e148..835e7fa66a610 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -54,7 +54,6 @@ import java.util.HashMap; import java.util.List; import java.util.Optional; -import java.util.Properties; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FileSystem; @@ -159,11 +158,8 @@ public void sync() throws Exception { } } } else { - Properties properties = new Properties(); - properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName); - HoodieTableMetaClient - .initializePathAsHoodieDataset(jssc.hadoopConfiguration(), cfg.targetBasePath, - properties); + HoodieTableMetaClient.initTableType(jssc.hadoopConfiguration(), cfg.targetBasePath, + cfg.storageType, cfg.targetTableName, "archived"); } log.info("Checkpoint to resume from : " + resumeCheckpointStr); @@ -247,6 +243,10 @@ public static class Config implements Serializable { @Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true) public String targetTableName; + @Parameter(names = {"--storage-type"}, description = "Type of Storage. " + + "COPY_ON_WRITE (or) MERGE_ON_READ", required = true) + public String storageType; + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are " + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer" diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java index d2ff8ace1c89e..1a17a687c00b9 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java @@ -89,6 +89,7 @@ static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op) { HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config(); cfg.targetBasePath = basePath; cfg.targetTableName = "hoodie_trips"; + cfg.storageType = "COPY_ON_WRITE"; cfg.sourceClassName = TestDataSource.class.getName(); cfg.operation = op; cfg.sourceOrderingField = "timestamp"; diff --git a/pom.xml b/pom.xml index c86adfa11026f..62bfd162b604e 100644 --- a/pom.xml +++ b/pom.xml @@ -625,7 +625,7 @@ junit junit - 4.12 + ${junit.version} org.apache.hadoop From 8791bc0e49f52eade5db21329fd7f766cac46948 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Fri, 28 Sep 2018 11:03:08 +0530 Subject: [PATCH 109/374] Update RELEASE_NOTES for release 0.4.4 --- RELEASE_NOTES.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index b7bc5c3c01d29..c95724796044e 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -5,12 +5,20 @@ Release 0.4.4 * Dependencies are now decoupled from CDH and based on apache versions! * Support for Hive 2 is here!! Use -Dhive11 to build for older hive versions * Deltastreamer tool reworked to make configs simpler, hardended tests, added Confluent Kafka support + * Provide strong consistency for S3 datasets * Removed dependency on commons lang3, to ease use with different hadoop/spark versions * Better CLI support and docs for managing async compactions * New CLI commands to manage datasets ### Full PR List + * **@vinothchandar** - Perform consistency checks during write finalize #464 + * **@bvaradar** - Travis CI tests needs to be run in quieter mode (WARN log level) to avoid max log-size errors #465 + * **@lys0716** - Fix the name of avro schema file in Test #467 + * **@bvaradar** - Hive Sync handling must work for datasets with multi-partition keys #460 + * **@bvaradar** - Explicitly release resources in LogFileReader and TestHoodieClientBase. Fixes Memory allocation errors #463 + * **@bvaradar** - [Release Blocking] Ensure packaging modules create sources/javadoc jars #461 + * **@vinothchandar** - Fix bug with incrementally pulling older data #458 * **@saravsars** - Updated jcommander version to fix NPE in HoodieDeltaStreamer tool #443 * **@n3nash** - Removing dependency on apache-commons lang 3, adding necessary classes as needed #444 * **@n3nash** - Small file size handling for inserts into log files. #413 From fba6debc7c0ce9961f102e13b724cd79d7ab0717 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Fri, 28 Sep 2018 11:26:15 +0530 Subject: [PATCH 110/374] [maven-release-plugin] prepare release hoodie-0.4.4 --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 4 ++-- packaging/hoodie-hive-bundle/pom.xml | 4 ++-- packaging/hoodie-spark-bundle/pom.xml | 4 ++-- pom.xml | 4 ++-- 11 files changed, 15 insertions(+), 15 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 515ea1d7f186f..91e99552bcb77 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index b8a316db89525..ee7a570ba9cb0 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index df28b11007faf..72c27c03c52b8 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 3424009cc4b7f..3c7bba57e741f 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index e9cff4750cc08..c4b9ebfb18bdc 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 55d373d6da83e..856db0d4fa14e 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index de5018989761a..91bab7b10f765 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 121f72fbcd3be..aec55df727433 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 ../../pom.xml 4.0.0 @@ -192,7 +192,7 @@ - + diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index c55666d1febb2..09082f2c56121 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 ../../pom.xml 4.0.0 @@ -205,7 +205,7 @@ - + diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 75bb572ac844d..c5e5d0407d138 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4-SNAPSHOT + 0.4.4 ../../pom.xml 4.0.0 @@ -279,7 +279,7 @@ - + diff --git a/pom.xml b/pom.xml index 62bfd162b604e..e68dbf4fd1979 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.4-SNAPSHOT + 0.4.4 Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - HEAD + hoodie-0.4.4
    From a8f21d2e2cdaadb0e192075c3b9636ae488877ce Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Fri, 28 Sep 2018 11:27:00 +0530 Subject: [PATCH 111/374] [maven-release-plugin] prepare for next development iteration --- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 2 +- packaging/hoodie-hive-bundle/pom.xml | 2 +- packaging/hoodie-spark-bundle/pom.xml | 2 +- pom.xml | 4 ++-- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 91e99552bcb77..2920f81ccc899 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index ee7a570ba9cb0..eccfa802c58b8 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 72c27c03c52b8..e56d35279da68 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 3c7bba57e741f..fe1df4396a740 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index c4b9ebfb18bdc..f74e5ea8350b0 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 856db0d4fa14e..15fb3ae930b96 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 91bab7b10f765..8a9feac7f31ef 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index aec55df727433..79a4a757a4ac4 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index 09082f2c56121..eafad25922921 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index c5e5d0407d138..95b4aedd42e54 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.4 + 0.4.5-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index e68dbf4fd1979..ce05e8cd41013 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.4 + 0.4.5-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -133,7 +133,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - hoodie-0.4.4 + HEAD From edecc32cc1e74337d821cdfa5ed6e57f6bc8bd47 Mon Sep 17 00:00:00 2001 From: "jiale.tan" Date: Mon, 13 Aug 2018 17:35:21 -0700 Subject: [PATCH 112/374] feature(HoodieGlobalBloomIndex): adds a new type of bloom index to allow global record key lookup --- .../com/uber/hoodie/index/HoodieIndex.java | 5 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 9 +- .../index/bloom/HoodieGlobalBloomIndex.java | 110 +++++++ .../hoodie/table/HoodieCopyOnWriteTable.java | 2 +- .../hoodie/common/HoodieClientTestUtils.java | 76 ++++- .../index/bloom/TestHoodieBloomIndex.java | 98 ++---- .../bloom/TestHoodieGlobalBloomIndex.java | 308 ++++++++++++++++++ .../hoodie/common/model/HoodieTestUtils.java | 4 + 8 files changed, 531 insertions(+), 81 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieGlobalBloomIndex.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java index bc7dc7eb48a4c..17897846dc192 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java @@ -24,6 +24,7 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.bloom.HoodieBloomIndex; +import com.uber.hoodie.index.bloom.HoodieGlobalBloomIndex; import com.uber.hoodie.index.bucketed.BucketedIndex; import com.uber.hoodie.index.hbase.HBaseIndex; import com.uber.hoodie.table.HoodieTable; @@ -53,6 +54,8 @@ public static HoodieIndex createIndex(HoodieW return new InMemoryHashIndex<>(config); case BLOOM: return new HoodieBloomIndex<>(config); + case GLOBAL_BLOOM: + return new HoodieGlobalBloomIndex<>(config); case BUCKETED: return new BucketedIndex<>(config); default: @@ -116,6 +119,6 @@ public abstract JavaRDD updateLocation(JavaRDD writeSt public enum IndexType { - HBASE, INMEMORY, BLOOM, BUCKETED + HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, BUCKETED } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index b0f37e8ac3ce5..050958159035a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -303,7 +303,7 @@ public boolean isImplicitWithStorage() { * if we dont have key ranges, then also we need to compare against the file. no other choice if * we do, then only compare the file if the record key falls in range. */ - private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) { + boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) { return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey); } @@ -313,10 +313,11 @@ private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recor * record's key needs to be checked. For datasets, where the keys have a definite insert order * (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range * pruning. + * + * Sub-partition to ensure the records can be looked up against files & also prune + * file<=>record comparisons based on recordKey + * ranges in the index info. */ - // sub-partition to ensure the records can be looked up against files & also prune - // file<=>record comparisons based on recordKey - // ranges in the index info. @VisibleForTesting JavaPairRDD> explodeRecordRDDWithFileComparisons( final Map> partitionToFileIndexInfo, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java new file mode 100644 index 0000000000000..49c5fd3fc1053 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java @@ -0,0 +1,110 @@ +/* + * 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.index.bloom; + +import com.google.common.annotations.VisibleForTesting; +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.table.HoodieTable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; + +/** + * This filter will only work with hoodie dataset since it will only load partitions + * with .hoodie_partition_metadata file in it. + */ +public class HoodieGlobalBloomIndex extends HoodieBloomIndex { + + public HoodieGlobalBloomIndex(HoodieWriteConfig config) { + super(config); + } + + /** + * Load all involved files as pair RDD from all partitions in the table. + */ + @Override + @VisibleForTesting + List> loadInvolvedFiles(List partitions, final JavaSparkContext jsc, + final HoodieTable hoodieTable) { + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + try { + List allPartitionPaths = FSUtils + .getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), + config.shouldAssumeDatePartitioning()); + return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable); + } catch (IOException e) { + throw new HoodieIOException("Failed to load all partitions", e); + } + } + + /** + * For each incoming record, produce N output records, 1 each for each file against which the + * record's key needs to be checked. For datasets, where the keys have a definite insert order + * (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range + * pruning. + * + * Sub-partition to ensure the records can be looked up against files & also prune + * file<=>record comparisons based on recordKey + * ranges in the index info. + * the partition path of the incoming record (partitionRecordKeyPairRDD._2()) will be ignored + * since the search scope should be bigger than that + */ + @Override + @VisibleForTesting + JavaPairRDD> explodeRecordRDDWithFileComparisons( + final Map> partitionToFileIndexInfo, + JavaPairRDD partitionRecordKeyPairRDD) { + List> indexInfos = + partitionToFileIndexInfo.entrySet().stream() + .flatMap(e1 -> e1.getValue().stream() + .map(e2 -> new Tuple2<>(e1.getKey(), e2))) + .collect(Collectors.toList()); + + return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> { + String recordKey = partitionRecordKeyPair._2(); + + List>> recordComparisons = new ArrayList<>(); + if (indexInfos != null) { // could be null, if there are no files in a given partition yet. + // for each candidate file in partition, that needs to be compared. + for (Tuple2 indexInfo : indexInfos) { + if (shouldCompareWithFile(indexInfo._2(), recordKey)) { + recordComparisons.add( + new Tuple2<>(String.format("%s#%s", indexInfo._2().getFileName(), recordKey), + new Tuple2<>(indexInfo._2().getFileName(), + new HoodieKey(recordKey, indexInfo._1())))); + } + } + } + return recordComparisons; + }).flatMapToPair(t -> t.iterator()); + } + +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 81efc83d94553..3fa8a175ac9ad 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -195,7 +195,7 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups "Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId); } else { AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); - ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) + ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) .withConf(getHadoopConf()).build(); BoundedInMemoryExecutor wrapper = null; try { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index cc1dad8e18a34..54ea125b14520 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -18,29 +18,36 @@ import com.uber.hoodie.HoodieReadClient; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.io.storage.HoodieParquetConfig; +import com.uber.hoodie.io.storage.HoodieParquetWriter; + import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; @@ -174,4 +181,59 @@ public static Dataset read(JavaSparkContext jsc, String basePath, SQLContex throw new HoodieException("Error reading hoodie dataset as a dataframe", e); } } + + public static String writeParquetFile(String basePath, + String partitionPath, + String filename, + List records, + Schema schema, + BloomFilter filter, + boolean createCommitTime) throws IOException { + + if (filter == null) { + filter = new BloomFilter(10000, 0.0000001); + } + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, + filter); + String commitTime = FSUtils.getCommitTime(filename); + HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, + ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, + HoodieTestUtils.getDefaultHadoopConf(), + Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO)); + HoodieParquetWriter writer = new HoodieParquetWriter( + commitTime, + new Path(basePath + "/" + partitionPath + "/" + filename), + config, + schema); + int seqId = 1; + for (HoodieRecord record : records) { + GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++); + HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename); + writer.writeAvro(record.getRecordKey(), avroRecord); + filter.add(record.getRecordKey()); + } + writer.close(); + + if (createCommitTime) { + HoodieTestUtils.createMetadataFolder(basePath); + HoodieTestUtils.createCommitFiles(basePath, commitTime); + } + return filename; + } + + public static String writeParquetFile(String basePath, + String partitionPath, + List records, + Schema schema, + BloomFilter filter, + boolean createCommitTime) throws IOException, InterruptedException { + Thread.sleep(1000); + String commitTime = HoodieTestUtils.makeNewCommitTime(); + String fileId = UUID.randomUUID().toString(); + String filename = FSUtils.makeDataFileName(commitTime, 1, fileId); + HoodieTestUtils.createCommitFiles(basePath, commitTime); + return HoodieClientTestUtils + .writeParquetFile(basePath, partitionPath, filename, records, schema, filter, createCommitTime); + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 82a4c1a29f855..5f602d7f7c88b 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -27,7 +27,6 @@ import com.google.common.base.Optional; import com.google.common.collect.Lists; -import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.TestRawTripPayload; @@ -37,29 +36,19 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; -import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.io.storage.HoodieParquetConfig; -import com.uber.hoodie.io.storage.HoodieParquetWriter; import com.uber.hoodie.table.HoodieTable; import java.io.File; import java.io.IOException; -import java.text.SimpleDateFormat; import java.util.Arrays; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.stream.Collectors; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.parquet.avro.AvroSchemaConverter; -import org.apache.parquet.hadoop.ParquetWriter; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -173,10 +162,18 @@ public void testLoadInvolvedFiles() throws IOException { HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); - writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, false); - writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, false); - writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema, null, false); - writeParquetFile("2015/03/12", "4_0_20150312101010.parquet", Arrays.asList(record2, record3, record4), schema, null, + HoodieClientTestUtils + .writeParquetFile(basePath, "2016/04/01", "2_0_20160401010101.parquet", + Lists.newArrayList(), schema, null, false); + HoodieClientTestUtils + .writeParquetFile(basePath, "2015/03/12", "1_0_20150312101010.parquet", + Lists.newArrayList(), schema, null, false); + HoodieClientTestUtils + .writeParquetFile(basePath, "2015/03/12", "3_0_20150312101010.parquet", + Arrays.asList(record1), schema, null, false); + HoodieClientTestUtils + .writeParquetFile(basePath, "2015/03/12", "4_0_20150312101010.parquet", + Arrays.asList(record2, record3, record4), schema, null, false); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); @@ -270,7 +267,9 @@ public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedExcept // record2, record3). BloomFilter filter = new BloomFilter(10000, 0.0000001); filter.add(record3.getRecordKey()); - String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, filter, true); + String filename = HoodieClientTestUtils + .writeParquetFile(basePath, "2016/01/31", + Arrays.asList(record1, record2), schema, filter, true); // The bloom filter contains 3 records assertTrue(filter.mightContain(record1.getRecordKey())); @@ -355,9 +354,12 @@ public void testTagLocation() throws Exception { } // We create three parquet file, each having one record. (two different partitions) - String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true); - String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true); - String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true); + String filename1 = + HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1), schema, null, true); + String filename2 = + HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record2), schema, null, true); + String filename3 = + HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Arrays.asList(record4), schema, null, true); // We do the tag again metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -420,9 +422,12 @@ public void testCheckExists() throws Exception { } // We create three parquet file, each having one record. (two different partitions) - String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true); - String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true); - String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true); + String filename1 = + HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1), schema, null, true); + String filename2 = + HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record2), schema, null, true); + String filename3 = + HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Arrays.asList(record4), schema, null, true); // We do the tag again metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -468,7 +473,9 @@ public void testBloomFilterFalseError() throws IOException, InterruptedException BloomFilter filter = new BloomFilter(10000, 0.0000001); filter.add(record2.getRecordKey()); - String filename = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, filter, true); + String filename = HoodieClientTestUtils + .writeParquetFile(basePath, "2016/01/31", + Arrays.asList(record1), schema, filter, true); assertTrue(filter.mightContain(record1.getRecordKey())); assertTrue(filter.mightContain(record2.getRecordKey())); @@ -491,49 +498,4 @@ public void testBloomFilterFalseError() throws IOException, InterruptedException } } - private String writeParquetFile(String partitionPath, List records, Schema schema, BloomFilter filter, - boolean createCommitTime) throws IOException, InterruptedException { - Thread.sleep(1000); - String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); - String fileId = UUID.randomUUID().toString(); - String filename = FSUtils.makeDataFileName(commitTime, 1, fileId); - - return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime); - } - - private String writeParquetFile(String partitionPath, String filename, List records, Schema schema, - BloomFilter filter, boolean createCommitTime) throws IOException { - - if (filter == null) { - filter = new BloomFilter(10000, 0.0000001); - } - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, - filter); - String commitTime = FSUtils.getCommitTime(filename); - HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, - ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, - HoodieTestUtils.getDefaultHadoopConf(), - Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO)); - HoodieParquetWriter writer = new HoodieParquetWriter( - commitTime, - new Path(basePath + "/" + partitionPath + "/" + filename), - config, - schema); - int seqId = 1; - for (HoodieRecord record : records) { - GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); - HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++); - HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename); - writer.writeAvro(record.getRecordKey(), avroRecord); - filter.add(record.getRecordKey()); - } - writer.close(); - - if (createCommitTime) { - // Also make sure the commit is valid - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs(); - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile(); - } - return filename; - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieGlobalBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieGlobalBloomIndex.java new file mode 100644 index 0000000000000..8f222b37f3e1d --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieGlobalBloomIndex.java @@ -0,0 +1,308 @@ +/* + * 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.index.bloom; + +import static org.junit.Assert.*; + +import com.google.common.collect.Lists; +import com.uber.hoodie.common.HoodieClientTestUtils; +import com.uber.hoodie.common.TestRawTripPayload; +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.model.HoodiePartitionMetadata; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.table.HoodieTable; + +import java.io.File; +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; + +import org.apache.avro.Schema; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import scala.Tuple2; + +public class TestHoodieGlobalBloomIndex { + + private JavaSparkContext jsc = null; + private String basePath = null; + private transient FileSystem fs; + private String schemaStr; + private Schema schema; + + public TestHoodieGlobalBloomIndex() throws Exception { + } + + @Before + public void init() throws IOException { + // Initialize a local spark env + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieGlobalBloomIndex")); + // Create a temp folder as the base path + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + basePath = folder.getRoot().getAbsolutePath(); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + // We have some records to be tagged (two different partitions) + schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); + schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); + } + + @After + public void clean() { + if (basePath != null) { + new File(basePath).delete(); + } + if (jsc != null) { + jsc.stop(); + } + } + + @Test + public void testLoadInvolvedFiles() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); + HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); + + // Create some partitions, and put some files, along with the meta file + // "2016/01/21": 0 file + // "2016/04/01": 1 file (2_0_20160401010101.parquet) + // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, + // 4_0_20150312101010.parquet) + new File(basePath + "/2016/01/21").mkdirs(); + new File(basePath + "/2016/01/21/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile(); + new File(basePath + "/2016/04/01").mkdirs(); + new File(basePath + "/2016/04/01/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile(); + new File(basePath + "/2015/03/12").mkdirs(); + new File(basePath + "/2015/03/12/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile(); + + TestRawTripPayload rowChange1 = new TestRawTripPayload( + "{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); + TestRawTripPayload rowChange2 = new TestRawTripPayload( + "{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); + TestRawTripPayload rowChange3 = new TestRawTripPayload( + "{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); + TestRawTripPayload rowChange4 = new TestRawTripPayload( + "{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); + + HoodieClientTestUtils + .writeParquetFile(basePath, "2016/04/01", "2_0_20160401010101.parquet", + Lists.newArrayList(), schema, null, false); + HoodieClientTestUtils + .writeParquetFile(basePath, "2015/03/12", "1_0_20150312101010.parquet", + Lists.newArrayList(), schema, null, false); + HoodieClientTestUtils + .writeParquetFile(basePath, "2015/03/12", "3_0_20150312101010.parquet", + Arrays.asList(record1), schema, null, false); + HoodieClientTestUtils + .writeParquetFile(basePath, "2015/03/12", "4_0_20150312101010.parquet", + Arrays.asList(record2, record3, record4), schema, null, false); + + // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up + List partitions = Arrays.asList("2016/01/21", "2016/04/01"); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); + // partitions will NOT be respected by this loadInvolvedFiles(...) call + List> filesList = index.loadInvolvedFiles(partitions, jsc, table); + // Still 0, as no valid commit + assertEquals(filesList.size(), 0); + + // Add some commits + new File(basePath + "/.hoodie").mkdirs(); + new File(basePath + "/.hoodie/20160401010101.commit").createNewFile(); + new File(basePath + "/.hoodie/20150312101010.commit").createNewFile(); + + table = HoodieTable.getHoodieTable(metadata, config, jsc); + filesList = index.loadInvolvedFiles(partitions, jsc, table); + assertEquals(filesList.size(), 4); + + Map filesMap = toFileMap(filesList); + // key ranges checks + assertNull(filesMap.get("2016/04/01/2_0_20160401010101.parquet").getMaxRecordKey()); + assertNull(filesMap.get("2016/04/01/2_0_20160401010101.parquet").getMinRecordKey()); + assertFalse(filesMap.get("2015/03/12/1_0_20150312101010.parquet").hasKeyRanges()); + assertNotNull(filesMap.get("2015/03/12/3_0_20150312101010.parquet").getMaxRecordKey()); + assertNotNull(filesMap.get("2015/03/12/3_0_20150312101010.parquet").getMinRecordKey()); + assertTrue(filesMap.get("2015/03/12/3_0_20150312101010.parquet").hasKeyRanges()); + + Map expected = new HashMap<>(); + expected.put("2016/04/01/2_0_20160401010101.parquet", new BloomIndexFileInfo("2_0_20160401010101.parquet")); + expected.put("2015/03/12/1_0_20150312101010.parquet", new BloomIndexFileInfo("1_0_20150312101010.parquet")); + expected.put("2015/03/12/3_0_20150312101010.parquet", + new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")); + expected.put("2015/03/12/4_0_20150312101010.parquet", + new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003")); + + assertEquals(expected, filesMap); + } + + @Test + public void testExplodeRecordRDDWithFileComparisons() { + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); + HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); + + final Map> partitionToFileIndexInfo = new HashMap<>(); + partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), + new BloomIndexFileInfo("f2", "000", "000"), new BloomIndexFileInfo("f3", "001", "003"))); + + partitionToFileIndexInfo.put("2017/10/23", Arrays.asList( + new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010"))); + + // the partition partition of the key of the incoming records will be ignored + JavaPairRDD partitionRecordKeyPairRDD = jsc.parallelize(Arrays.asList( + new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), + new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t); + + List>> comparisonKeyList = index.explodeRecordRDDWithFileComparisons( + partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect(); + + /* epecting: + f4#003, f4, HoodieKey { recordKey=003 partitionPath=2017/10/23} + f1#003, f1, HoodieKey { recordKey=003 partitionPath=2017/10/22} + f3#003, f3, HoodieKey { recordKey=003 partitionPath=2017/10/22} + f4#002, f4, HoodieKey { recordKey=002 partitionPath=2017/10/23} + f1#002, f1, HoodieKey { recordKey=002 partitionPath=2017/10/22} + f3#002, f3, HoodieKey { recordKey=002 partitionPath=2017/10/22} + f4#005, f4, HoodieKey { recordKey=005 partitionPath=2017/10/23} + f1#005, f1, HoodieKey { recordKey=005 partitionPath=2017/10/22} + f4#004, f4, HoodieKey { recordKey=004 partitionPath=2017/10/23} + f1#004, f1, HoodieKey { recordKey=004 partitionPath=2017/10/22} + */ + assertEquals(10, comparisonKeyList.size()); + + Map> recordKeyToFileComps = comparisonKeyList.stream().collect(Collectors.groupingBy( + t -> t._2()._2().getRecordKey(), Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()))); + + assertEquals(4, recordKeyToFileComps.size()); + assertEquals(Arrays.asList("f4", "f1", "f3"), recordKeyToFileComps.get("002")); + assertEquals(Arrays.asList("f4", "f1", "f3"), recordKeyToFileComps.get("003")); + assertEquals(Arrays.asList("f4", "f1"), recordKeyToFileComps.get("004")); + assertEquals(Arrays.asList("f4", "f1"), recordKeyToFileComps.get("005")); + } + + + @Test + public void testTagLocation() throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); + HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config); + + // Create some partitions, and put some files, along with the meta file + // "2016/01/21": 0 file + // "2016/04/01": 1 file (2_0_20160401010101.parquet) + // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, + // 4_0_20150312101010.parquet) + new File(basePath + "/2016/01/21").mkdirs(); + new File(basePath + "/2016/01/21/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile(); + new File(basePath + "/2016/04/01").mkdirs(); + new File(basePath + "/2016/04/01/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile(); + new File(basePath + "/2015/03/12").mkdirs(); + new File(basePath + "/2015/03/12/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile(); + + TestRawTripPayload rowChange1 = new TestRawTripPayload( + "{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); + TestRawTripPayload rowChange2 = new TestRawTripPayload( + "{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); + TestRawTripPayload rowChange3 = new TestRawTripPayload( + "{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); + + // this record will be saved in table and will be tagged to the incoming record5 + TestRawTripPayload rowChange4 = new TestRawTripPayload( + "{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); + + // this has the same record key as record4 but different time so different partition, but globalbloomIndex should + // tag the original partition of the saved record4 + TestRawTripPayload rowChange5 = new TestRawTripPayload( + "{\"_row_key\":\"003\",\"time\":\"2016-02-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record5 = new HoodieRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), + rowChange4); + + JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5)); + + String filename0 = + HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Arrays.asList(record1), schema, null, false); + String filename1 = + HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Lists.newArrayList(), schema, null, false); + String filename2 = + HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Arrays.asList(record2), schema, null, false); + String filename3 = + HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Arrays.asList(record4), schema, null, false); + + // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); + + + // Add some commits + new File(basePath + "/.hoodie").mkdirs(); + + // partitions will NOT be respected by this loadInvolvedFiles(...) call + JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, jsc, table); + + for (HoodieRecord record : taggedRecordRDD.collect()) { + if (record.getRecordKey().equals("000")) { + assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename0))); + } else if (record.getRecordKey().equals("001")) { + assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2))); + } else if (record.getRecordKey().equals("002")) { + assertTrue(!record.isCurrentLocationKnown()); + } else if (record.getRecordKey().equals("004")) { + assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3))); + } + } + } + + // convert list to map to avoid sorting order dependencies + private Map toFileMap(List> filesList) { + Map filesMap = new HashMap<>(); + for (Tuple2 t : filesList) { + filesMap.put(t._1() + "/" + t._2().getFileName(), t._2()); + } + return filesMap; + } + +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 62877bd2c0c89..eda4bc77c4a02 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -126,6 +126,10 @@ public static final void createDeltaCommitFiles(String basePath, String... commi } } + public static final void createMetadataFolder(String basePath) throws IOException { + new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs(); + } + public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName( From 0debd65da2069064786c6a1ea8708fb79c42d895 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Sat, 29 Sep 2018 20:14:35 +0530 Subject: [PATCH 113/374] Adding Jiale & Anbu to contributors list --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index ce05e8cd41013..05501b8ab202c 100644 --- a/pom.xml +++ b/pom.xml @@ -101,6 +101,14 @@ Balaji Varadharajan Uber + + Anbu Cheeralan + DoubleVerify + + + Jiale Tan + Vungle + 2015-2016 From 8c6e492aff76e29841c2f68259bfa4f7ef174884 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Sat, 29 Sep 2018 20:50:03 +0530 Subject: [PATCH 114/374] Update Gemfile.lock with newer jekyll version --- docs/Gemfile.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index 8fac90615c295..65eae02f0a366 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -25,7 +25,7 @@ GEM github-pages (106) activesupport (= 4.2.7) github-pages-health-check (= 1.2.0) - jekyll (= 3.3.1) + jekyll (= 3.6.3) jekyll-avatar (= 0.4.2) jekyll-coffeescript (= 1.0.1) jekyll-feed (= 0.8.0) @@ -58,7 +58,7 @@ GEM nokogiri (>= 1.4) i18n (0.9.1) concurrent-ruby (~> 1.0) - jekyll (3.3.1) + jekyll (3.6.3) addressable (~> 2.4) colorator (~> 1.0) jekyll-sass-converter (~> 1.0) @@ -149,7 +149,7 @@ PLATFORMS DEPENDENCIES github-pages (~> 106) - jekyll (= 3.3.1) + jekyll (= 3.6.3) jekyll-feed (~> 0.6) BUNDLED WITH From 3983ef9a57ea390cf52bfc4868e9812954ebc6a2 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Fri, 28 Sep 2018 21:41:28 -0700 Subject: [PATCH 115/374] Ensure Hoodie metadata folder and files are filtered out when constructing Parquet Data Source --- .../uber/hoodie/common/model/HoodieTestUtils.java | 10 ++++++++++ .../hoodie/hadoop/HoodieROTablePathFilter.java | 10 ++++++++++ .../hadoop/TestHoodieROTablePathFilter.java | 15 +++++++++++++++ 3 files changed, 35 insertions(+) diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index eda4bc77c4a02..9049d81a4f185 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -219,6 +219,16 @@ public static final String getCommitFilePath(String basePath, String commitTime) return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION; } + public static final String getInflightCommitFilePath(String basePath, String commitTime) throws IOException { + return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION; + } + + public static final String getRequestedCompactionFilePath(String basePath, String commitTime) throws IOException { + return basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + commitTime + + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION; + } + public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID) throws IOException { return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists(); diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index ae55ea21b9ce1..f408cb9456bb3 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -111,6 +111,16 @@ public boolean accept(Path path) { return hoodiePathCache.get(folder.toString()).contains(path); } + // Skip all files that are descendants of .hoodie in its path. + String filePath = path.toString(); + if (filePath.contains("/" + HoodieTableMetaClient.METAFOLDER_NAME + "/") + || filePath.endsWith("/" + HoodieTableMetaClient.METAFOLDER_NAME)) { + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Skipping Hoodie Metadata file %s \n", filePath)); + } + return false; + } + // Perform actual checking. Path baseDir; if (HoodiePartitionMetadata.hasPartitionMetadata(fs, folder)) { diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java index e139b737177dc..041e56db97a1c 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java @@ -23,6 +23,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import org.apache.hadoop.fs.Path; import org.junit.Before; import org.junit.Rule; @@ -51,6 +52,7 @@ public void testHoodiePaths() throws IOException { HoodieTestUtils.createCommitFiles(basePath, "001", "002"); HoodieTestUtils.createInflightCommitFiles(basePath, "003"); + HoodieTestUtils.createCompactionRequest(metaClient, "004", new ArrayList<>()); HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f1"); HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f2"); @@ -72,6 +74,19 @@ public void testHoodiePaths() throws IOException { "file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "002", "f2")))); assertFalse(pathFilter.accept(new Path( "file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3")))); + assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "001")))); + assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "002")))); + assertFalse(pathFilter.accept(new Path("file:///" + + HoodieTestUtils.getInflightCommitFilePath(basePath, "003")))); + assertFalse(pathFilter.accept(new Path("file:///" + + HoodieTestUtils.getRequestedCompactionFilePath(basePath, "004")))); + assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + + HoodieTableMetaClient.METAFOLDER_NAME + "/"))); + assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME))); + + assertFalse(pathFilter.accept(new Path( + "file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3")))); + } @Test From 8ac24d0848c2f1db34f8839d85eadd4f5b380e50 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 21 Aug 2018 22:54:57 -0700 Subject: [PATCH 116/374] Docker Container Build and Run setup with foundations for adding docker integration tests. Docker images built with Hadoop 2.8.4 Hive 2.3.3 and Spark 2.3.1 and published to docker-hub Look at quickstart document for how to setup docker and run demo --- docker/build_local_docker_images.sh | 13 + ...ker-compose_hadoop284_hive233_spark231.yml | 217 + docker/compose/hadoop.env | 33 + docker/demo/config/base.properties | 21 + docker/demo/config/kafka-source.properties | 29 + docker/demo/config/schema.avsc | 41 + docker/demo/config/spark-defaults.conf | 26 + docker/demo/data/batch_1.json | 3482 +++++++++++++++++ docker/demo/data/batch_2.json | 1668 ++++++++ docker/demo/setup_demo_container.sh | 6 + docker/hoodie/hadoop/base/Dockerfile | 45 + docker/hoodie/hadoop/base/entrypoint.sh | 91 + .../hoodie/hadoop/base/export_container_ip.sh | 13 + docker/hoodie/hadoop/base/pom.xml | 90 + docker/hoodie/hadoop/datanode/Dockerfile | 14 + docker/hoodie/hadoop/datanode/pom.xml | 89 + docker/hoodie/hadoop/datanode/run_dn.sh | 9 + docker/hoodie/hadoop/historyserver/Dockerfile | 14 + docker/hoodie/hadoop/historyserver/pom.xml | 89 + .../hadoop/historyserver/run_history.sh | 3 + docker/hoodie/hadoop/hive_base/Dockerfile | 51 + .../hive_base/conf/beeline-log4j2.properties | 45 + .../hoodie/hadoop/hive_base/conf/hive-env.sh | 54 + .../conf/hive-exec-log4j2.properties | 66 + .../hive_base/conf/hive-log4j2.properties | 73 + .../hadoop/hive_base/conf/hive-site.xml | 18 + .../hadoop/hive_base/conf/ivysettings.xml | 45 + .../conf/llap-daemon-log4j2.properties | 93 + docker/hoodie/hadoop/hive_base/entrypoint.sh | 118 + docker/hoodie/hadoop/hive_base/pom.xml | 113 + docker/hoodie/hadoop/hive_base/startup.sh | 10 + docker/hoodie/hadoop/namenode/Dockerfile | 14 + docker/hoodie/hadoop/namenode/pom.xml | 89 + docker/hoodie/hadoop/namenode/run_nn.sh | 19 + docker/hoodie/hadoop/pom.xml | 78 + docker/hoodie/hadoop/spark_base/Dockerfile | 46 + .../hoodie/hadoop/spark_base/execute-step.sh | 14 + .../hoodie/hadoop/spark_base/finish-step.sh | 16 + docker/hoodie/hadoop/spark_base/pom.xml | 89 + .../hoodie/hadoop/spark_base/wait-for-step.sh | 13 + docker/hoodie/hadoop/sparkadhoc/Dockerfile | 12 + docker/hoodie/hadoop/sparkadhoc/adhoc.sh | 13 + docker/hoodie/hadoop/sparkadhoc/pom.xml | 89 + docker/hoodie/hadoop/sparkmaster/Dockerfile | 14 + docker/hoodie/hadoop/sparkmaster/master.sh | 16 + docker/hoodie/hadoop/sparkmaster/pom.xml | 89 + docker/hoodie/hadoop/sparkworker/Dockerfile | 14 + docker/hoodie/hadoop/sparkworker/pom.xml | 89 + docker/hoodie/hadoop/sparkworker/worker.sh | 16 + docker/setup_demo.sh | 16 + docs/quickstart.md | 945 ++++- hoodie-hive/run_sync_tool.sh | 0 .../hive}/MultiPartKeysValueExtractor.java | 3 +- .../uber/hoodie/hive/HiveSyncToolTest.java | 1 - hoodie-integ-test/pom.xml | 212 + .../com/uber/hoodie/integ/ITTestBase.java | 178 + .../uber/hoodie/integ/ITTestHoodieSanity.java | 139 + .../test/resources/log4j-surefire.properties | 23 + hoodie-spark/pom.xml | 15 + hoodie-spark/run_hoodie_app.sh | 2 +- .../com/uber/hoodie/DataSourceOptions.scala | 2 +- hoodie-spark/src/test/java/HoodieJavaApp.java | 14 +- pom.xml | 2 + 63 files changed, 8952 insertions(+), 9 deletions(-) create mode 100755 docker/build_local_docker_images.sh create mode 100644 docker/compose/docker-compose_hadoop284_hive233_spark231.yml create mode 100644 docker/compose/hadoop.env create mode 100644 docker/demo/config/base.properties create mode 100644 docker/demo/config/kafka-source.properties create mode 100644 docker/demo/config/schema.avsc create mode 100644 docker/demo/config/spark-defaults.conf create mode 100644 docker/demo/data/batch_1.json create mode 100644 docker/demo/data/batch_2.json create mode 100755 docker/demo/setup_demo_container.sh create mode 100644 docker/hoodie/hadoop/base/Dockerfile create mode 100644 docker/hoodie/hadoop/base/entrypoint.sh create mode 100755 docker/hoodie/hadoop/base/export_container_ip.sh create mode 100644 docker/hoodie/hadoop/base/pom.xml create mode 100644 docker/hoodie/hadoop/datanode/Dockerfile create mode 100644 docker/hoodie/hadoop/datanode/pom.xml create mode 100644 docker/hoodie/hadoop/datanode/run_dn.sh create mode 100644 docker/hoodie/hadoop/historyserver/Dockerfile create mode 100644 docker/hoodie/hadoop/historyserver/pom.xml create mode 100644 docker/hoodie/hadoop/historyserver/run_history.sh create mode 100644 docker/hoodie/hadoop/hive_base/Dockerfile create mode 100644 docker/hoodie/hadoop/hive_base/conf/beeline-log4j2.properties create mode 100644 docker/hoodie/hadoop/hive_base/conf/hive-env.sh create mode 100644 docker/hoodie/hadoop/hive_base/conf/hive-exec-log4j2.properties create mode 100644 docker/hoodie/hadoop/hive_base/conf/hive-log4j2.properties create mode 100644 docker/hoodie/hadoop/hive_base/conf/hive-site.xml create mode 100644 docker/hoodie/hadoop/hive_base/conf/ivysettings.xml create mode 100644 docker/hoodie/hadoop/hive_base/conf/llap-daemon-log4j2.properties create mode 100644 docker/hoodie/hadoop/hive_base/entrypoint.sh create mode 100644 docker/hoodie/hadoop/hive_base/pom.xml create mode 100644 docker/hoodie/hadoop/hive_base/startup.sh create mode 100644 docker/hoodie/hadoop/namenode/Dockerfile create mode 100644 docker/hoodie/hadoop/namenode/pom.xml create mode 100644 docker/hoodie/hadoop/namenode/run_nn.sh create mode 100644 docker/hoodie/hadoop/pom.xml create mode 100644 docker/hoodie/hadoop/spark_base/Dockerfile create mode 100644 docker/hoodie/hadoop/spark_base/execute-step.sh create mode 100644 docker/hoodie/hadoop/spark_base/finish-step.sh create mode 100644 docker/hoodie/hadoop/spark_base/pom.xml create mode 100644 docker/hoodie/hadoop/spark_base/wait-for-step.sh create mode 100644 docker/hoodie/hadoop/sparkadhoc/Dockerfile create mode 100644 docker/hoodie/hadoop/sparkadhoc/adhoc.sh create mode 100644 docker/hoodie/hadoop/sparkadhoc/pom.xml create mode 100644 docker/hoodie/hadoop/sparkmaster/Dockerfile create mode 100644 docker/hoodie/hadoop/sparkmaster/master.sh create mode 100644 docker/hoodie/hadoop/sparkmaster/pom.xml create mode 100644 docker/hoodie/hadoop/sparkworker/Dockerfile create mode 100644 docker/hoodie/hadoop/sparkworker/pom.xml create mode 100644 docker/hoodie/hadoop/sparkworker/worker.sh create mode 100755 docker/setup_demo.sh mode change 100644 => 100755 hoodie-hive/run_sync_tool.sh rename hoodie-hive/src/{test/java/com/uber/hoodie/hive/util => main/java/com/uber/hoodie/hive}/MultiPartKeysValueExtractor.java (92%) create mode 100644 hoodie-integ-test/pom.xml create mode 100644 hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestBase.java create mode 100644 hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java create mode 100644 hoodie-integ-test/src/test/resources/log4j-surefire.properties mode change 100644 => 100755 hoodie-spark/run_hoodie_app.sh diff --git a/docker/build_local_docker_images.sh b/docker/build_local_docker_images.sh new file mode 100755 index 0000000000000..bb8d33b9abcf4 --- /dev/null +++ b/docker/build_local_docker_images.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +while true; do + read -p "Docker images can be downloaded from docker hub and seamlessly mounted with latest HUDI jars. Do you still want to build docker images from scratch ?" yn + case $yn in + [Yy]* ) make install; break;; + [Nn]* ) exit;; + * ) echo "Please answer yes or no.";; + esac +done +pushd ../ +mvn clean pre-integration-test -DskipTests -Ddocker.compose.skip=true -Ddocker.build.skip=false +popd diff --git a/docker/compose/docker-compose_hadoop284_hive233_spark231.yml b/docker/compose/docker-compose_hadoop284_hive233_spark231.yml new file mode 100644 index 0000000000000..6eb874f0099d3 --- /dev/null +++ b/docker/compose/docker-compose_hadoop284_hive233_spark231.yml @@ -0,0 +1,217 @@ +version: "3.3" + +services: + + namenode: + image: varadarb/hudi-hadoop_2.8.4-namenode:latest + hostname: namenode + container_name: namenode + volumes: + - /tmp/hadoop_name:/hadoop/dfs/name + environment: + - CLUSTER_NAME=hudi_hadoop284_hive232_spark231 + ports: + - "50070:50070" + - "8020:8020" + env_file: + - ./hadoop.env + healthcheck: + test: ["CMD", "curl", "-f", "http://namenode:50070"] + interval: 30s + timeout: 10s + retries: 3 + + datanode1: + image: varadarb/hudi-hadoop_2.8.4-datanode:latest + container_name: datanode1 + hostname: datanode1 + environment: + - CLUSTER_NAME=hudi_hadoop284_hive232_spark231 + env_file: + - ./hadoop.env + ports: + - "50075:50075" + - "50010:50010" + links: + - "namenode" + - "historyserver" + healthcheck: + test: ["CMD", "curl", "-f", "http://datanode1:50075"] + interval: 30s + timeout: 10s + retries: 3 + depends_on: + - namenode + volumes: + - /tmp/hadoop_data:/hadoop/dfs/data + + historyserver: + image: varadarb/hudi-hadoop_2.8.4-history:latest + hostname: historyserver + container_name: historyserver + environment: + - CLUSTER_NAME=hudi_hadoop284_hive232_spark231 + depends_on: + - "namenode" + links: + - "namenode" + ports: + - "58188:8188" + healthcheck: + test: ["CMD", "curl", "-f", "http://historyserver:8188"] + interval: 30s + timeout: 10s + retries: 3 + env_file: + - ./hadoop.env + volumes: + - historyserver:/hadoop/yarn/timeline + + hive-metastore-postgresql: + image: bde2020/hive-metastore-postgresql:2.3.0 + volumes: + - hive-metastore-postgresql:/var/lib/postgresql + hostname: hive-metastore-postgresql + container_name: hive-metastore-postgresql + + hivemetastore: + image: varadarb/hudi-hadoop_2.8.4-hive_2.3.3:latest + hostname: hivemetastore + container_name: hivemetastore + links: + - "hive-metastore-postgresql" + - "namenode" + env_file: + - ./hadoop.env + command: /opt/hive/bin/hive --service metastore + environment: + SERVICE_PRECONDITION: "namenode:50070 hive-metastore-postgresql:5432" + ports: + - "9083:9083" + healthcheck: + test: ["CMD", "nc", "-z", "hivemetastore", "9083"] + interval: 30s + timeout: 10s + retries: 3 + depends_on: + - "hive-metastore-postgresql" + - "namenode" + + hiveserver: + image: varadarb/hudi-hadoop_2.8.4-hive_2.3.3:latest + hostname: hiveserver + container_name: hiveserver + env_file: + - ./hadoop.env + environment: + SERVICE_PRECONDITION: "hivemetastore:9083" + ports: + - "10000:10000" + depends_on: + - "hivemetastore" + links: + - "hivemetastore" + - "hive-metastore-postgresql" + - "namenode" + volumes: + - ${HUDI_WS}:/var/hoodie/ws + + sparkmaster: + image: varadarb/hudi-hadoop_2.8.4-hive_2.3.3-sparkmaster_2.3.1:latest + hostname: sparkmaster + container_name: sparkmaster + env_file: + - ./hadoop.env + ports: + - "8080:8080" + - "7077:7077" + environment: + - INIT_DAEMON_STEP=setup_spark + links: + - "hivemetastore" + - "hiveserver" + - "hive-metastore-postgresql" + - "namenode" + + spark-worker-1: + image: varadarb/hudi-hadoop_2.8.4-hive_2.3.3-sparkworker_2.3.1:latest + hostname: spark-worker-1 + container_name: spark-worker-1 + env_file: + - ./hadoop.env + depends_on: + - sparkmaster + ports: + - "8081:8081" + environment: + - "SPARK_MASTER=spark://sparkmaster:7077" + links: + - "hivemetastore" + - "hiveserver" + - "hive-metastore-postgresql" + - "namenode" + + zookeeper: + image: 'bitnami/zookeeper:3.4.12-r68' + hostname: zookeeper + container_name: zookeeper + ports: + - '2181:2181' + environment: + - ALLOW_ANONYMOUS_LOGIN=yes + + kafka: + image: 'bitnami/kafka:2.0.0' + hostname: kafkabroker + container_name: kafkabroker + ports: + - '9092:9092' + environment: + - KAFKA_ZOOKEEPER_CONNECT=zookeeper:2181 + - ALLOW_PLAINTEXT_LISTENER=yes + + adhoc-1: + image: varadarb/hudi-hadoop_2.8.4-hive_2.3.3-sparkadhoc_2.3.1:latest + hostname: adhoc-1 + container_name: adhoc-1 + env_file: + - ./hadoop.env + depends_on: + - sparkmaster + ports: + - '4040:4040' + environment: + - "SPARK_MASTER=spark://sparkmaster:7077" + links: + - "hivemetastore" + - "hiveserver" + - "hive-metastore-postgresql" + - "namenode" + volumes: + - ${HUDI_WS}:/var/hoodie/ws + + adhoc-2: + image: varadarb/hudi-hadoop_2.8.4-hive_2.3.3-sparkadhoc_2.3.1:latest + hostname: adhoc-2 + container_name: adhoc-2 + env_file: + - ./hadoop.env + depends_on: + - sparkmaster + environment: + - "SPARK_MASTER=spark://sparkmaster:7077" + links: + - "hivemetastore" + - "hiveserver" + - "hive-metastore-postgresql" + - "namenode" + volumes: + - ${HUDI_WS}:/var/hoodie/ws + +volumes: + namenode: + historyserver: + hive-metastore-postgresql: + +networks: + default: diff --git a/docker/compose/hadoop.env b/docker/compose/hadoop.env new file mode 100644 index 0000000000000..5cad0a5bede67 --- /dev/null +++ b/docker/compose/hadoop.env @@ -0,0 +1,33 @@ +HIVE_SITE_CONF_javax_jdo_option_ConnectionURL=jdbc:postgresql://hive-metastore-postgresql/metastore +HIVE_SITE_CONF_javax_jdo_option_ConnectionDriverName=org.postgresql.Driver +HIVE_SITE_CONF_javax_jdo_option_ConnectionUserName=hive +HIVE_SITE_CONF_javax_jdo_option_ConnectionPassword=hive +HIVE_SITE_CONF_datanucleus_autoCreateSchema=false +HIVE_SITE_CONF_hive_metastore_uris=thrift://hivemetastore:9083 +HDFS_CONF_dfs_namenode_datanode_registration_ip___hostname___check=false + +HDFS_CONF_dfs_webhdfs_enabled=true +HDFS_CONF_dfs_permissions_enabled=false +#HDFS_CONF_dfs_client_use_datanode_hostname=true +#HDFS_CONF_dfs_namenode_use_datanode_hostname=true + +CORE_CONF_fs_defaultFS=hdfs://namenode:8020 +CORE_CONF_hadoop_http_staticuser_user=root +CORE_CONF_hadoop_proxyuser_hue_hosts=* +CORE_CONF_hadoop_proxyuser_hue_groups=* + +YARN_CONF_yarn_log___aggregation___enable=true +YARN_CONF_yarn_resourcemanager_recovery_enabled=true +YARN_CONF_yarn_resourcemanager_store_class=org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore +YARN_CONF_yarn_resourcemanager_fs_state___store_uri=/rmstate +YARN_CONF_yarn_nodemanager_remote___app___log___dir=/app-logs +YARN_CONF_yarn_log_server_url=http://historyserver:8188/applicationhistory/logs/ +YARN_CONF_yarn_timeline___service_enabled=true +YARN_CONF_yarn_timeline___service_generic___application___history_enabled=true +YARN_CONF_yarn_resourcemanager_system___metrics___publisher_enabled=true +YARN_CONF_yarn_resourcemanager_hostname=resourcemanager +YARN_CONF_yarn_timeline___service_hostname=historyserver +YARN_CONF_yarn_resourcemanager_address=resourcemanager:8032 +YARN_CONF_yarn_resourcemanager_scheduler_address=resourcemanager:8030 +YARN_CONF_yarn_resourcemanager_resource___tracker_address=resourcemanager:8031 +YARN_CONF_yarn_nodemanager_vmem___check___enabled=false diff --git a/docker/demo/config/base.properties b/docker/demo/config/base.properties new file mode 100644 index 0000000000000..9075810065aa6 --- /dev/null +++ b/docker/demo/config/base.properties @@ -0,0 +1,21 @@ +# +# 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. +# +# +# +# Common hoodie client configs +hoodie.upsert.shuffle.parallelism=2 +hoodie.insert.shuffle.parallelism=2 +hoodie.bulkinsert.shuffle.parallelism=2 diff --git a/docker/demo/config/kafka-source.properties b/docker/demo/config/kafka-source.properties new file mode 100644 index 0000000000000..1588c47f914cc --- /dev/null +++ b/docker/demo/config/kafka-source.properties @@ -0,0 +1,29 @@ +# +# 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. +# +# +# +include=base.properties +# Key fields, for kafka example +hoodie.datasource.write.recordkey.field=key +hoodie.datasource.write.partitionpath.field=date +# Schema provider props (change to absolute path based on your installation) +hoodie.deltastreamer.schemaprovider.source.schema.file=/var/demo/config/schema.avsc +hoodie.deltastreamer.schemaprovider.target.schema.file=/var/demo/config/schema.avsc +# Kafka Source +hoodie.deltastreamer.source.kafka.topic=stock_ticks +#Kafka props +metadata.broker.list=kafkabroker:9092 +auto.offset.reset=smallest diff --git a/docker/demo/config/schema.avsc b/docker/demo/config/schema.avsc new file mode 100644 index 0000000000000..dd0779a5faf7c --- /dev/null +++ b/docker/demo/config/schema.avsc @@ -0,0 +1,41 @@ +{ + "type":"record", + "name":"stock_ticks", + "fields":[{ + "name": "volume", + "type": "long" + }, { + "name": "ts", + "type": "string" + }, { + "name": "symbol", + "type": "string" + },{ + "name": "year", + "type": "int" + },{ + "name": "month", + "type": "string" + },{ + "name": "high", + "type": "double" + },{ + "name": "low", + "type": "double" + },{ + "name": "key", + "type": "string" + },{ + "name": "date", + "type":"string" + }, { + "name": "close", + "type": "double" + }, { + "name": "open", + "type": "double" + }, { + "name": "day", + "type":"string" + } +]} diff --git a/docker/demo/config/spark-defaults.conf b/docker/demo/config/spark-defaults.conf new file mode 100644 index 0000000000000..e496b46c28de1 --- /dev/null +++ b/docker/demo/config/spark-defaults.conf @@ -0,0 +1,26 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You 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. +# + +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +spark.master local[3] +spark.eventLog.dir hdfs://namenode:8020/tmp/spark-events +spark.serializer org.apache.spark.serializer.KryoSerializer +#spark.executor.memory 4g +# spark.executor.extraJavaOptions -XX:+PrintGCDetails -Dkey=value -Dnumbers="one two three" diff --git a/docker/demo/data/batch_1.json b/docker/demo/data/batch_1.json new file mode 100644 index 0000000000000..3e7b149a03b23 --- /dev/null +++ b/docker/demo/data/batch_1.json @@ -0,0 +1,3482 @@ +{"volume": 483951, "symbol": "MSFT", "ts": "2018-08-31 09:30:00", "month": "08", "high": 111.74, "low": 111.55, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 111.72, "open": 111.55, "day": "31"} +{"volume": 1533226, "symbol": "AAPL", "ts": "2018-08-31 09:30:00", "month": "08", "high": 227.3101, "low": 226.23, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.3101, "open": 226.53, "day": "31"} +{"volume": 36179, "symbol": "GOOG", "ts": "2018-08-31 09:30:00", "month": "08", "high": 1236.0, "low": 1234.36, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1234.54, "open": 1236.0, "day": "31"} +{"volume": 456506, "symbol": "FB", "ts": "2018-08-31 09:30:00", "month": "08", "high": 177.5, "low": 176.465, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.83, "open": 177.29, "day": "31"} +{"volume": 142747, "symbol": "NFLX", "ts": "2018-08-31 09:30:00", "month": "08", "high": 372.0, "low": 370.49, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 371.9, "open": 370.49, "day": "31"} +{"volume": 126884, "symbol": "TSLA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 301.81, "low": 300.11, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 300.61, "open": 301.81, "day": "31"} +{"volume": 1201915, "symbol": "F", "ts": "2018-08-31 09:30:00", "month": "08", "high": 9.63, "low": 9.6, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.61, "open": 9.63, "day": "31"} +{"volume": 176474, "symbol": "AMZN", "ts": "2018-08-31 09:30:00", "month": "08", "high": 2010.8101, "low": 2007.0, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2010.5, "open": 2009.8199, "day": "31"} +{"volume": 142523, "symbol": "NVDA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 277.1899, "low": 276.64, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.1899, "open": 276.875, "day": "31"} +{"volume": 351118, "symbol": "INTC", "ts": "2018-08-31 09:30:00", "month": "08", "high": 48.06, "low": 47.96, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.03, "open": 47.961, "day": "31"} +{"volume": 96028, "symbol": "TNDM", "ts": "2018-08-31 09:30:00", "month": "08", "high": 44.709, "low": 44.49, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.709, "open": 44.49, "day": "31"} +{"volume": 0, "symbol": "CDNA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 23.01, "low": 23.01, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.01, "open": 23.01, "day": "31"} +{"volume": 2964, "symbol": "IIN", "ts": "2018-08-31 09:30:00", "month": "08", "high": 72.75, "low": 72.75, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 72.75, "open": 72.75, "day": "31"} +{"volume": 0, "symbol": "TPNL", "ts": "2018-08-31 09:30:00", "month": "08", "high": 3.37, "low": 3.37, "key": "TPNL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 558465, "symbol": "CGC", "ts": "2018-08-31 09:30:00", "month": "08", "high": 44.58, "low": 44.2876, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.29, "open": 44.5, "day": "31"} +{"volume": 8629, "symbol": "FNKO", "ts": "2018-08-31 09:30:00", "month": "08", "high": 24.1888, "low": 24.1888, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.1888, "open": 24.1888, "day": "31"} +{"volume": 0, "symbol": "SSTI", "ts": "2018-08-31 09:30:00", "month": "08", "high": 55.09, "low": 55.09, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 55.09, "open": 55.09, "day": "31"} +{"volume": 0, "symbol": "MRTX", "ts": "2018-08-31 09:30:00", "month": "08", "high": 56.75, "low": 56.75, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.75, "open": 56.75, "day": "31"} +{"volume": 980, "symbol": "RFIL", "ts": "2018-08-31 09:30:00", "month": "08", "high": 11.6, "low": 11.6, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.6, "open": 11.6, "day": "31"} +{"volume": 0, "symbol": "MED", "ts": "2018-08-31 09:30:00", "month": "08", "high": 225.04, "low": 225.04, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 225.04, "open": 225.04, "day": "31"} +{"volume": 0, "symbol": "I", "ts": "2018-08-31 09:30:00", "month": "08", "high": 21.56, "low": 21.56, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.56, "open": 21.56, "day": "31"} +{"volume": 30054, "symbol": "ARQL", "ts": "2018-08-31 09:30:00", "month": "08", "high": 6.515, "low": 6.515, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.515, "open": 6.515, "day": "31"} +{"volume": 87608, "symbol": "DNR", "ts": "2018-08-31 09:30:00", "month": "08", "high": 5.59, "low": 5.59, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.59, "open": 5.59, "day": "31"} +{"volume": 0, "symbol": "CVNA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 61.7, "low": 61.7, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.7, "open": 61.7, "day": "31"} +{"volume": 0, "symbol": "BOOT", "ts": "2018-08-31 09:30:00", "month": "08", "high": 29.33, "low": 29.33, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.33, "open": 29.33, "day": "31"} +{"volume": 148009, "symbol": "SQ", "ts": "2018-08-31 09:30:00", "month": "08", "high": 87.9, "low": 87.9, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.9, "open": 87.9, "day": "31"} +{"volume": 0, "symbol": "RCKT", "ts": "2018-08-31 09:30:00", "month": "08", "high": 23.76, "low": 23.76, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.76, "open": 23.76, "day": "31"} +{"volume": 0, "symbol": "NGVC", "ts": "2018-08-31 09:30:00", "month": "08", "high": 18.87, "low": 18.87, "key": "NGVC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.87, "open": 18.87, "day": "31"} +{"volume": 0, "symbol": "VNCE", "ts": "2018-08-31 09:30:00", "month": "08", "high": 20.7, "low": 20.7, "key": "VNCE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.7, "open": 20.7, "day": "31"} +{"volume": 20143, "symbol": "CRC", "ts": "2018-08-31 09:30:00", "month": "08", "high": 40.1301, "low": 40.1301, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.1301, "open": 40.1301, "day": "31"} +{"volume": 36394, "symbol": "VKTX", "ts": "2018-08-31 09:30:00", "month": "08", "high": 12.9423, "low": 12.9, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.9423, "open": 12.9, "day": "31"} +{"volume": 0, "symbol": "CDXS", "ts": "2018-08-31 09:30:00", "month": "08", "high": 17.2, "low": 17.2, "key": "CDXS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.2, "day": "31"} +{"volume": 2948, "symbol": "LFVN", "ts": "2018-08-31 09:30:00", "month": "08", "high": 11.83, "low": 11.83, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.83, "open": 11.83, "day": "31"} +{"volume": 0, "symbol": "SRDX", "ts": "2018-08-31 09:30:00", "month": "08", "high": 76.6, "low": 76.6, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.6, "open": 76.6, "day": "31"} +{"volume": 0, "symbol": "SDPI", "ts": "2018-08-31 09:30:00", "month": "08", "high": 1.95, "low": 1.94, "key": "SDPI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1.94, "open": 1.95, "day": "31"} +{"volume": 0, "symbol": "MDB", "ts": "2018-08-31 09:30:00", "month": "08", "high": 70.01, "low": 70.01, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.01, "open": 70.01, "day": "31"} +{"volume": 0, "symbol": "LGCY", "ts": "2018-08-31 09:30:00", "month": "08", "high": 5.38, "low": 5.38, "key": "LGCY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.38, "open": 5.38, "day": "31"} +{"volume": 0, "symbol": "IRMD", "ts": "2018-08-31 09:30:00", "month": "08", "high": 26.95, "low": 26.95, "key": "IRMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 26.95, "open": 26.95, "day": "31"} +{"volume": 0, "symbol": "NSP", "ts": "2018-08-31 09:30:00", "month": "08", "high": 118.75, "low": 118.75, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.75, "open": 118.75, "day": "31"} +{"volume": 3703314, "symbol": "CRON", "ts": "2018-08-31 09:30:00", "month": "08", "high": 8.82, "low": 8.5475, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.6199, "open": 8.82, "day": "31"} +{"volume": 10430, "symbol": "SRPT", "ts": "2018-08-31 09:30:00", "month": "08", "high": 138.45, "low": 138.45, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.45, "open": 138.45, "day": "31"} +{"volume": 35616, "symbol": "HEAR", "ts": "2018-08-31 09:30:00", "month": "08", "high": 22.8, "low": 22.8, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.8, "open": 22.8, "day": "31"} +{"volume": 5748, "symbol": "BLFS", "ts": "2018-08-31 09:30:00", "month": "08", "high": 24.33, "low": 24.24, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.33, "open": 24.24, "day": "31"} +{"volume": 0, "symbol": "LGCYO", "ts": "2018-08-31 09:30:00", "month": "08", "high": 15.3, "low": 15.3, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.3, "open": 15.3, "day": "31"} +{"volume": 56045, "symbol": "NEPT", "ts": "2018-08-31 09:30:00", "month": "08", "high": 3.88, "low": 3.88, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.88, "open": 3.88, "day": "31"} +{"volume": 0, "symbol": "INGN", "ts": "2018-08-31 09:30:00", "month": "08", "high": 260.09, "low": 260.09, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.09, "open": 260.09, "day": "31"} +{"volume": 0, "symbol": "GFN", "ts": "2018-08-31 09:30:00", "month": "08", "high": 13.155, "low": 13.155, "key": "GFN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.155, "open": 13.155, "day": "31"} +{"volume": 0, "symbol": "ECYT", "ts": "2018-08-31 09:30:00", "month": "08", "high": 19.4999, "low": 19.4999, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.4999, "open": 19.4999, "day": "31"} +{"volume": 1462419, "symbol": "AMD", "ts": "2018-08-31 09:30:00", "month": "08", "high": 24.939, "low": 24.78, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.939, "open": 24.85, "day": "31"} +{"volume": 21202, "symbol": "SFIX", "ts": "2018-08-31 09:30:00", "month": "08", "high": 38.66, "low": 38.66, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 38.66, "open": 38.66, "day": "31"} +{"volume": 0, "symbol": "DRNA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 15.67, "low": 15.67, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.67, "open": 15.67, "day": "31"} +{"volume": 0, "symbol": "RFP", "ts": "2018-08-31 09:30:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 0, "symbol": "NOG", "ts": "2018-08-31 09:30:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 0, "symbol": "PTSI", "ts": "2018-08-31 09:30:00", "month": "08", "high": 57.41, "low": 57.41, "key": "PTSI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.41, "open": 57.41, "day": "31"} +{"volume": 46864, "symbol": "AMRS", "ts": "2018-08-31 09:30:00", "month": "08", "high": 8.5252, "low": 8.47, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.5252, "open": 8.47, "day": "31"} +{"volume": 0, "symbol": "COUP", "ts": "2018-08-31 09:30:00", "month": "08", "high": 70.5208, "low": 70.5208, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.5208, "open": 70.5208, "day": "31"} +{"volume": 5039, "symbol": "AAXN", "ts": "2018-08-31 09:30:00", "month": "08", "high": 66.24, "low": 66.24, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.24, "open": 66.24, "day": "31"} +{"volume": 0, "symbol": "EVBG", "ts": "2018-08-31 09:30:00", "month": "08", "high": 59.7, "low": 59.7, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.7, "open": 59.7, "day": "31"} +{"volume": 0, "symbol": "AMED", "ts": "2018-08-31 09:30:00", "month": "08", "high": 120.6, "low": 120.3449, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 120.3449, "open": 120.6, "day": "31"} +{"volume": 0, "symbol": "HAE", "ts": "2018-08-31 09:30:00", "month": "08", "high": 109.16, "low": 109.16, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.16, "open": 109.16, "day": "31"} +{"volume": 19288, "symbol": "NVTA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 14.15, "low": 14.15, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.15, "open": 14.15, "day": "31"} +{"volume": 0, "symbol": "NTRA", "ts": "2018-08-31 09:30:00", "month": "08", "high": 27.328, "low": 27.328, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.328, "open": 27.328, "day": "31"} +{"volume": 6067, "symbol": "RCM", "ts": "2018-08-31 09:30:00", "month": "08", "high": 9.82, "low": 9.82, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.82, "open": 9.82, "day": "31"} +{"volume": 705294, "symbol": "LULU", "ts": "2018-08-31 09:30:00", "month": "08", "high": 154.69, "low": 153.1, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 154.214, "open": 153.27, "day": "31"} +{"volume": 11664, "symbol": "GRUB", "ts": "2018-08-31 09:30:00", "month": "08", "high": 141.8, "low": 141.8, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 141.8, "open": 141.8, "day": "31"} +{"volume": 5217, "symbol": "DXCM", "ts": "2018-08-31 09:30:00", "month": "08", "high": 144.0681, "low": 144.0681, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.0681, "open": 144.0681, "day": "31"} +{"volume": 0, "symbol": "QURE", "ts": "2018-08-31 09:30:00", "month": "08", "high": 42.26, "low": 42.26, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.26, "open": 42.26, "day": "31"} +{"volume": 96435, "symbol": "CRM", "ts": "2018-08-31 09:30:00", "month": "08", "high": 152.16, "low": 152.0297, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.16, "open": 152.0297, "day": "31"} +{"volume": 71958, "symbol": "MSFT", "ts": "2018-08-31 09:31:00", "month": "08", "high": 111.77, "low": 111.69, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 111.69, "open": 111.735, "day": "31"} +{"volume": 50261, "symbol": "MSFT", "ts": "2018-08-31 09:32:00", "month": "08", "high": 111.8308, "low": 111.71, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 111.8308, "open": 111.7187, "day": "31"} +{"volume": 91222, "symbol": "MSFT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 111.87, "low": 111.8, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 111.8471, "open": 111.86, "day": "31"} +{"volume": 117574, "symbol": "MSFT", "ts": "2018-08-31 09:34:00", "month": "08", "high": 111.905, "low": 111.86, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 111.89, "open": 111.86, "day": "31"} +{"volume": 132109, "symbol": "MSFT", "ts": "2018-08-31 09:35:00", "month": "08", "high": 111.96, "low": 111.88, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 111.95, "open": 111.9, "day": "31"} +{"volume": 122795, "symbol": "MSFT", "ts": "2018-08-31 09:36:00", "month": "08", "high": 112.13, "low": 111.96, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.13, "open": 111.99, "day": "31"} +{"volume": 53886, "symbol": "MSFT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 112.145, "low": 112.1, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.1355, "open": 112.14, "day": "31"} +{"volume": 68846, "symbol": "MSFT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 112.21, "low": 112.115, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.21, "open": 112.16, "day": "31"} +{"volume": 107240, "symbol": "MSFT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 112.34, "low": 112.2, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.34, "open": 112.209, "day": "31"} +{"volume": 66134, "symbol": "MSFT", "ts": "2018-08-31 09:40:00", "month": "08", "high": 112.27, "low": 112.1832, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.24, "open": 112.27, "day": "31"} +{"volume": 74243, "symbol": "MSFT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 112.31, "low": 112.22, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.22, "open": 112.265, "day": "31"} +{"volume": 39501, "symbol": "MSFT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 112.245, "low": 112.2, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.2019, "open": 112.23, "day": "31"} +{"volume": 48929, "symbol": "MSFT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 112.23, "low": 112.19, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.23, "open": 112.21, "day": "31"} +{"volume": 44053, "symbol": "MSFT", "ts": "2018-08-31 09:44:00", "month": "08", "high": 112.23, "low": 112.18, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.18, "open": 112.23, "day": "31"} +{"volume": 96939, "symbol": "MSFT", "ts": "2018-08-31 09:45:00", "month": "08", "high": 112.24, "low": 112.17, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.24, "open": 112.215, "day": "31"} +{"volume": 51376, "symbol": "MSFT", "ts": "2018-08-31 09:46:00", "month": "08", "high": 112.31, "low": 112.27, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.27, "open": 112.29, "day": "31"} +{"volume": 56744, "symbol": "MSFT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 112.28, "low": 112.23, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.28, "open": 112.25, "day": "31"} +{"volume": 67629, "symbol": "MSFT", "ts": "2018-08-31 09:48:00", "month": "08", "high": 112.29, "low": 112.2, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.22, "open": 112.285, "day": "31"} +{"volume": 62570, "symbol": "MSFT", "ts": "2018-08-31 09:49:00", "month": "08", "high": 112.28, "low": 112.24, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.28, "open": 112.24, "day": "31"} +{"volume": 47739, "symbol": "MSFT", "ts": "2018-08-31 09:50:00", "month": "08", "high": 112.31, "low": 112.246, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.246, "open": 112.31, "day": "31"} +{"volume": 55457, "symbol": "MSFT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 112.29, "low": 112.24, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.27, "open": 112.241, "day": "31"} +{"volume": 44712, "symbol": "MSFT", "ts": "2018-08-31 09:52:00", "month": "08", "high": 112.32, "low": 112.26, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.32, "open": 112.27, "day": "31"} +{"volume": 53053, "symbol": "MSFT", "ts": "2018-08-31 09:53:00", "month": "08", "high": 112.39, "low": 112.33, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.39, "open": 112.33, "day": "31"} +{"volume": 32749, "symbol": "MSFT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 112.39, "low": 112.335, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.335, "open": 112.39, "day": "31"} +{"volume": 50325, "symbol": "MSFT", "ts": "2018-08-31 09:55:00", "month": "08", "high": 112.34, "low": 112.32, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.32, "open": 112.33, "day": "31"} +{"volume": 38704, "symbol": "MSFT", "ts": "2018-08-31 09:56:00", "month": "08", "high": 112.35, "low": 112.32, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.32, "open": 112.34, "day": "31"} +{"volume": 37943, "symbol": "MSFT", "ts": "2018-08-31 09:57:00", "month": "08", "high": 112.33, "low": 112.31, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.32, "open": 112.315, "day": "31"} +{"volume": 52541, "symbol": "MSFT", "ts": "2018-08-31 09:58:00", "month": "08", "high": 112.32, "low": 112.2413, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.2413, "open": 112.32, "day": "31"} +{"volume": 53411, "symbol": "MSFT", "ts": "2018-08-31 09:59:00", "month": "08", "high": 112.2208, "low": 112.2, "key": "MSFT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 112.216, "open": 112.2208, "day": "31"} +{"volume": 233341, "symbol": "AAPL", "ts": "2018-08-31 09:31:00", "month": "08", "high": 227.17, "low": 226.78, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.09, "open": 227.17, "day": "31"} +{"volume": 268272, "symbol": "AAPL", "ts": "2018-08-31 09:32:00", "month": "08", "high": 227.46, "low": 227.1299, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.46, "open": 227.17, "day": "31"} +{"volume": 354532, "symbol": "AAPL", "ts": "2018-08-31 09:33:00", "month": "08", "high": 227.98, "low": 227.47, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.9, "open": 227.47, "day": "31"} +{"volume": 302337, "symbol": "AAPL", "ts": "2018-08-31 09:34:00", "month": "08", "high": 227.93, "low": 227.53, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.71, "open": 227.8899, "day": "31"} +{"volume": 224749, "symbol": "AAPL", "ts": "2018-08-31 09:35:00", "month": "08", "high": 227.71, "low": 227.35, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.39, "open": 227.71, "day": "31"} +{"volume": 287257, "symbol": "AAPL", "ts": "2018-08-31 09:36:00", "month": "08", "high": 227.4, "low": 226.78, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 226.8, "open": 227.37, "day": "31"} +{"volume": 233269, "symbol": "AAPL", "ts": "2018-08-31 09:37:00", "month": "08", "high": 227.16, "low": 226.81, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.1, "open": 226.81, "day": "31"} +{"volume": 208470, "symbol": "AAPL", "ts": "2018-08-31 09:38:00", "month": "08", "high": 227.215, "low": 227.109, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.15, "open": 227.11, "day": "31"} +{"volume": 161512, "symbol": "AAPL", "ts": "2018-08-31 09:39:00", "month": "08", "high": 227.2, "low": 227.09, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.1, "open": 227.12, "day": "31"} +{"volume": 204875, "symbol": "AAPL", "ts": "2018-08-31 09:40:00", "month": "08", "high": 227.42, "low": 227.08, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.32, "open": 227.09, "day": "31"} +{"volume": 159326, "symbol": "AAPL", "ts": "2018-08-31 09:41:00", "month": "08", "high": 227.64, "low": 227.34, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.545, "open": 227.34, "day": "31"} +{"volume": 138022, "symbol": "AAPL", "ts": "2018-08-31 09:42:00", "month": "08", "high": 227.72, "low": 227.52, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.72, "open": 227.52, "day": "31"} +{"volume": 177351, "symbol": "AAPL", "ts": "2018-08-31 09:43:00", "month": "08", "high": 227.84, "low": 227.67, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.7899, "open": 227.715, "day": "31"} +{"volume": 179427, "symbol": "AAPL", "ts": "2018-08-31 09:44:00", "month": "08", "high": 227.78, "low": 227.4, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.41, "open": 227.76, "day": "31"} +{"volume": 149903, "symbol": "AAPL", "ts": "2018-08-31 09:45:00", "month": "08", "high": 227.64, "low": 227.3677, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.64, "open": 227.42, "day": "31"} +{"volume": 108546, "symbol": "AAPL", "ts": "2018-08-31 09:46:00", "month": "08", "high": 227.68, "low": 227.6, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.63, "open": 227.645, "day": "31"} +{"volume": 152657, "symbol": "AAPL", "ts": "2018-08-31 09:47:00", "month": "08", "high": 227.63, "low": 227.37, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.45, "open": 227.62, "day": "31"} +{"volume": 105889, "symbol": "AAPL", "ts": "2018-08-31 09:48:00", "month": "08", "high": 227.66, "low": 227.39, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.6319, "open": 227.4672, "day": "31"} +{"volume": 127795, "symbol": "AAPL", "ts": "2018-08-31 09:49:00", "month": "08", "high": 227.84, "low": 227.59, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.8, "open": 227.6, "day": "31"} +{"volume": 193871, "symbol": "AAPL", "ts": "2018-08-31 09:50:00", "month": "08", "high": 227.97, "low": 227.82, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.95, "open": 227.82, "day": "31"} +{"volume": 299837, "symbol": "AAPL", "ts": "2018-08-31 09:51:00", "month": "08", "high": 228.18, "low": 227.93, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 228.17, "open": 227.93, "day": "31"} +{"volume": 238317, "symbol": "AAPL", "ts": "2018-08-31 09:52:00", "month": "08", "high": 228.26, "low": 228.15, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 228.23, "open": 228.18, "day": "31"} +{"volume": 192605, "symbol": "AAPL", "ts": "2018-08-31 09:53:00", "month": "08", "high": 228.28, "low": 227.96, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.98, "open": 228.25, "day": "31"} +{"volume": 112988, "symbol": "AAPL", "ts": "2018-08-31 09:54:00", "month": "08", "high": 228.0, "low": 227.87, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.87, "open": 227.99, "day": "31"} +{"volume": 129443, "symbol": "AAPL", "ts": "2018-08-31 09:55:00", "month": "08", "high": 228.03, "low": 227.88, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 228.0136, "open": 227.8929, "day": "31"} +{"volume": 130411, "symbol": "AAPL", "ts": "2018-08-31 09:56:00", "month": "08", "high": 228.19, "low": 228.06, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 228.1099, "open": 228.06, "day": "31"} +{"volume": 96287, "symbol": "AAPL", "ts": "2018-08-31 09:57:00", "month": "08", "high": 228.185, "low": 228.08, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 228.1581, "open": 228.11, "day": "31"} +{"volume": 156435, "symbol": "AAPL", "ts": "2018-08-31 09:58:00", "month": "08", "high": 228.16, "low": 227.92, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.92, "open": 228.14, "day": "31"} +{"volume": 144475, "symbol": "AAPL", "ts": "2018-08-31 09:59:00", "month": "08", "high": 227.94, "low": 227.84, "key": "AAPL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.84, "open": 227.94, "day": "31"} +{"volume": 8870, "symbol": "GOOG", "ts": "2018-08-31 09:31:00", "month": "08", "high": 1235.22, "low": 1234.631, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1234.631, "open": 1235.22, "day": "31"} +{"volume": 6927, "symbol": "GOOG", "ts": "2018-08-31 09:32:00", "month": "08", "high": 1235.5, "low": 1234.8425, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1235.5, "open": 1234.8425, "day": "31"} +{"volume": 5343, "symbol": "GOOG", "ts": "2018-08-31 09:33:00", "month": "08", "high": 1236.5, "low": 1236.46, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1236.5, "open": 1236.46, "day": "31"} +{"volume": 3203, "symbol": "GOOG", "ts": "2018-08-31 09:34:00", "month": "08", "high": 1236.64, "low": 1236.11, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1236.11, "open": 1236.64, "day": "31"} +{"volume": 11060, "symbol": "GOOG", "ts": "2018-08-31 09:35:00", "month": "08", "high": 1235.98, "low": 1235.14, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1235.14, "open": 1235.98, "day": "31"} +{"volume": 5254, "symbol": "GOOG", "ts": "2018-08-31 09:36:00", "month": "08", "high": 1234.155, "low": 1233.5, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.5, "open": 1234.155, "day": "31"} +{"volume": 8980, "symbol": "GOOG", "ts": "2018-08-31 09:37:00", "month": "08", "high": 1234.11, "low": 1233.13, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.13, "open": 1234.11, "day": "31"} +{"volume": 5914, "symbol": "GOOG", "ts": "2018-08-31 09:38:00", "month": "08", "high": 1233.77, "low": 1233.5179, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.5179, "open": 1233.59, "day": "31"} +{"volume": 6301, "symbol": "GOOG", "ts": "2018-08-31 09:39:00", "month": "08", "high": 1233.37, "low": 1233.0, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.0, "open": 1233.37, "day": "31"} +{"volume": 7716, "symbol": "GOOG", "ts": "2018-08-31 09:40:00", "month": "08", "high": 1233.87, "low": 1232.6, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.87, "open": 1232.6, "day": "31"} +{"volume": 1933, "symbol": "GOOG", "ts": "2018-08-31 09:41:00", "month": "08", "high": 1233.8051, "low": 1233.8051, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.8051, "open": 1233.8051, "day": "31"} +{"volume": 6653, "symbol": "GOOG", "ts": "2018-08-31 09:42:00", "month": "08", "high": 1232.5, "low": 1231.7, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1232.03, "open": 1232.5, "day": "31"} +{"volume": 5947, "symbol": "GOOG", "ts": "2018-08-31 09:43:00", "month": "08", "high": 1230.9, "low": 1230.78, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1230.9, "open": 1230.78, "day": "31"} +{"volume": 16246, "symbol": "GOOG", "ts": "2018-08-31 09:44:00", "month": "08", "high": 1230.84, "low": 1229.0, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1229.0, "open": 1230.84, "day": "31"} +{"volume": 21988, "symbol": "GOOG", "ts": "2018-08-31 09:45:00", "month": "08", "high": 1230.83, "low": 1227.59, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1230.83, "open": 1228.41, "day": "31"} +{"volume": 14673, "symbol": "GOOG", "ts": "2018-08-31 09:46:00", "month": "08", "high": 1236.47, "low": 1231.39, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1234.88, "open": 1231.39, "day": "31"} +{"volume": 6937, "symbol": "GOOG", "ts": "2018-08-31 09:47:00", "month": "08", "high": 1231.0649, "low": 1230.62, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1231.0, "open": 1230.62, "day": "31"} +{"volume": 3165, "symbol": "GOOG", "ts": "2018-08-31 09:48:00", "month": "08", "high": 1230.0, "low": 1229.66, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1229.66, "open": 1230.0, "day": "31"} +{"volume": 4836, "symbol": "GOOG", "ts": "2018-08-31 09:49:00", "month": "08", "high": 1231.33, "low": 1230.28, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1231.33, "open": 1230.28, "day": "31"} +{"volume": 2983, "symbol": "GOOG", "ts": "2018-08-31 09:50:00", "month": "08", "high": 1231.495, "low": 1231.495, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1231.495, "open": 1231.495, "day": "31"} +{"volume": 1453, "symbol": "GOOG", "ts": "2018-08-31 09:51:00", "month": "08", "high": 1231.78, "low": 1231.78, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1231.78, "open": 1231.78, "day": "31"} +{"volume": 8242, "symbol": "GOOG", "ts": "2018-08-31 09:52:00", "month": "08", "high": 1232.51, "low": 1231.98, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1232.51, "open": 1231.98, "day": "31"} +{"volume": 4987, "symbol": "GOOG", "ts": "2018-08-31 09:53:00", "month": "08", "high": 1233.25, "low": 1233.01, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1233.25, "open": 1233.01, "day": "31"} +{"volume": 2363, "symbol": "GOOG", "ts": "2018-08-31 09:54:00", "month": "08", "high": 1232.0, "low": 1232.0, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1232.0, "open": 1232.0, "day": "31"} +{"volume": 1721, "symbol": "GOOG", "ts": "2018-08-31 09:55:00", "month": "08", "high": 1231.63, "low": 1231.63, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1231.63, "open": 1231.63, "day": "31"} +{"volume": 5547, "symbol": "GOOG", "ts": "2018-08-31 09:56:00", "month": "08", "high": 1230.51, "low": 1229.3101, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1229.3101, "open": 1230.51, "day": "31"} +{"volume": 2019, "symbol": "GOOG", "ts": "2018-08-31 09:57:00", "month": "08", "high": 1229.5, "low": 1229.5, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1229.5, "open": 1229.5, "day": "31"} +{"volume": 7821, "symbol": "GOOG", "ts": "2018-08-31 09:58:00", "month": "08", "high": 1230.04, "low": 1229.5, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1229.5, "open": 1229.9, "day": "31"} +{"volume": 6330, "symbol": "GOOG", "ts": "2018-08-31 09:59:00", "month": "08", "high": 1230.5, "low": 1230.02, "key": "GOOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1230.02, "open": 1230.5, "day": "31"} +{"volume": 93391, "symbol": "FB", "ts": "2018-08-31 09:31:00", "month": "08", "high": 176.96, "low": 176.65, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.905, "open": 176.75, "day": "31"} +{"volume": 139188, "symbol": "FB", "ts": "2018-08-31 09:32:00", "month": "08", "high": 177.0, "low": 176.58, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.64, "open": 176.82, "day": "31"} +{"volume": 100143, "symbol": "FB", "ts": "2018-08-31 09:33:00", "month": "08", "high": 176.77, "low": 176.558, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.558, "open": 176.66, "day": "31"} +{"volume": 115683, "symbol": "FB", "ts": "2018-08-31 09:34:00", "month": "08", "high": 176.72, "low": 176.49, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.49, "open": 176.69, "day": "31"} +{"volume": 191593, "symbol": "FB", "ts": "2018-08-31 09:35:00", "month": "08", "high": 176.5122, "low": 176.2299, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.31, "open": 176.5, "day": "31"} +{"volume": 105472, "symbol": "FB", "ts": "2018-08-31 09:36:00", "month": "08", "high": 177.04, "low": 176.34, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 177.04, "open": 176.34, "day": "31"} +{"volume": 73460, "symbol": "FB", "ts": "2018-08-31 09:37:00", "month": "08", "high": 177.0, "low": 176.67, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.8, "open": 177.0, "day": "31"} +{"volume": 95717, "symbol": "FB", "ts": "2018-08-31 09:38:00", "month": "08", "high": 177.025, "low": 176.72, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.84, "open": 176.89, "day": "31"} +{"volume": 113695, "symbol": "FB", "ts": "2018-08-31 09:39:00", "month": "08", "high": 176.79, "low": 176.42, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.56, "open": 176.79, "day": "31"} +{"volume": 91343, "symbol": "FB", "ts": "2018-08-31 09:40:00", "month": "08", "high": 176.61, "low": 176.41, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 176.41, "open": 176.54, "day": "31"} +{"volume": 292607, "symbol": "FB", "ts": "2018-08-31 09:41:00", "month": "08", "high": 176.5499, "low": 175.84, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.84, "open": 176.4377, "day": "31"} +{"volume": 146532, "symbol": "FB", "ts": "2018-08-31 09:42:00", "month": "08", "high": 176.05, "low": 175.75, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.94, "open": 175.77, "day": "31"} +{"volume": 70556, "symbol": "FB", "ts": "2018-08-31 09:43:00", "month": "08", "high": 176.04, "low": 175.8, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.8418, "open": 175.924, "day": "31"} +{"volume": 159676, "symbol": "FB", "ts": "2018-08-31 09:44:00", "month": "08", "high": 175.81, "low": 175.2856, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.43, "open": 175.81, "day": "31"} +{"volume": 204841, "symbol": "FB", "ts": "2018-08-31 09:45:00", "month": "08", "high": 175.4457, "low": 175.12, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.19, "open": 175.4457, "day": "31"} +{"volume": 240143, "symbol": "FB", "ts": "2018-08-31 09:46:00", "month": "08", "high": 175.84, "low": 175.02, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.35, "open": 175.2494, "day": "31"} +{"volume": 205234, "symbol": "FB", "ts": "2018-08-31 09:47:00", "month": "08", "high": 175.5, "low": 175.0, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.481, "open": 175.3595, "day": "31"} +{"volume": 119350, "symbol": "FB", "ts": "2018-08-31 09:48:00", "month": "08", "high": 175.63, "low": 175.36, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.395, "open": 175.465, "day": "31"} +{"volume": 96532, "symbol": "FB", "ts": "2018-08-31 09:49:00", "month": "08", "high": 175.5101, "low": 175.26, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.39, "open": 175.3999, "day": "31"} +{"volume": 74413, "symbol": "FB", "ts": "2018-08-31 09:50:00", "month": "08", "high": 175.5, "low": 175.3596, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.43, "open": 175.37, "day": "31"} +{"volume": 105026, "symbol": "FB", "ts": "2018-08-31 09:51:00", "month": "08", "high": 175.41, "low": 175.06, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.06, "open": 175.41, "day": "31"} +{"volume": 114656, "symbol": "FB", "ts": "2018-08-31 09:52:00", "month": "08", "high": 175.2099, "low": 175.06, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.15, "open": 175.07, "day": "31"} +{"volume": 131237, "symbol": "FB", "ts": "2018-08-31 09:53:00", "month": "08", "high": 175.6, "low": 175.12, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.6, "open": 175.16, "day": "31"} +{"volume": 116784, "symbol": "FB", "ts": "2018-08-31 09:54:00", "month": "08", "high": 175.65, "low": 175.399, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.52, "open": 175.64, "day": "31"} +{"volume": 77394, "symbol": "FB", "ts": "2018-08-31 09:55:00", "month": "08", "high": 175.58, "low": 175.39, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.39, "open": 175.53, "day": "31"} +{"volume": 59407, "symbol": "FB", "ts": "2018-08-31 09:56:00", "month": "08", "high": 175.47, "low": 175.28, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.28, "open": 175.42, "day": "31"} +{"volume": 62530, "symbol": "FB", "ts": "2018-08-31 09:57:00", "month": "08", "high": 175.35, "low": 175.21, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.2399, "open": 175.3078, "day": "31"} +{"volume": 71679, "symbol": "FB", "ts": "2018-08-31 09:58:00", "month": "08", "high": 175.5, "low": 175.22, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.45, "open": 175.27, "day": "31"} +{"volume": 129694, "symbol": "FB", "ts": "2018-08-31 09:59:00", "month": "08", "high": 175.57, "low": 175.21, "key": "FB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 175.45, "open": 175.38, "day": "31"} +{"volume": 34409, "symbol": "NFLX", "ts": "2018-08-31 09:31:00", "month": "08", "high": 372.3844, "low": 371.7, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.1275, "open": 371.7, "day": "31"} +{"volume": 32038, "symbol": "NFLX", "ts": "2018-08-31 09:32:00", "month": "08", "high": 372.9899, "low": 372.1905, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.9899, "open": 372.1905, "day": "31"} +{"volume": 43699, "symbol": "NFLX", "ts": "2018-08-31 09:33:00", "month": "08", "high": 373.46, "low": 372.8102, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.94, "open": 372.82, "day": "31"} +{"volume": 42777, "symbol": "NFLX", "ts": "2018-08-31 09:34:00", "month": "08", "high": 372.6301, "low": 372.25, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.361, "open": 372.6, "day": "31"} +{"volume": 36596, "symbol": "NFLX", "ts": "2018-08-31 09:35:00", "month": "08", "high": 372.365, "low": 371.67, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.365, "open": 372.36, "day": "31"} +{"volume": 59357, "symbol": "NFLX", "ts": "2018-08-31 09:36:00", "month": "08", "high": 373.39, "low": 372.685, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 373.16, "open": 372.685, "day": "31"} +{"volume": 94398, "symbol": "NFLX", "ts": "2018-08-31 09:37:00", "month": "08", "high": 374.49, "low": 373.3647, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.2104, "open": 373.39, "day": "31"} +{"volume": 76692, "symbol": "NFLX", "ts": "2018-08-31 09:38:00", "month": "08", "high": 375.15, "low": 374.19, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.8375, "open": 374.589, "day": "31"} +{"volume": 72439, "symbol": "NFLX", "ts": "2018-08-31 09:39:00", "month": "08", "high": 375.9119, "low": 374.71, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 375.9119, "open": 374.95, "day": "31"} +{"volume": 78612, "symbol": "NFLX", "ts": "2018-08-31 09:40:00", "month": "08", "high": 375.9324, "low": 374.802, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 375.0, "open": 375.75, "day": "31"} +{"volume": 51243, "symbol": "NFLX", "ts": "2018-08-31 09:41:00", "month": "08", "high": 375.3, "low": 374.591, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 375.0, "open": 374.591, "day": "31"} +{"volume": 34277, "symbol": "NFLX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 375.25, "low": 374.841, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 375.25, "open": 374.841, "day": "31"} +{"volume": 27237, "symbol": "NFLX", "ts": "2018-08-31 09:43:00", "month": "08", "high": 375.46, "low": 375.19, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 375.46, "open": 375.19, "day": "31"} +{"volume": 51444, "symbol": "NFLX", "ts": "2018-08-31 09:44:00", "month": "08", "high": 375.449, "low": 374.91, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.91, "open": 375.37, "day": "31"} +{"volume": 59342, "symbol": "NFLX", "ts": "2018-08-31 09:45:00", "month": "08", "high": 374.74, "low": 373.51, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 373.9606, "open": 374.74, "day": "31"} +{"volume": 65336, "symbol": "NFLX", "ts": "2018-08-31 09:46:00", "month": "08", "high": 374.7, "low": 373.7281, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 373.8105, "open": 373.89, "day": "31"} +{"volume": 16968, "symbol": "NFLX", "ts": "2018-08-31 09:47:00", "month": "08", "high": 374.1524, "low": 373.725, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.1524, "open": 373.725, "day": "31"} +{"volume": 27852, "symbol": "NFLX", "ts": "2018-08-31 09:48:00", "month": "08", "high": 374.82, "low": 374.365, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.47, "open": 374.4912, "day": "31"} +{"volume": 16972, "symbol": "NFLX", "ts": "2018-08-31 09:49:00", "month": "08", "high": 374.5, "low": 374.31, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.3403, "open": 374.31, "day": "31"} +{"volume": 24367, "symbol": "NFLX", "ts": "2018-08-31 09:50:00", "month": "08", "high": 374.94, "low": 374.09, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.09, "open": 374.68, "day": "31"} +{"volume": 18314, "symbol": "NFLX", "ts": "2018-08-31 09:51:00", "month": "08", "high": 374.49, "low": 374.1628, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.1628, "open": 374.2, "day": "31"} +{"volume": 20966, "symbol": "NFLX", "ts": "2018-08-31 09:52:00", "month": "08", "high": 374.26, "low": 373.9012, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.26, "open": 374.07, "day": "31"} +{"volume": 32555, "symbol": "NFLX", "ts": "2018-08-31 09:53:00", "month": "08", "high": 375.06, "low": 374.4337, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.7839, "open": 374.45, "day": "31"} +{"volume": 13153, "symbol": "NFLX", "ts": "2018-08-31 09:54:00", "month": "08", "high": 374.65, "low": 374.45, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 374.46, "open": 374.45, "day": "31"} +{"volume": 31677, "symbol": "NFLX", "ts": "2018-08-31 09:55:00", "month": "08", "high": 374.35, "low": 373.57, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 373.57, "open": 374.35, "day": "31"} +{"volume": 28785, "symbol": "NFLX", "ts": "2018-08-31 09:56:00", "month": "08", "high": 373.8, "low": 373.4, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 373.4, "open": 373.8, "day": "31"} +{"volume": 21538, "symbol": "NFLX", "ts": "2018-08-31 09:57:00", "month": "08", "high": 373.6574, "low": 373.51, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 373.51, "open": 373.61, "day": "31"} +{"volume": 51747, "symbol": "NFLX", "ts": "2018-08-31 09:58:00", "month": "08", "high": 373.4673, "low": 372.6178, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.6178, "open": 373.4673, "day": "31"} +{"volume": 29000, "symbol": "NFLX", "ts": "2018-08-31 09:59:00", "month": "08", "high": 372.874, "low": 372.4962, "key": "NFLX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 372.874, "open": 372.582, "day": "31"} +{"volume": 24286, "symbol": "TSLA", "ts": "2018-08-31 09:31:00", "month": "08", "high": 301.77, "low": 300.67, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 301.34, "open": 301.317, "day": "31"} +{"volume": 29412, "symbol": "TSLA", "ts": "2018-08-31 09:32:00", "month": "08", "high": 301.0, "low": 300.511, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 301.0, "open": 300.78, "day": "31"} +{"volume": 46906, "symbol": "TSLA", "ts": "2018-08-31 09:33:00", "month": "08", "high": 300.86, "low": 299.5651, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 300.0, "open": 300.84, "day": "31"} +{"volume": 31320, "symbol": "TSLA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 299.87, "low": 299.3311, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 299.3311, "open": 299.52, "day": "31"} +{"volume": 39259, "symbol": "TSLA", "ts": "2018-08-31 09:35:00", "month": "08", "high": 300.01, "low": 299.162, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 300.01, "open": 299.162, "day": "31"} +{"volume": 27885, "symbol": "TSLA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 300.0353, "low": 299.47, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 299.47, "open": 299.77, "day": "31"} +{"volume": 29535, "symbol": "TSLA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 300.16, "low": 299.1, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 300.12, "open": 299.495, "day": "31"} +{"volume": 56997, "symbol": "TSLA", "ts": "2018-08-31 09:38:00", "month": "08", "high": 301.7656, "low": 300.15, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 301.26, "open": 300.15, "day": "31"} +{"volume": 39725, "symbol": "TSLA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 302.1999, "low": 300.89, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 301.78, "open": 300.89, "day": "31"} +{"volume": 40712, "symbol": "TSLA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 302.06, "low": 301.6373, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 301.9028, "open": 301.78, "day": "31"} +{"volume": 24535, "symbol": "TSLA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 302.5, "low": 302.03, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 302.27, "open": 302.05, "day": "31"} +{"volume": 28437, "symbol": "TSLA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 302.82, "low": 302.4371, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 302.82, "open": 302.4371, "day": "31"} +{"volume": 45517, "symbol": "TSLA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 303.275, "low": 302.32, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 303.275, "open": 302.575, "day": "31"} +{"volume": 35860, "symbol": "TSLA", "ts": "2018-08-31 09:44:00", "month": "08", "high": 303.63, "low": 302.85, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 303.4686, "open": 302.85, "day": "31"} +{"volume": 30199, "symbol": "TSLA", "ts": "2018-08-31 09:45:00", "month": "08", "high": 303.25, "low": 302.9, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 303.25, "open": 303.2325, "day": "31"} +{"volume": 61219, "symbol": "TSLA", "ts": "2018-08-31 09:46:00", "month": "08", "high": 304.57, "low": 303.51, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.49, "open": 303.51, "day": "31"} +{"volume": 69577, "symbol": "TSLA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 305.0, "low": 304.41, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.97, "open": 304.5492, "day": "31"} +{"volume": 49859, "symbol": "TSLA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 305.19, "low": 304.37, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.615, "open": 305.19, "day": "31"} +{"volume": 33634, "symbol": "TSLA", "ts": "2018-08-31 09:49:00", "month": "08", "high": 304.69, "low": 303.94, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.2034, "open": 304.69, "day": "31"} +{"volume": 14073, "symbol": "TSLA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 304.35, "low": 303.67, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 303.67, "open": 304.35, "day": "31"} +{"volume": 24968, "symbol": "TSLA", "ts": "2018-08-31 09:51:00", "month": "08", "high": 304.2, "low": 303.79, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.2, "open": 303.79, "day": "31"} +{"volume": 18234, "symbol": "TSLA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 304.43, "low": 304.04, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.04, "open": 304.13, "day": "31"} +{"volume": 14477, "symbol": "TSLA", "ts": "2018-08-31 09:53:00", "month": "08", "high": 304.3, "low": 304.04, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 304.282, "open": 304.04, "day": "31"} +{"volume": 19048, "symbol": "TSLA", "ts": "2018-08-31 09:54:00", "month": "08", "high": 303.9129, "low": 303.02, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 303.02, "open": 303.9129, "day": "31"} +{"volume": 33546, "symbol": "TSLA", "ts": "2018-08-31 09:55:00", "month": "08", "high": 303.1512, "low": 302.3, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 302.44, "open": 303.138, "day": "31"} +{"volume": 26069, "symbol": "TSLA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 302.38, "low": 302.0, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 302.17, "open": 302.38, "day": "31"} +{"volume": 28559, "symbol": "TSLA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 302.04, "low": 301.33, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 301.49, "open": 302.04, "day": "31"} +{"volume": 42284, "symbol": "TSLA", "ts": "2018-08-31 09:58:00", "month": "08", "high": 301.7, "low": 300.44, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 300.9, "open": 301.7, "day": "31"} +{"volume": 29353, "symbol": "TSLA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 300.96, "low": 300.65, "key": "TSLA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 300.928, "open": 300.7036, "day": "31"} +{"volume": 225997, "symbol": "F", "ts": "2018-08-31 09:31:00", "month": "08", "high": 9.62, "low": 9.6, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.615, "open": 9.61, "day": "31"} +{"volume": 630124, "symbol": "F", "ts": "2018-08-31 09:32:00", "month": "08", "high": 9.6064, "low": 9.56, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.5703, "open": 9.6, "day": "31"} +{"volume": 120146, "symbol": "F", "ts": "2018-08-31 09:33:00", "month": "08", "high": 9.585, "low": 9.57, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.585, "open": 9.57, "day": "31"} +{"volume": 124975, "symbol": "F", "ts": "2018-08-31 09:34:00", "month": "08", "high": 9.581, "low": 9.56, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.57, "open": 9.58, "day": "31"} +{"volume": 243088, "symbol": "F", "ts": "2018-08-31 09:35:00", "month": "08", "high": 9.6, "low": 9.58, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.58, "day": "31"} +{"volume": 101269, "symbol": "F", "ts": "2018-08-31 09:36:00", "month": "08", "high": 9.6, "low": 9.5943, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.5961, "open": 9.6, "day": "31"} +{"volume": 87079, "symbol": "F", "ts": "2018-08-31 09:37:00", "month": "08", "high": 9.6, "low": 9.5942, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.5942, "day": "31"} +{"volume": 153084, "symbol": "F", "ts": "2018-08-31 09:38:00", "month": "08", "high": 9.61, "low": 9.5999, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.5999, "open": 9.6, "day": "31"} +{"volume": 251319, "symbol": "F", "ts": "2018-08-31 09:39:00", "month": "08", "high": 9.6, "low": 9.59, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.59, "open": 9.6, "day": "31"} +{"volume": 157399, "symbol": "F", "ts": "2018-08-31 09:40:00", "month": "08", "high": 9.58, "low": 9.58, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.58, "open": 9.58, "day": "31"} +{"volume": 147364, "symbol": "F", "ts": "2018-08-31 09:41:00", "month": "08", "high": 9.58, "low": 9.57, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.58, "open": 9.5736, "day": "31"} +{"volume": 67606, "symbol": "F", "ts": "2018-08-31 09:42:00", "month": "08", "high": 9.595, "low": 9.585, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.595, "open": 9.5858, "day": "31"} +{"volume": 37891, "symbol": "F", "ts": "2018-08-31 09:43:00", "month": "08", "high": 9.6, "low": 9.594, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.5997, "open": 9.6, "day": "31"} +{"volume": 219701, "symbol": "F", "ts": "2018-08-31 09:44:00", "month": "08", "high": 9.65, "low": 9.6, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.6, "day": "31"} +{"volume": 277938, "symbol": "F", "ts": "2018-08-31 09:45:00", "month": "08", "high": 9.655, "low": 9.635, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.645, "open": 9.65, "day": "31"} +{"volume": 45923, "symbol": "F", "ts": "2018-08-31 09:46:00", "month": "08", "high": 9.645, "low": 9.64, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.645, "open": 9.64, "day": "31"} +{"volume": 35153, "symbol": "F", "ts": "2018-08-31 09:47:00", "month": "08", "high": 9.645, "low": 9.64, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.64, "open": 9.64, "day": "31"} +{"volume": 99167, "symbol": "F", "ts": "2018-08-31 09:48:00", "month": "08", "high": 9.64, "low": 9.6321, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6321, "open": 9.64, "day": "31"} +{"volume": 168235, "symbol": "F", "ts": "2018-08-31 09:49:00", "month": "08", "high": 9.655, "low": 9.635, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6501, "open": 9.635, "day": "31"} +{"volume": 43725, "symbol": "F", "ts": "2018-08-31 09:50:00", "month": "08", "high": 9.66, "low": 9.65, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.655, "open": 9.65, "day": "31"} +{"volume": 68557, "symbol": "F", "ts": "2018-08-31 09:51:00", "month": "08", "high": 9.655, "low": 9.645, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.645, "open": 9.65, "day": "31"} +{"volume": 74313, "symbol": "F", "ts": "2018-08-31 09:52:00", "month": "08", "high": 9.645, "low": 9.63, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.63, "open": 9.64, "day": "31"} +{"volume": 17124, "symbol": "F", "ts": "2018-08-31 09:53:00", "month": "08", "high": 9.63, "low": 9.63, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.63, "open": 9.63, "day": "31"} +{"volume": 16239, "symbol": "F", "ts": "2018-08-31 09:54:00", "month": "08", "high": 9.6313, "low": 9.63, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.63, "open": 9.6313, "day": "31"} +{"volume": 107746, "symbol": "F", "ts": "2018-08-31 09:55:00", "month": "08", "high": 9.65, "low": 9.64, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.64, "day": "31"} +{"volume": 54722, "symbol": "F", "ts": "2018-08-31 09:56:00", "month": "08", "high": 9.65, "low": 9.641, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.65, "day": "31"} +{"volume": 149482, "symbol": "F", "ts": "2018-08-31 09:57:00", "month": "08", "high": 9.655, "low": 9.65, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.65, "day": "31"} +{"volume": 48596, "symbol": "F", "ts": "2018-08-31 09:58:00", "month": "08", "high": 9.6499, "low": 9.645, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6457, "open": 9.6499, "day": "31"} +{"volume": 29445, "symbol": "F", "ts": "2018-08-31 09:59:00", "month": "08", "high": 9.6464, "low": 9.64, "key": "F_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6464, "open": 9.64, "day": "31"} +{"volume": 31892, "symbol": "AMZN", "ts": "2018-08-31 09:31:00", "month": "08", "high": 2011.0594, "low": 2009.4399, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2011.0594, "open": 2009.92, "day": "31"} +{"volume": 57022, "symbol": "AMZN", "ts": "2018-08-31 09:32:00", "month": "08", "high": 2015.36, "low": 2011.6, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2014.75, "open": 2011.86, "day": "31"} +{"volume": 46623, "symbol": "AMZN", "ts": "2018-08-31 09:33:00", "month": "08", "high": 2015.6, "low": 2014.088, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2015.37, "open": 2014.77, "day": "31"} +{"volume": 21852, "symbol": "AMZN", "ts": "2018-08-31 09:34:00", "month": "08", "high": 2014.95, "low": 2012.575, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2012.7603, "open": 2014.95, "day": "31"} +{"volume": 21081, "symbol": "AMZN", "ts": "2018-08-31 09:35:00", "month": "08", "high": 2012.49, "low": 2010.84, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2011.86, "open": 2012.0601, "day": "31"} +{"volume": 17230, "symbol": "AMZN", "ts": "2018-08-31 09:36:00", "month": "08", "high": 2014.01, "low": 2012.24, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2014.01, "open": 2012.24, "day": "31"} +{"volume": 46746, "symbol": "AMZN", "ts": "2018-08-31 09:37:00", "month": "08", "high": 2017.0, "low": 2014.9449, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.0, "open": 2015.0, "day": "31"} +{"volume": 45842, "symbol": "AMZN", "ts": "2018-08-31 09:38:00", "month": "08", "high": 2018.51, "low": 2016.955, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.5, "open": 2016.955, "day": "31"} +{"volume": 26756, "symbol": "AMZN", "ts": "2018-08-31 09:39:00", "month": "08", "high": 2018.1628, "low": 2016.14, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.24, "open": 2018.12, "day": "31"} +{"volume": 18542, "symbol": "AMZN", "ts": "2018-08-31 09:40:00", "month": "08", "high": 2017.042, "low": 2015.3051, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.042, "open": 2015.5, "day": "31"} +{"volume": 27665, "symbol": "AMZN", "ts": "2018-08-31 09:41:00", "month": "08", "high": 2019.3199, "low": 2017.1226, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2019.3199, "open": 2017.1226, "day": "31"} +{"volume": 26960, "symbol": "AMZN", "ts": "2018-08-31 09:42:00", "month": "08", "high": 2019.39, "low": 2017.5, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2019.38, "open": 2018.245, "day": "31"} +{"volume": 21657, "symbol": "AMZN", "ts": "2018-08-31 09:43:00", "month": "08", "high": 2019.51, "low": 2018.3199, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2018.3199, "open": 2018.8199, "day": "31"} +{"volume": 22278, "symbol": "AMZN", "ts": "2018-08-31 09:44:00", "month": "08", "high": 2018.08, "low": 2016.4087, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.8478, "open": 2017.02, "day": "31"} +{"volume": 30512, "symbol": "AMZN", "ts": "2018-08-31 09:45:00", "month": "08", "high": 2015.86, "low": 2014.0, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2015.86, "open": 2015.63, "day": "31"} +{"volume": 20723, "symbol": "AMZN", "ts": "2018-08-31 09:46:00", "month": "08", "high": 2017.54, "low": 2015.02, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.2524, "open": 2015.02, "day": "31"} +{"volume": 12869, "symbol": "AMZN", "ts": "2018-08-31 09:47:00", "month": "08", "high": 2016.5173, "low": 2015.755, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.5173, "open": 2016.3583, "day": "31"} +{"volume": 22404, "symbol": "AMZN", "ts": "2018-08-31 09:48:00", "month": "08", "high": 2018.1801, "low": 2017.22, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.41, "open": 2017.35, "day": "31"} +{"volume": 16733, "symbol": "AMZN", "ts": "2018-08-31 09:49:00", "month": "08", "high": 2018.7236, "low": 2017.856, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2018.2524, "open": 2018.4399, "day": "31"} +{"volume": 27296, "symbol": "AMZN", "ts": "2018-08-31 09:50:00", "month": "08", "high": 2019.6, "low": 2018.62, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2018.77, "open": 2019.0, "day": "31"} +{"volume": 12816, "symbol": "AMZN", "ts": "2018-08-31 09:51:00", "month": "08", "high": 2018.7374, "low": 2017.51, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.51, "open": 2018.5699, "day": "31"} +{"volume": 12496, "symbol": "AMZN", "ts": "2018-08-31 09:52:00", "month": "08", "high": 2017.5699, "low": 2016.97, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.5699, "open": 2017.474, "day": "31"} +{"volume": 50228, "symbol": "AMZN", "ts": "2018-08-31 09:53:00", "month": "08", "high": 2020.41, "low": 2019.0, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2019.66, "open": 2019.0, "day": "31"} +{"volume": 16700, "symbol": "AMZN", "ts": "2018-08-31 09:54:00", "month": "08", "high": 2019.6421, "low": 2019.12, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2019.4, "open": 2019.5701, "day": "31"} +{"volume": 13998, "symbol": "AMZN", "ts": "2018-08-31 09:55:00", "month": "08", "high": 2019.15, "low": 2017.746, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.746, "open": 2019.15, "day": "31"} +{"volume": 8827, "symbol": "AMZN", "ts": "2018-08-31 09:56:00", "month": "08", "high": 2019.1906, "low": 2018.66, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2019.1906, "open": 2018.66, "day": "31"} +{"volume": 12218, "symbol": "AMZN", "ts": "2018-08-31 09:57:00", "month": "08", "high": 2018.38, "low": 2017.5699, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2017.5699, "open": 2018.38, "day": "31"} +{"volume": 10331, "symbol": "AMZN", "ts": "2018-08-31 09:58:00", "month": "08", "high": 2017.35, "low": 2016.49, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.49, "open": 2017.35, "day": "31"} +{"volume": 21613, "symbol": "AMZN", "ts": "2018-08-31 09:59:00", "month": "08", "high": 2016.25, "low": 2014.77, "key": "AMZN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 2016.25, "open": 2015.7493, "day": "31"} +{"volume": 21439, "symbol": "NVDA", "ts": "2018-08-31 09:31:00", "month": "08", "high": 277.62, "low": 277.1525, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.62, "open": 277.1525, "day": "31"} +{"volume": 52330, "symbol": "NVDA", "ts": "2018-08-31 09:32:00", "month": "08", "high": 278.08, "low": 277.5316, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.8, "open": 277.5316, "day": "31"} +{"volume": 19443, "symbol": "NVDA", "ts": "2018-08-31 09:33:00", "month": "08", "high": 277.8424, "low": 277.57, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.57, "open": 277.8424, "day": "31"} +{"volume": 7863, "symbol": "NVDA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 277.7592, "low": 277.7308, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.7592, "open": 277.7308, "day": "31"} +{"volume": 10981, "symbol": "NVDA", "ts": "2018-08-31 09:35:00", "month": "08", "high": 277.5, "low": 277.29, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.29, "open": 277.5, "day": "31"} +{"volume": 20422, "symbol": "NVDA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 277.645, "low": 277.3499, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 277.645, "open": 277.3499, "day": "31"} +{"volume": 30001, "symbol": "NVDA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 278.26, "low": 277.98, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 278.26, "open": 277.98, "day": "31"} +{"volume": 37397, "symbol": "NVDA", "ts": "2018-08-31 09:38:00", "month": "08", "high": 278.98, "low": 278.59, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 278.98, "open": 278.59, "day": "31"} +{"volume": 25993, "symbol": "NVDA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 278.9688, "low": 278.7, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 278.86, "open": 278.9688, "day": "31"} +{"volume": 38844, "symbol": "NVDA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 279.2, "low": 278.66, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.2, "open": 278.66, "day": "31"} +{"volume": 38861, "symbol": "NVDA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 279.615, "low": 279.2424, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.43, "open": 279.29, "day": "31"} +{"volume": 34384, "symbol": "NVDA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 279.4, "low": 279.155, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.26, "open": 279.4, "day": "31"} +{"volume": 32201, "symbol": "NVDA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 279.61, "low": 279.01, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.01, "open": 279.4, "day": "31"} +{"volume": 30211, "symbol": "NVDA", "ts": "2018-08-31 09:44:00", "month": "08", "high": 279.02, "low": 278.71, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 278.9101, "open": 279.02, "day": "31"} +{"volume": 13082, "symbol": "NVDA", "ts": "2018-08-31 09:45:00", "month": "08", "high": 279.0, "low": 278.85, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 278.87, "open": 279.0, "day": "31"} +{"volume": 54183, "symbol": "NVDA", "ts": "2018-08-31 09:46:00", "month": "08", "high": 279.57, "low": 278.965, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.37, "open": 278.965, "day": "31"} +{"volume": 17724, "symbol": "NVDA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 279.42, "low": 279.1266, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.42, "open": 279.23, "day": "31"} +{"volume": 45364, "symbol": "NVDA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 279.49, "low": 279.13, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.13, "open": 279.49, "day": "31"} +{"volume": 39711, "symbol": "NVDA", "ts": "2018-08-31 09:49:00", "month": "08", "high": 279.48, "low": 279.23, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.45, "open": 279.23, "day": "31"} +{"volume": 36841, "symbol": "NVDA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 279.468, "low": 279.25, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.45, "open": 279.46, "day": "31"} +{"volume": 57356, "symbol": "NVDA", "ts": "2018-08-31 09:51:00", "month": "08", "high": 279.65, "low": 279.38, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.45, "open": 279.42, "day": "31"} +{"volume": 6753, "symbol": "NVDA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 279.4, "low": 279.3401, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.3401, "open": 279.4, "day": "31"} +{"volume": 17969, "symbol": "NVDA", "ts": "2018-08-31 09:53:00", "month": "08", "high": 279.49, "low": 279.3, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.49, "open": 279.3, "day": "31"} +{"volume": 54442, "symbol": "NVDA", "ts": "2018-08-31 09:54:00", "month": "08", "high": 279.5, "low": 279.25, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.27, "open": 279.48, "day": "31"} +{"volume": 53065, "symbol": "NVDA", "ts": "2018-08-31 09:55:00", "month": "08", "high": 279.84, "low": 279.27, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.84, "open": 279.28, "day": "31"} +{"volume": 106833, "symbol": "NVDA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 280.2452, "low": 279.9342, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.9342, "open": 280.0, "day": "31"} +{"volume": 32777, "symbol": "NVDA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 279.99, "low": 279.64, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.64, "open": 279.92, "day": "31"} +{"volume": 37160, "symbol": "NVDA", "ts": "2018-08-31 09:58:00", "month": "08", "high": 279.64, "low": 279.49, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.6, "open": 279.64, "day": "31"} +{"volume": 44086, "symbol": "NVDA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 279.65, "low": 279.25, "key": "NVDA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 279.6, "open": 279.57, "day": "31"} +{"volume": 63090, "symbol": "INTC", "ts": "2018-08-31 09:31:00", "month": "08", "high": 48.16, "low": 48.04, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.14, "open": 48.04, "day": "31"} +{"volume": 32610, "symbol": "INTC", "ts": "2018-08-31 09:32:00", "month": "08", "high": 48.1186, "low": 48.04, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.09, "open": 48.1186, "day": "31"} +{"volume": 108150, "symbol": "INTC", "ts": "2018-08-31 09:33:00", "month": "08", "high": 48.05, "low": 48.0, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.01, "open": 48.045, "day": "31"} +{"volume": 73148, "symbol": "INTC", "ts": "2018-08-31 09:34:00", "month": "08", "high": 47.9899, "low": 47.93, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.95, "open": 47.9899, "day": "31"} +{"volume": 55459, "symbol": "INTC", "ts": "2018-08-31 09:35:00", "month": "08", "high": 48.03, "low": 47.98, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.0002, "open": 47.98, "day": "31"} +{"volume": 39136, "symbol": "INTC", "ts": "2018-08-31 09:36:00", "month": "08", "high": 48.15, "low": 48.09, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.15, "open": 48.09, "day": "31"} +{"volume": 34772, "symbol": "INTC", "ts": "2018-08-31 09:37:00", "month": "08", "high": 48.18, "low": 48.15, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.15, "open": 48.18, "day": "31"} +{"volume": 26792, "symbol": "INTC", "ts": "2018-08-31 09:38:00", "month": "08", "high": 48.18, "low": 48.14, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.18, "open": 48.15, "day": "31"} +{"volume": 37731, "symbol": "INTC", "ts": "2018-08-31 09:39:00", "month": "08", "high": 48.16, "low": 48.14, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.15, "open": 48.14, "day": "31"} +{"volume": 78398, "symbol": "INTC", "ts": "2018-08-31 09:40:00", "month": "08", "high": 48.17, "low": 48.12, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.17, "open": 48.13, "day": "31"} +{"volume": 64069, "symbol": "INTC", "ts": "2018-08-31 09:41:00", "month": "08", "high": 48.2, "low": 48.15, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.17, "open": 48.18, "day": "31"} +{"volume": 43922, "symbol": "INTC", "ts": "2018-08-31 09:42:00", "month": "08", "high": 48.17, "low": 48.145, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.155, "open": 48.17, "day": "31"} +{"volume": 13524, "symbol": "INTC", "ts": "2018-08-31 09:43:00", "month": "08", "high": 48.2, "low": 48.19, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.2, "open": 48.2, "day": "31"} +{"volume": 28314, "symbol": "INTC", "ts": "2018-08-31 09:44:00", "month": "08", "high": 48.19, "low": 48.1428, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.15, "open": 48.19, "day": "31"} +{"volume": 44623, "symbol": "INTC", "ts": "2018-08-31 09:45:00", "month": "08", "high": 48.185, "low": 48.14, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.175, "open": 48.15, "day": "31"} +{"volume": 40806, "symbol": "INTC", "ts": "2018-08-31 09:46:00", "month": "08", "high": 48.23, "low": 48.2, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.215, "open": 48.203, "day": "31"} +{"volume": 41183, "symbol": "INTC", "ts": "2018-08-31 09:47:00", "month": "08", "high": 48.23, "low": 48.19, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.195, "open": 48.2163, "day": "31"} +{"volume": 11401, "symbol": "INTC", "ts": "2018-08-31 09:48:00", "month": "08", "high": 48.24, "low": 48.21, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.24, "open": 48.21, "day": "31"} +{"volume": 49983, "symbol": "INTC", "ts": "2018-08-31 09:49:00", "month": "08", "high": 48.27, "low": 48.22, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.22, "open": 48.25, "day": "31"} +{"volume": 152956, "symbol": "INTC", "ts": "2018-08-31 09:50:00", "month": "08", "high": 48.23, "low": 48.09, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.09, "open": 48.23, "day": "31"} +{"volume": 99450, "symbol": "INTC", "ts": "2018-08-31 09:51:00", "month": "08", "high": 48.1, "low": 48.045, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.05, "open": 48.09, "day": "31"} +{"volume": 59783, "symbol": "INTC", "ts": "2018-08-31 09:52:00", "month": "08", "high": 48.06, "low": 48.03, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.04, "open": 48.06, "day": "31"} +{"volume": 42191, "symbol": "INTC", "ts": "2018-08-31 09:53:00", "month": "08", "high": 48.07, "low": 48.035, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.035, "open": 48.05, "day": "31"} +{"volume": 28556, "symbol": "INTC", "ts": "2018-08-31 09:54:00", "month": "08", "high": 48.051, "low": 48.025, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.025, "open": 48.045, "day": "31"} +{"volume": 40270, "symbol": "INTC", "ts": "2018-08-31 09:55:00", "month": "08", "high": 48.045, "low": 48.0233, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.0325, "open": 48.03, "day": "31"} +{"volume": 32551, "symbol": "INTC", "ts": "2018-08-31 09:56:00", "month": "08", "high": 48.0601, "low": 48.035, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.0601, "open": 48.035, "day": "31"} +{"volume": 53683, "symbol": "INTC", "ts": "2018-08-31 09:57:00", "month": "08", "high": 48.05, "low": 48.01, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.03, "open": 48.05, "day": "31"} +{"volume": 51842, "symbol": "INTC", "ts": "2018-08-31 09:58:00", "month": "08", "high": 48.03, "low": 48.02, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.02, "open": 48.03, "day": "31"} +{"volume": 49702, "symbol": "INTC", "ts": "2018-08-31 09:59:00", "month": "08", "high": 48.11, "low": 48.02, "key": "INTC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.11, "open": 48.02, "day": "31"} +{"volume": 50495, "symbol": "TNDM", "ts": "2018-08-31 09:31:00", "month": "08", "high": 45.07, "low": 44.73, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.07, "open": 44.73, "day": "31"} +{"volume": 42764, "symbol": "TNDM", "ts": "2018-08-31 09:32:00", "month": "08", "high": 45.4163, "low": 45.2, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.4163, "open": 45.2, "day": "31"} +{"volume": 62325, "symbol": "TNDM", "ts": "2018-08-31 09:33:00", "month": "08", "high": 45.69, "low": 45.16, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.16, "open": 45.59, "day": "31"} +{"volume": 43044, "symbol": "TNDM", "ts": "2018-08-31 09:34:00", "month": "08", "high": 44.9475, "low": 44.7, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.8151, "open": 44.9475, "day": "31"} +{"volume": 28263, "symbol": "TNDM", "ts": "2018-08-31 09:35:00", "month": "08", "high": 44.8115, "low": 44.57, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.57, "open": 44.8115, "day": "31"} +{"volume": 15879, "symbol": "TNDM", "ts": "2018-08-31 09:36:00", "month": "08", "high": 44.62, "low": 44.58, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.58, "open": 44.62, "day": "31"} +{"volume": 21259, "symbol": "TNDM", "ts": "2018-08-31 09:37:00", "month": "08", "high": 44.7, "low": 44.56, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.56, "open": 44.7, "day": "31"} +{"volume": 20446, "symbol": "TNDM", "ts": "2018-08-31 09:38:00", "month": "08", "high": 44.76, "low": 44.55, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.76, "open": 44.65, "day": "31"} +{"volume": 14201, "symbol": "TNDM", "ts": "2018-08-31 09:39:00", "month": "08", "high": 44.75, "low": 44.63, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.63, "open": 44.75, "day": "31"} +{"volume": 9461, "symbol": "TNDM", "ts": "2018-08-31 09:40:00", "month": "08", "high": 44.7, "low": 44.6762, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.7, "open": 44.6762, "day": "31"} +{"volume": 21261, "symbol": "TNDM", "ts": "2018-08-31 09:41:00", "month": "08", "high": 44.75, "low": 44.66, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.75, "open": 44.66, "day": "31"} +{"volume": 16469, "symbol": "TNDM", "ts": "2018-08-31 09:42:00", "month": "08", "high": 44.839, "low": 44.75, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.839, "open": 44.75, "day": "31"} +{"volume": 28954, "symbol": "TNDM", "ts": "2018-08-31 09:43:00", "month": "08", "high": 45.23, "low": 45.0, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.23, "open": 45.0, "day": "31"} +{"volume": 36111, "symbol": "TNDM", "ts": "2018-08-31 09:44:00", "month": "08", "high": 45.31, "low": 45.15, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.31, "open": 45.26, "day": "31"} +{"volume": 24244, "symbol": "TNDM", "ts": "2018-08-31 09:45:00", "month": "08", "high": 45.287, "low": 45.115, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.205, "open": 45.287, "day": "31"} +{"volume": 7653, "symbol": "TNDM", "ts": "2018-08-31 09:46:00", "month": "08", "high": 45.36, "low": 45.36, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.36, "open": 45.36, "day": "31"} +{"volume": 16687, "symbol": "TNDM", "ts": "2018-08-31 09:47:00", "month": "08", "high": 45.25, "low": 45.2, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.2358, "open": 45.2, "day": "31"} +{"volume": 17932, "symbol": "TNDM", "ts": "2018-08-31 09:48:00", "month": "08", "high": 45.21, "low": 45.14, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.1601, "open": 45.21, "day": "31"} +{"volume": 8409, "symbol": "TNDM", "ts": "2018-08-31 09:49:00", "month": "08", "high": 45.26, "low": 45.2, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.26, "open": 45.2, "day": "31"} +{"volume": 18372, "symbol": "TNDM", "ts": "2018-08-31 09:50:00", "month": "08", "high": 45.2767, "low": 45.19, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.19, "open": 45.2767, "day": "31"} +{"volume": 14942, "symbol": "TNDM", "ts": "2018-08-31 09:51:00", "month": "08", "high": 45.1782, "low": 45.05, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.05, "open": 45.12, "day": "31"} +{"volume": 18018, "symbol": "TNDM", "ts": "2018-08-31 09:52:00", "month": "08", "high": 45.05, "low": 44.93, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.93, "open": 45.025, "day": "31"} +{"volume": 4863, "symbol": "TNDM", "ts": "2018-08-31 09:53:00", "month": "08", "high": 45.06, "low": 45.06, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.06, "open": 45.06, "day": "31"} +{"volume": 7628, "symbol": "TNDM", "ts": "2018-08-31 09:54:00", "month": "08", "high": 45.21, "low": 45.18, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.21, "open": 45.18, "day": "31"} +{"volume": 2729, "symbol": "TNDM", "ts": "2018-08-31 09:55:00", "month": "08", "high": 45.1842, "low": 45.1842, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.1842, "open": 45.1842, "day": "31"} +{"volume": 8895, "symbol": "TNDM", "ts": "2018-08-31 09:56:00", "month": "08", "high": 45.04, "low": 45.04, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.04, "open": 45.04, "day": "31"} +{"volume": 21089, "symbol": "TNDM", "ts": "2018-08-31 09:57:00", "month": "08", "high": 45.02, "low": 44.9, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.9, "open": 45.02, "day": "31"} +{"volume": 11154, "symbol": "TNDM", "ts": "2018-08-31 09:58:00", "month": "08", "high": 44.8777, "low": 44.86, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.8777, "open": 44.86, "day": "31"} +{"volume": 24349, "symbol": "TNDM", "ts": "2018-08-31 09:59:00", "month": "08", "high": 44.75, "low": 44.58, "key": "TNDM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.7231, "open": 44.75, "day": "31"} +{"volume": 1321, "symbol": "CDNA", "ts": "2018-08-31 09:32:00", "month": "08", "high": 23.1442, "low": 23.1442, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.1442, "open": 23.1442, "day": "31"} +{"volume": 2939, "symbol": "CDNA", "ts": "2018-08-31 09:33:00", "month": "08", "high": 23.1337, "low": 23.1337, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.1337, "open": 23.1337, "day": "31"} +{"volume": 2650, "symbol": "CDNA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 23.15, "low": 23.15, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.15, "open": 23.15, "day": "31"} +{"volume": 764, "symbol": "CDNA", "ts": "2018-08-31 09:38:00", "month": "08", "high": 23.23, "low": 23.23, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.23, "open": 23.23, "day": "31"} +{"volume": 2952, "symbol": "CDNA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 23.3816, "low": 23.3816, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.3816, "open": 23.3816, "day": "31"} +{"volume": 825, "symbol": "CDNA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 23.32, "low": 23.32, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.32, "open": 23.32, "day": "31"} +{"volume": 856, "symbol": "CDNA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 23.2016, "low": 23.2016, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.2016, "open": 23.2016, "day": "31"} +{"volume": 2006, "symbol": "CDNA", "ts": "2018-08-31 09:44:00", "month": "08", "high": 23.2, "low": 23.2, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.2, "open": 23.2, "day": "31"} +{"volume": 8895, "symbol": "CDNA", "ts": "2018-08-31 09:46:00", "month": "08", "high": 23.14, "low": 23.14, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.14, "open": 23.14, "day": "31"} +{"volume": 9615, "symbol": "CDNA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 23.14, "low": 23.14, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.14, "open": 23.14, "day": "31"} +{"volume": 453, "symbol": "CDNA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 23.111, "low": 23.111, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.111, "open": 23.111, "day": "31"} +{"volume": 3763, "symbol": "CDNA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 23.31, "low": 23.31, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.31, "open": 23.31, "day": "31"} +{"volume": 1228, "symbol": "CDNA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 23.41, "low": 23.41, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.41, "open": 23.41, "day": "31"} +{"volume": 1409, "symbol": "CDNA", "ts": "2018-08-31 09:55:00", "month": "08", "high": 23.6, "low": 23.6, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.6, "open": 23.6, "day": "31"} +{"volume": 1460, "symbol": "CDNA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 23.695, "low": 23.695, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.695, "open": 23.695, "day": "31"} +{"volume": 7515, "symbol": "CDNA", "ts": "2018-08-31 09:58:00", "month": "08", "high": 23.5575, "low": 23.5575, "key": "CDNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.5575, "open": 23.5575, "day": "31"} +{"volume": 421, "symbol": "IIN", "ts": "2018-08-31 09:32:00", "month": "08", "high": 73.125, "low": 73.125, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.125, "open": 73.125, "day": "31"} +{"volume": 841, "symbol": "IIN", "ts": "2018-08-31 09:35:00", "month": "08", "high": 73.35, "low": 73.35, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.35, "open": 73.35, "day": "31"} +{"volume": 1416, "symbol": "IIN", "ts": "2018-08-31 09:37:00", "month": "08", "high": 73.6597, "low": 73.6597, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.6597, "open": 73.6597, "day": "31"} +{"volume": 843, "symbol": "IIN", "ts": "2018-08-31 09:41:00", "month": "08", "high": 73.555, "low": 73.555, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.555, "open": 73.555, "day": "31"} +{"volume": 1400, "symbol": "IIN", "ts": "2018-08-31 09:44:00", "month": "08", "high": 74.0, "low": 74.0, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.0, "open": 74.0, "day": "31"} +{"volume": 1996, "symbol": "IIN", "ts": "2018-08-31 09:47:00", "month": "08", "high": 74.0844, "low": 74.0844, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.0844, "open": 74.0844, "day": "31"} +{"volume": 1395, "symbol": "IIN", "ts": "2018-08-31 09:48:00", "month": "08", "high": 74.35, "low": 74.35, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.35, "open": 74.35, "day": "31"} +{"volume": 1044, "symbol": "IIN", "ts": "2018-08-31 09:50:00", "month": "08", "high": 74.1583, "low": 74.1583, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.1583, "open": 74.1583, "day": "31"} +{"volume": 846, "symbol": "IIN", "ts": "2018-08-31 09:53:00", "month": "08", "high": 74.125, "low": 74.125, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.125, "open": 74.125, "day": "31"} +{"volume": 1599, "symbol": "IIN", "ts": "2018-08-31 09:54:00", "month": "08", "high": 74.2152, "low": 74.2152, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.2152, "open": 74.2152, "day": "31"} +{"volume": 616, "symbol": "IIN", "ts": "2018-08-31 09:57:00", "month": "08", "high": 74.075, "low": 74.075, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.075, "open": 74.075, "day": "31"} +{"volume": 2464, "symbol": "IIN", "ts": "2018-08-31 09:59:00", "month": "08", "high": 73.61, "low": 73.61, "key": "IIN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.61, "open": 73.61, "day": "31"} +{"volume": 1380, "symbol": "TPNL", "ts": "2018-08-31 09:41:00", "month": "08", "high": 3.4, "low": 3.4, "key": "TPNL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.4, "open": 3.4, "day": "31"} +{"volume": 162, "symbol": "TPNL", "ts": "2018-08-31 09:46:00", "month": "08", "high": 3.39, "low": 3.39, "key": "TPNL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.39, "open": 3.39, "day": "31"} +{"volume": 2786, "symbol": "TPNL", "ts": "2018-08-31 09:52:00", "month": "08", "high": 3.39, "low": 3.37, "key": "TPNL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.39, "day": "31"} +{"volume": 184500, "symbol": "CGC", "ts": "2018-08-31 09:31:00", "month": "08", "high": 44.97, "low": 44.59, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.9, "open": 44.59, "day": "31"} +{"volume": 149049, "symbol": "CGC", "ts": "2018-08-31 09:32:00", "month": "08", "high": 44.95, "low": 44.78, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.8993, "open": 44.9, "day": "31"} +{"volume": 67438, "symbol": "CGC", "ts": "2018-08-31 09:33:00", "month": "08", "high": 44.93, "low": 44.79, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.83, "open": 44.8659, "day": "31"} +{"volume": 138395, "symbol": "CGC", "ts": "2018-08-31 09:34:00", "month": "08", "high": 45.19, "low": 44.82, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.0618, "open": 44.82, "day": "31"} +{"volume": 109993, "symbol": "CGC", "ts": "2018-08-31 09:35:00", "month": "08", "high": 45.25, "low": 45.01, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.25, "open": 45.0972, "day": "31"} +{"volume": 188071, "symbol": "CGC", "ts": "2018-08-31 09:36:00", "month": "08", "high": 45.5, "low": 45.17, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.26, "open": 45.25, "day": "31"} +{"volume": 80770, "symbol": "CGC", "ts": "2018-08-31 09:37:00", "month": "08", "high": 45.427, "low": 45.27, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.4165, "open": 45.33, "day": "31"} +{"volume": 95992, "symbol": "CGC", "ts": "2018-08-31 09:38:00", "month": "08", "high": 45.41, "low": 45.28, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.3399, "open": 45.41, "day": "31"} +{"volume": 84639, "symbol": "CGC", "ts": "2018-08-31 09:39:00", "month": "08", "high": 45.267, "low": 45.12, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.12, "open": 45.2577, "day": "31"} +{"volume": 64481, "symbol": "CGC", "ts": "2018-08-31 09:40:00", "month": "08", "high": 45.07, "low": 44.9603, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.9603, "open": 45.07, "day": "31"} +{"volume": 79580, "symbol": "CGC", "ts": "2018-08-31 09:41:00", "month": "08", "high": 45.06, "low": 44.92, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.06, "open": 44.95, "day": "31"} +{"volume": 55982, "symbol": "CGC", "ts": "2018-08-31 09:42:00", "month": "08", "high": 45.0512, "low": 44.74, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.74, "open": 45.0396, "day": "31"} +{"volume": 87011, "symbol": "CGC", "ts": "2018-08-31 09:43:00", "month": "08", "high": 44.88, "low": 44.62, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.78, "open": 44.73, "day": "31"} +{"volume": 55292, "symbol": "CGC", "ts": "2018-08-31 09:44:00", "month": "08", "high": 44.8, "low": 44.68, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.77, "open": 44.8, "day": "31"} +{"volume": 34227, "symbol": "CGC", "ts": "2018-08-31 09:45:00", "month": "08", "high": 44.82, "low": 44.7297, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.82, "open": 44.7799, "day": "31"} +{"volume": 67103, "symbol": "CGC", "ts": "2018-08-31 09:46:00", "month": "08", "high": 45.07, "low": 44.875, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.9983, "open": 44.875, "day": "31"} +{"volume": 42142, "symbol": "CGC", "ts": "2018-08-31 09:47:00", "month": "08", "high": 45.15, "low": 45.0277, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.0277, "open": 45.035, "day": "31"} +{"volume": 52208, "symbol": "CGC", "ts": "2018-08-31 09:48:00", "month": "08", "high": 45.1, "low": 44.9, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.9687, "open": 44.99, "day": "31"} +{"volume": 43085, "symbol": "CGC", "ts": "2018-08-31 09:49:00", "month": "08", "high": 45.07, "low": 44.9746, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.9746, "open": 45.03, "day": "31"} +{"volume": 78322, "symbol": "CGC", "ts": "2018-08-31 09:50:00", "month": "08", "high": 45.15, "low": 44.9767, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.15, "open": 44.9767, "day": "31"} +{"volume": 107814, "symbol": "CGC", "ts": "2018-08-31 09:51:00", "month": "08", "high": 45.28, "low": 45.145, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.24, "open": 45.145, "day": "31"} +{"volume": 42103, "symbol": "CGC", "ts": "2018-08-31 09:52:00", "month": "08", "high": 45.1948, "low": 45.03, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.0999, "open": 45.1948, "day": "31"} +{"volume": 18196, "symbol": "CGC", "ts": "2018-08-31 09:53:00", "month": "08", "high": 45.1162, "low": 45.09, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.094, "open": 45.09, "day": "31"} +{"volume": 36419, "symbol": "CGC", "ts": "2018-08-31 09:54:00", "month": "08", "high": 45.0514, "low": 45.03, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.0514, "open": 45.04, "day": "31"} +{"volume": 55374, "symbol": "CGC", "ts": "2018-08-31 09:55:00", "month": "08", "high": 45.2422, "low": 45.0935, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.24, "open": 45.0935, "day": "31"} +{"volume": 116152, "symbol": "CGC", "ts": "2018-08-31 09:56:00", "month": "08", "high": 45.36, "low": 45.27, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.28, "open": 45.29, "day": "31"} +{"volume": 37200, "symbol": "CGC", "ts": "2018-08-31 09:57:00", "month": "08", "high": 45.32, "low": 45.1835, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.32, "open": 45.31, "day": "31"} +{"volume": 93471, "symbol": "CGC", "ts": "2018-08-31 09:58:00", "month": "08", "high": 45.4399, "low": 45.31, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.4, "open": 45.32, "day": "31"} +{"volume": 137411, "symbol": "CGC", "ts": "2018-08-31 09:59:00", "month": "08", "high": 45.59, "low": 45.3988, "key": "CGC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.5573, "open": 45.42, "day": "31"} +{"volume": 745, "symbol": "FNKO", "ts": "2018-08-31 09:32:00", "month": "08", "high": 24.14, "low": 24.14, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.14, "open": 24.14, "day": "31"} +{"volume": 1883, "symbol": "FNKO", "ts": "2018-08-31 09:35:00", "month": "08", "high": 24.15, "low": 24.15, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.15, "open": 24.15, "day": "31"} +{"volume": 882, "symbol": "FNKO", "ts": "2018-08-31 09:37:00", "month": "08", "high": 24.2007, "low": 24.2007, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.2007, "open": 24.2007, "day": "31"} +{"volume": 2775, "symbol": "FNKO", "ts": "2018-08-31 09:38:00", "month": "08", "high": 24.25, "low": 24.25, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.25, "open": 24.25, "day": "31"} +{"volume": 1869, "symbol": "FNKO", "ts": "2018-08-31 09:40:00", "month": "08", "high": 24.399, "low": 24.399, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.399, "open": 24.399, "day": "31"} +{"volume": 2018, "symbol": "FNKO", "ts": "2018-08-31 09:41:00", "month": "08", "high": 24.4012, "low": 24.4012, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.4012, "open": 24.4012, "day": "31"} +{"volume": 1183, "symbol": "FNKO", "ts": "2018-08-31 09:42:00", "month": "08", "high": 24.41, "low": 24.41, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.41, "open": 24.41, "day": "31"} +{"volume": 1324, "symbol": "FNKO", "ts": "2018-08-31 09:44:00", "month": "08", "high": 24.421, "low": 24.421, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.421, "open": 24.421, "day": "31"} +{"volume": 3257, "symbol": "FNKO", "ts": "2018-08-31 09:46:00", "month": "08", "high": 24.5, "low": 24.5, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.5, "open": 24.5, "day": "31"} +{"volume": 1038, "symbol": "FNKO", "ts": "2018-08-31 09:48:00", "month": "08", "high": 24.5648, "low": 24.5648, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.5648, "open": 24.5648, "day": "31"} +{"volume": 1635, "symbol": "FNKO", "ts": "2018-08-31 09:49:00", "month": "08", "high": 24.574, "low": 24.574, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.574, "open": 24.574, "day": "31"} +{"volume": 2285, "symbol": "FNKO", "ts": "2018-08-31 09:51:00", "month": "08", "high": 24.5997, "low": 24.5997, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.5997, "open": 24.5997, "day": "31"} +{"volume": 1669, "symbol": "FNKO", "ts": "2018-08-31 09:53:00", "month": "08", "high": 24.66, "low": 24.66, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.66, "open": 24.66, "day": "31"} +{"volume": 1546, "symbol": "FNKO", "ts": "2018-08-31 09:55:00", "month": "08", "high": 24.6975, "low": 24.6975, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.6975, "open": 24.6975, "day": "31"} +{"volume": 2873, "symbol": "FNKO", "ts": "2018-08-31 09:56:00", "month": "08", "high": 24.6664, "low": 24.6664, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.6664, "open": 24.6664, "day": "31"} +{"volume": 2979, "symbol": "FNKO", "ts": "2018-08-31 09:59:00", "month": "08", "high": 24.6388, "low": 24.6388, "key": "FNKO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.6388, "open": 24.6388, "day": "31"} +{"volume": 4272, "symbol": "XENE", "ts": "2018-08-31 09:33:00", "month": "08", "high": 13.4, "low": 13.4, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 2345, "symbol": "XENE", "ts": "2018-08-31 09:35:00", "month": "08", "high": 13.45, "low": 13.45, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.45, "open": 13.45, "day": "31"} +{"volume": 7408, "symbol": "XENE", "ts": "2018-08-31 09:39:00", "month": "08", "high": 13.5, "low": 13.5, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.5, "open": 13.5, "day": "31"} +{"volume": 3182, "symbol": "XENE", "ts": "2018-08-31 09:41:00", "month": "08", "high": 13.55, "low": 13.55, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.55, "open": 13.55, "day": "31"} +{"volume": 6538, "symbol": "XENE", "ts": "2018-08-31 09:43:00", "month": "08", "high": 13.6, "low": 13.55, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.6, "open": 13.55, "day": "31"} +{"volume": 1806, "symbol": "XENE", "ts": "2018-08-31 09:47:00", "month": "08", "high": 13.4, "low": 13.4, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 1050, "symbol": "XENE", "ts": "2018-08-31 09:49:00", "month": "08", "high": 13.405, "low": 13.405, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.405, "open": 13.405, "day": "31"} +{"volume": 3150, "symbol": "XENE", "ts": "2018-08-31 09:52:00", "month": "08", "high": 13.545, "low": 13.478, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.545, "open": 13.478, "day": "31"} +{"volume": 2624, "symbol": "XENE", "ts": "2018-08-31 09:55:00", "month": "08", "high": 13.5, "low": 13.5, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.5, "open": 13.5, "day": "31"} +{"volume": 8405, "symbol": "XENE", "ts": "2018-08-31 09:57:00", "month": "08", "high": 13.25, "low": 13.25, "key": "XENE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.25, "open": 13.25, "day": "31"} +{"volume": 3904, "symbol": "SSTI", "ts": "2018-08-31 09:32:00", "month": "08", "high": 55.65, "low": 55.65, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 55.65, "open": 55.65, "day": "31"} +{"volume": 7200, "symbol": "SSTI", "ts": "2018-08-31 09:34:00", "month": "08", "high": 55.25, "low": 55.249, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 55.249, "open": 55.25, "day": "31"} +{"volume": 7468, "symbol": "SSTI", "ts": "2018-08-31 09:35:00", "month": "08", "high": 55.8, "low": 55.8, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 55.8, "open": 55.8, "day": "31"} +{"volume": 8977, "symbol": "SSTI", "ts": "2018-08-31 09:36:00", "month": "08", "high": 55.01, "low": 55.0, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 55.0, "open": 55.01, "day": "31"} +{"volume": 9865, "symbol": "SSTI", "ts": "2018-08-31 09:37:00", "month": "08", "high": 54.9562, "low": 54.74, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.74, "open": 54.9562, "day": "31"} +{"volume": 1526, "symbol": "SSTI", "ts": "2018-08-31 09:38:00", "month": "08", "high": 54.78, "low": 54.78, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.78, "open": 54.78, "day": "31"} +{"volume": 6460, "symbol": "SSTI", "ts": "2018-08-31 09:39:00", "month": "08", "high": 54.995, "low": 54.9, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.995, "open": 54.9, "day": "31"} +{"volume": 4094, "symbol": "SSTI", "ts": "2018-08-31 09:40:00", "month": "08", "high": 54.93, "low": 54.93, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.93, "open": 54.93, "day": "31"} +{"volume": 471, "symbol": "SSTI", "ts": "2018-08-31 09:41:00", "month": "08", "high": 54.81, "low": 54.81, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.81, "open": 54.81, "day": "31"} +{"volume": 844, "symbol": "SSTI", "ts": "2018-08-31 09:44:00", "month": "08", "high": 55.13, "low": 55.13, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 55.13, "open": 55.13, "day": "31"} +{"volume": 3716, "symbol": "SSTI", "ts": "2018-08-31 09:45:00", "month": "08", "high": 54.4479, "low": 54.4479, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.4479, "open": 54.4479, "day": "31"} +{"volume": 2596, "symbol": "SSTI", "ts": "2018-08-31 09:46:00", "month": "08", "high": 54.71, "low": 54.71, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.71, "open": 54.71, "day": "31"} +{"volume": 155, "symbol": "SSTI", "ts": "2018-08-31 09:54:00", "month": "08", "high": 54.721, "low": 54.721, "key": "SSTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 54.721, "open": 54.721, "day": "31"} +{"volume": 0, "symbol": "VRS", "ts": "2018-08-31 09:31:00", "month": "08", "high": 30.716, "low": 30.716, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.716, "open": 30.716, "day": "31"} +{"volume": 4422, "symbol": "VRS", "ts": "2018-08-31 09:35:00", "month": "08", "high": 30.829, "low": 30.829, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.829, "open": 30.829, "day": "31"} +{"volume": 2462, "symbol": "VRS", "ts": "2018-08-31 09:36:00", "month": "08", "high": 30.8099, "low": 30.8099, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.8099, "open": 30.8099, "day": "31"} +{"volume": 5651, "symbol": "VRS", "ts": "2018-08-31 09:38:00", "month": "08", "high": 30.8473, "low": 30.8473, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.8473, "open": 30.8473, "day": "31"} +{"volume": 1700, "symbol": "VRS", "ts": "2018-08-31 09:39:00", "month": "08", "high": 30.8597, "low": 30.8597, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.8597, "open": 30.8597, "day": "31"} +{"volume": 2000, "symbol": "VRS", "ts": "2018-08-31 09:40:00", "month": "08", "high": 31.01, "low": 31.01, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.01, "open": 31.01, "day": "31"} +{"volume": 9541, "symbol": "VRS", "ts": "2018-08-31 09:41:00", "month": "08", "high": 31.035, "low": 31.01, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.01, "open": 31.035, "day": "31"} +{"volume": 3959, "symbol": "VRS", "ts": "2018-08-31 09:43:00", "month": "08", "high": 30.96, "low": 30.96, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.96, "open": 30.96, "day": "31"} +{"volume": 4151, "symbol": "VRS", "ts": "2018-08-31 09:44:00", "month": "08", "high": 31.03, "low": 31.03, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.03, "open": 31.03, "day": "31"} +{"volume": 2830, "symbol": "VRS", "ts": "2018-08-31 09:46:00", "month": "08", "high": 30.92, "low": 30.92, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 30.92, "open": 30.92, "day": "31"} +{"volume": 5120, "symbol": "VRS", "ts": "2018-08-31 09:47:00", "month": "08", "high": 31.03, "low": 31.0, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.01, "open": 31.0, "day": "31"} +{"volume": 2169, "symbol": "VRS", "ts": "2018-08-31 09:49:00", "month": "08", "high": 31.0, "low": 31.0, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.0, "open": 31.0, "day": "31"} +{"volume": 53973, "symbol": "VRS", "ts": "2018-08-31 09:51:00", "month": "08", "high": 31.326, "low": 30.99, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.26, "open": 30.99, "day": "31"} +{"volume": 51471, "symbol": "VRS", "ts": "2018-08-31 09:52:00", "month": "08", "high": 31.395, "low": 31.1179, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.395, "open": 31.1179, "day": "31"} +{"volume": 27213, "symbol": "VRS", "ts": "2018-08-31 09:53:00", "month": "08", "high": 31.36, "low": 31.17, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.17, "open": 31.36, "day": "31"} +{"volume": 25694, "symbol": "VRS", "ts": "2018-08-31 09:54:00", "month": "08", "high": 31.43, "low": 31.38, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.4227, "open": 31.38, "day": "31"} +{"volume": 13989, "symbol": "VRS", "ts": "2018-08-31 09:55:00", "month": "08", "high": 31.45, "low": 31.4201, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.45, "open": 31.4201, "day": "31"} +{"volume": 11967, "symbol": "VRS", "ts": "2018-08-31 09:56:00", "month": "08", "high": 31.455, "low": 31.43, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.43, "open": 31.455, "day": "31"} +{"volume": 2973, "symbol": "VRS", "ts": "2018-08-31 09:57:00", "month": "08", "high": 31.4355, "low": 31.4355, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.4355, "open": 31.4355, "day": "31"} +{"volume": 2094, "symbol": "VRS", "ts": "2018-08-31 09:58:00", "month": "08", "high": 31.324, "low": 31.324, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.324, "open": 31.324, "day": "31"} +{"volume": 5150, "symbol": "VRS", "ts": "2018-08-31 09:59:00", "month": "08", "high": 31.28, "low": 31.28, "key": "VRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 31.28, "open": 31.28, "day": "31"} +{"volume": 20399, "symbol": "ARWR", "ts": "2018-08-31 09:31:00", "month": "08", "high": 14.38, "low": 14.38, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.38, "open": 14.38, "day": "31"} +{"volume": 2813, "symbol": "ARWR", "ts": "2018-08-31 09:33:00", "month": "08", "high": 14.37, "low": 14.37, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.37, "open": 14.37, "day": "31"} +{"volume": 11902, "symbol": "ARWR", "ts": "2018-08-31 09:34:00", "month": "08", "high": 14.4103, "low": 14.34, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.4103, "open": 14.34, "day": "31"} +{"volume": 6482, "symbol": "ARWR", "ts": "2018-08-31 09:36:00", "month": "08", "high": 14.41, "low": 14.41, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.41, "open": 14.41, "day": "31"} +{"volume": 7834, "symbol": "ARWR", "ts": "2018-08-31 09:37:00", "month": "08", "high": 14.545, "low": 14.5, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.545, "open": 14.5, "day": "31"} +{"volume": 7011, "symbol": "ARWR", "ts": "2018-08-31 09:39:00", "month": "08", "high": 14.5716, "low": 14.5716, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.5716, "open": 14.5716, "day": "31"} +{"volume": 4426, "symbol": "ARWR", "ts": "2018-08-31 09:40:00", "month": "08", "high": 14.4801, "low": 14.4801, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.4801, "open": 14.4801, "day": "31"} +{"volume": 7005, "symbol": "ARWR", "ts": "2018-08-31 09:42:00", "month": "08", "high": 14.57, "low": 14.5287, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.57, "open": 14.5287, "day": "31"} +{"volume": 1800, "symbol": "ARWR", "ts": "2018-08-31 09:43:00", "month": "08", "high": 14.555, "low": 14.555, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.555, "open": 14.555, "day": "31"} +{"volume": 5002, "symbol": "ARWR", "ts": "2018-08-31 09:44:00", "month": "08", "high": 14.54, "low": 14.54, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.54, "open": 14.54, "day": "31"} +{"volume": 4259, "symbol": "ARWR", "ts": "2018-08-31 09:46:00", "month": "08", "high": 14.51, "low": 14.51, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.51, "open": 14.51, "day": "31"} +{"volume": 601, "symbol": "ARWR", "ts": "2018-08-31 09:47:00", "month": "08", "high": 14.44, "low": 14.44, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.44, "open": 14.44, "day": "31"} +{"volume": 4002, "symbol": "ARWR", "ts": "2018-08-31 09:49:00", "month": "08", "high": 14.5077, "low": 14.5077, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.5077, "open": 14.5077, "day": "31"} +{"volume": 4775, "symbol": "ARWR", "ts": "2018-08-31 09:50:00", "month": "08", "high": 14.5449, "low": 14.5449, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.5449, "open": 14.5449, "day": "31"} +{"volume": 6424, "symbol": "ARWR", "ts": "2018-08-31 09:51:00", "month": "08", "high": 14.575, "low": 14.575, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.575, "open": 14.575, "day": "31"} +{"volume": 11645, "symbol": "ARWR", "ts": "2018-08-31 09:52:00", "month": "08", "high": 14.62, "low": 14.6162, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.62, "open": 14.6162, "day": "31"} +{"volume": 4930, "symbol": "ARWR", "ts": "2018-08-31 09:53:00", "month": "08", "high": 14.68, "low": 14.68, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.68, "open": 14.68, "day": "31"} +{"volume": 2216, "symbol": "ARWR", "ts": "2018-08-31 09:54:00", "month": "08", "high": 14.6, "low": 14.58, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.58, "open": 14.6, "day": "31"} +{"volume": 1802, "symbol": "ARWR", "ts": "2018-08-31 09:57:00", "month": "08", "high": 14.51, "low": 14.51, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.51, "open": 14.51, "day": "31"} +{"volume": 2302, "symbol": "ARWR", "ts": "2018-08-31 09:58:00", "month": "08", "high": 14.5, "low": 14.5, "key": "ARWR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.5, "open": 14.5, "day": "31"} +{"volume": 507, "symbol": "MRTX", "ts": "2018-08-31 09:33:00", "month": "08", "high": 57.35, "low": 57.35, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.35, "open": 57.35, "day": "31"} +{"volume": 1392, "symbol": "MRTX", "ts": "2018-08-31 09:36:00", "month": "08", "high": 57.6, "low": 57.6, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.6, "open": 57.6, "day": "31"} +{"volume": 1122, "symbol": "MRTX", "ts": "2018-08-31 09:37:00", "month": "08", "high": 57.25, "low": 57.25, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.25, "open": 57.25, "day": "31"} +{"volume": 108, "symbol": "MRTX", "ts": "2018-08-31 09:39:00", "month": "08", "high": 57.5, "low": 57.5, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.5, "open": 57.5, "day": "31"} +{"volume": 1940, "symbol": "MRTX", "ts": "2018-08-31 09:41:00", "month": "08", "high": 57.55, "low": 57.55, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.55, "open": 57.55, "day": "31"} +{"volume": 156, "symbol": "MRTX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 57.35, "low": 57.35, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.35, "open": 57.35, "day": "31"} +{"volume": 704, "symbol": "MRTX", "ts": "2018-08-31 09:44:00", "month": "08", "high": 57.4, "low": 57.4, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.4, "open": 57.4, "day": "31"} +{"volume": 2162, "symbol": "MRTX", "ts": "2018-08-31 09:46:00", "month": "08", "high": 57.15, "low": 57.15, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.15, "open": 57.15, "day": "31"} +{"volume": 968, "symbol": "MRTX", "ts": "2018-08-31 09:47:00", "month": "08", "high": 57.1, "low": 57.1, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.1, "open": 57.1, "day": "31"} +{"volume": 1947, "symbol": "MRTX", "ts": "2018-08-31 09:50:00", "month": "08", "high": 56.95, "low": 56.95, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.95, "open": 56.95, "day": "31"} +{"volume": 214, "symbol": "MRTX", "ts": "2018-08-31 09:51:00", "month": "08", "high": 56.925, "low": 56.925, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.925, "open": 56.925, "day": "31"} +{"volume": 102, "symbol": "MRTX", "ts": "2018-08-31 09:52:00", "month": "08", "high": 57.0, "low": 57.0, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.0, "open": 57.0, "day": "31"} +{"volume": 973, "symbol": "MRTX", "ts": "2018-08-31 09:55:00", "month": "08", "high": 56.825, "low": 56.825, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.825, "open": 56.825, "day": "31"} +{"volume": 200, "symbol": "MRTX", "ts": "2018-08-31 09:57:00", "month": "08", "high": 56.8, "low": 56.8, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.8, "open": 56.8, "day": "31"} +{"volume": 744, "symbol": "MRTX", "ts": "2018-08-31 09:59:00", "month": "08", "high": 56.65, "low": 56.65, "key": "MRTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.65, "open": 56.65, "day": "31"} +{"volume": 0, "symbol": "WWE", "ts": "2018-08-31 09:31:00", "month": "08", "high": 85.96, "low": 85.96, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 85.96, "open": 85.96, "day": "31"} +{"volume": 1431, "symbol": "WWE", "ts": "2018-08-31 09:33:00", "month": "08", "high": 85.793, "low": 85.793, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 85.793, "open": 85.793, "day": "31"} +{"volume": 1285, "symbol": "WWE", "ts": "2018-08-31 09:34:00", "month": "08", "high": 85.9311, "low": 85.9311, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 85.9311, "open": 85.9311, "day": "31"} +{"volume": 2562, "symbol": "WWE", "ts": "2018-08-31 09:35:00", "month": "08", "high": 86.06, "low": 86.06, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.06, "open": 86.06, "day": "31"} +{"volume": 1455, "symbol": "WWE", "ts": "2018-08-31 09:37:00", "month": "08", "high": 86.1567, "low": 86.1567, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.1567, "open": 86.1567, "day": "31"} +{"volume": 1239, "symbol": "WWE", "ts": "2018-08-31 09:38:00", "month": "08", "high": 86.46, "low": 86.46, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.46, "open": 86.46, "day": "31"} +{"volume": 2721, "symbol": "WWE", "ts": "2018-08-31 09:40:00", "month": "08", "high": 86.37, "low": 86.37, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.37, "open": 86.37, "day": "31"} +{"volume": 464, "symbol": "WWE", "ts": "2018-08-31 09:41:00", "month": "08", "high": 86.5752, "low": 86.5752, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.5752, "open": 86.5752, "day": "31"} +{"volume": 3752, "symbol": "WWE", "ts": "2018-08-31 09:44:00", "month": "08", "high": 86.444, "low": 86.41, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.444, "open": 86.41, "day": "31"} +{"volume": 338, "symbol": "WWE", "ts": "2018-08-31 09:45:00", "month": "08", "high": 86.01, "low": 86.01, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.01, "open": 86.01, "day": "31"} +{"volume": 15092, "symbol": "WWE", "ts": "2018-08-31 09:47:00", "month": "08", "high": 86.23, "low": 86.18, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.18, "open": 86.23, "day": "31"} +{"volume": 4198, "symbol": "WWE", "ts": "2018-08-31 09:48:00", "month": "08", "high": 86.15, "low": 86.15, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.15, "open": 86.15, "day": "31"} +{"volume": 7448, "symbol": "WWE", "ts": "2018-08-31 09:49:00", "month": "08", "high": 86.36, "low": 86.29, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.36, "open": 86.29, "day": "31"} +{"volume": 3740, "symbol": "WWE", "ts": "2018-08-31 09:50:00", "month": "08", "high": 86.66, "low": 86.66, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.66, "open": 86.66, "day": "31"} +{"volume": 2463, "symbol": "WWE", "ts": "2018-08-31 09:51:00", "month": "08", "high": 86.84, "low": 86.84, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.84, "open": 86.84, "day": "31"} +{"volume": 1480, "symbol": "WWE", "ts": "2018-08-31 09:52:00", "month": "08", "high": 86.86, "low": 86.86, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.86, "open": 86.86, "day": "31"} +{"volume": 1311, "symbol": "WWE", "ts": "2018-08-31 09:54:00", "month": "08", "high": 86.9, "low": 86.71, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.71, "open": 86.9, "day": "31"} +{"volume": 936, "symbol": "WWE", "ts": "2018-08-31 09:56:00", "month": "08", "high": 86.794, "low": 86.794, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.794, "open": 86.794, "day": "31"} +{"volume": 1773, "symbol": "WWE", "ts": "2018-08-31 09:58:00", "month": "08", "high": 86.6, "low": 86.6, "key": "WWE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.6, "open": 86.6, "day": "31"} +{"volume": 115, "symbol": "RFIL", "ts": "2018-08-31 09:35:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 709, "symbol": "RFIL", "ts": "2018-08-31 09:37:00", "month": "08", "high": 11.6, "low": 11.6, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.6, "open": 11.6, "day": "31"} +{"volume": 650, "symbol": "RFIL", "ts": "2018-08-31 09:39:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 1200, "symbol": "RFIL", "ts": "2018-08-31 09:42:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 1767, "symbol": "RFIL", "ts": "2018-08-31 09:43:00", "month": "08", "high": 11.591, "low": 11.591, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.591, "open": 11.591, "day": "31"} +{"volume": 1945, "symbol": "RFIL", "ts": "2018-08-31 09:48:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 4080, "symbol": "RFIL", "ts": "2018-08-31 09:55:00", "month": "08", "high": 11.8, "low": 11.8, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.8, "open": 11.8, "day": "31"} +{"volume": 5174, "symbol": "RFIL", "ts": "2018-08-31 09:58:00", "month": "08", "high": 11.8, "low": 11.7, "key": "RFIL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.7, "open": 11.8, "day": "31"} +{"volume": 913, "symbol": "MED", "ts": "2018-08-31 09:36:00", "month": "08", "high": 225.628, "low": 225.628, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 225.628, "open": 225.628, "day": "31"} +{"volume": 395, "symbol": "MED", "ts": "2018-08-31 09:41:00", "month": "08", "high": 225.7029, "low": 225.7029, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 225.7029, "open": 225.7029, "day": "31"} +{"volume": 2726, "symbol": "MED", "ts": "2018-08-31 09:42:00", "month": "08", "high": 227.64, "low": 227.64, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.64, "open": 227.64, "day": "31"} +{"volume": 1818, "symbol": "MED", "ts": "2018-08-31 09:44:00", "month": "08", "high": 228.9569, "low": 228.9569, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 228.9569, "open": 228.9569, "day": "31"} +{"volume": 1361, "symbol": "MED", "ts": "2018-08-31 09:47:00", "month": "08", "high": 227.68, "low": 227.68, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 227.68, "open": 227.68, "day": "31"} +{"volume": 4362, "symbol": "MED", "ts": "2018-08-31 09:48:00", "month": "08", "high": 226.77, "low": 226.5095, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 226.5095, "open": 226.77, "day": "31"} +{"volume": 501, "symbol": "MED", "ts": "2018-08-31 09:52:00", "month": "08", "high": 226.92, "low": 226.92, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 226.92, "open": 226.92, "day": "31"} +{"volume": 3434, "symbol": "MED", "ts": "2018-08-31 09:55:00", "month": "08", "high": 226.285, "low": 225.83, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 226.285, "open": 226.28, "day": "31"} +{"volume": 175, "symbol": "MED", "ts": "2018-08-31 09:57:00", "month": "08", "high": 226.3, "low": 226.3, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 226.3, "open": 226.3, "day": "31"} +{"volume": 243, "symbol": "MED", "ts": "2018-08-31 09:59:00", "month": "08", "high": 226.14, "low": 226.14, "key": "MED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 226.14, "open": 226.14, "day": "31"} +{"volume": 0, "symbol": "STAA", "ts": "2018-08-31 09:31:00", "month": "08", "high": 47.6747, "low": 47.6747, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.6747, "open": 47.6747, "day": "31"} +{"volume": 1552, "symbol": "STAA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 47.9, "low": 47.9, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.9, "open": 47.9, "day": "31"} +{"volume": 1814, "symbol": "STAA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 47.9, "low": 47.9, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.9, "open": 47.9, "day": "31"} +{"volume": 4723, "symbol": "STAA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 48.1, "low": 48.1, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.1, "open": 48.1, "day": "31"} +{"volume": 1731, "symbol": "STAA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 48.1, "low": 47.75, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.75, "open": 48.1, "day": "31"} +{"volume": 1956, "symbol": "STAA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 47.6, "low": 47.6, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.6, "open": 47.6, "day": "31"} +{"volume": 2147, "symbol": "STAA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 1924, "symbol": "STAA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 47.6, "low": 47.6, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.6, "open": 47.6, "day": "31"} +{"volume": 5660, "symbol": "STAA", "ts": "2018-08-31 09:45:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 1175, "symbol": "STAA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 47.55, "low": 47.55, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.55, "open": 47.55, "day": "31"} +{"volume": 1261, "symbol": "STAA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 47.55, "low": 47.55, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.55, "open": 47.55, "day": "31"} +{"volume": 345, "symbol": "STAA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 47.55, "low": 47.55, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.55, "open": 47.55, "day": "31"} +{"volume": 743, "symbol": "STAA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 47.716, "low": 47.716, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.716, "open": 47.716, "day": "31"} +{"volume": 919, "symbol": "STAA", "ts": "2018-08-31 09:54:00", "month": "08", "high": 47.7, "low": 47.7, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.7, "open": 47.7, "day": "31"} +{"volume": 7256, "symbol": "STAA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 47.7, "low": 47.7, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.7, "open": 47.7, "day": "31"} +{"volume": 7407, "symbol": "STAA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 47.85, "low": 47.85, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.85, "open": 47.85, "day": "31"} +{"volume": 1709, "symbol": "STAA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 47.95, "low": 47.95, "key": "STAA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 47.95, "open": 47.95, "day": "31"} +{"volume": 0, "symbol": "TRHC", "ts": "2018-08-31 09:31:00", "month": "08", "high": 87.22, "low": 87.22, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.22, "open": 87.22, "day": "31"} +{"volume": 944, "symbol": "TRHC", "ts": "2018-08-31 09:32:00", "month": "08", "high": 87.5596, "low": 87.5596, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.5596, "open": 87.5596, "day": "31"} +{"volume": 1914, "symbol": "TRHC", "ts": "2018-08-31 09:35:00", "month": "08", "high": 87.82, "low": 87.82, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.82, "open": 87.82, "day": "31"} +{"volume": 5007, "symbol": "TRHC", "ts": "2018-08-31 09:37:00", "month": "08", "high": 88.2744, "low": 88.05, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.05, "open": 88.2744, "day": "31"} +{"volume": 2576, "symbol": "TRHC", "ts": "2018-08-31 09:38:00", "month": "08", "high": 88.29, "low": 88.29, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.29, "open": 88.29, "day": "31"} +{"volume": 725, "symbol": "TRHC", "ts": "2018-08-31 09:40:00", "month": "08", "high": 88.3, "low": 88.3, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.3, "open": 88.3, "day": "31"} +{"volume": 429, "symbol": "TRHC", "ts": "2018-08-31 09:43:00", "month": "08", "high": 88.35, "low": 88.35, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.35, "open": 88.35, "day": "31"} +{"volume": 2655, "symbol": "TRHC", "ts": "2018-08-31 09:44:00", "month": "08", "high": 87.5, "low": 87.5, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.5, "open": 87.5, "day": "31"} +{"volume": 2419, "symbol": "TRHC", "ts": "2018-08-31 09:47:00", "month": "08", "high": 87.26, "low": 87.01, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.01, "open": 87.26, "day": "31"} +{"volume": 1402, "symbol": "TRHC", "ts": "2018-08-31 09:54:00", "month": "08", "high": 86.94, "low": 86.94, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.94, "open": 86.94, "day": "31"} +{"volume": 4062, "symbol": "TRHC", "ts": "2018-08-31 09:55:00", "month": "08", "high": 86.7387, "low": 86.7387, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.7387, "open": 86.7387, "day": "31"} +{"volume": 1638, "symbol": "TRHC", "ts": "2018-08-31 09:59:00", "month": "08", "high": 86.62, "low": 86.62, "key": "TRHC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.62, "open": 86.62, "day": "31"} +{"volume": 6005, "symbol": "I", "ts": "2018-08-31 09:31:00", "month": "08", "high": 21.61, "low": 21.61, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.61, "open": 21.61, "day": "31"} +{"volume": 4892, "symbol": "I", "ts": "2018-08-31 09:32:00", "month": "08", "high": 21.78, "low": 21.78, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.78, "open": 21.78, "day": "31"} +{"volume": 5250, "symbol": "I", "ts": "2018-08-31 09:33:00", "month": "08", "high": 21.86, "low": 21.86, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.86, "open": 21.86, "day": "31"} +{"volume": 8400, "symbol": "I", "ts": "2018-08-31 09:34:00", "month": "08", "high": 21.83, "low": 21.83, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.83, "open": 21.83, "day": "31"} +{"volume": 2646, "symbol": "I", "ts": "2018-08-31 09:35:00", "month": "08", "high": 21.8, "low": 21.8, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.8, "open": 21.8, "day": "31"} +{"volume": 1809, "symbol": "I", "ts": "2018-08-31 09:36:00", "month": "08", "high": 21.7, "low": 21.7, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.7, "open": 21.7, "day": "31"} +{"volume": 5359, "symbol": "I", "ts": "2018-08-31 09:37:00", "month": "08", "high": 21.75, "low": 21.75, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.75, "open": 21.75, "day": "31"} +{"volume": 3239, "symbol": "I", "ts": "2018-08-31 09:38:00", "month": "08", "high": 21.91, "low": 21.91, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.91, "open": 21.91, "day": "31"} +{"volume": 16847, "symbol": "I", "ts": "2018-08-31 09:39:00", "month": "08", "high": 22.06, "low": 21.98, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.06, "open": 21.98, "day": "31"} +{"volume": 12621, "symbol": "I", "ts": "2018-08-31 09:41:00", "month": "08", "high": 22.025, "low": 21.9056, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.9056, "open": 22.025, "day": "31"} +{"volume": 3774, "symbol": "I", "ts": "2018-08-31 09:42:00", "month": "08", "high": 21.9599, "low": 21.9599, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.9599, "open": 21.9599, "day": "31"} +{"volume": 4013, "symbol": "I", "ts": "2018-08-31 09:44:00", "month": "08", "high": 22.05, "low": 22.05, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.05, "open": 22.05, "day": "31"} +{"volume": 4200, "symbol": "I", "ts": "2018-08-31 09:45:00", "month": "08", "high": 22.023, "low": 22.023, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.023, "open": 22.023, "day": "31"} +{"volume": 5803, "symbol": "I", "ts": "2018-08-31 09:46:00", "month": "08", "high": 21.83, "low": 21.83, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.83, "open": 21.83, "day": "31"} +{"volume": 6801, "symbol": "I", "ts": "2018-08-31 09:47:00", "month": "08", "high": 21.94, "low": 21.94, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.94, "open": 21.94, "day": "31"} +{"volume": 11944, "symbol": "I", "ts": "2018-08-31 09:49:00", "month": "08", "high": 22.0099, "low": 21.85, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.0099, "open": 21.85, "day": "31"} +{"volume": 9191, "symbol": "I", "ts": "2018-08-31 09:50:00", "month": "08", "high": 22.0, "low": 21.9399, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.9399, "open": 22.0, "day": "31"} +{"volume": 1050, "symbol": "I", "ts": "2018-08-31 09:52:00", "month": "08", "high": 21.81, "low": 21.81, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.81, "open": 21.81, "day": "31"} +{"volume": 2612, "symbol": "I", "ts": "2018-08-31 09:53:00", "month": "08", "high": 21.8745, "low": 21.8745, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.8745, "open": 21.8745, "day": "31"} +{"volume": 1905, "symbol": "I", "ts": "2018-08-31 09:54:00", "month": "08", "high": 21.87, "low": 21.87, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.87, "open": 21.87, "day": "31"} +{"volume": 4737, "symbol": "I", "ts": "2018-08-31 09:56:00", "month": "08", "high": 21.92, "low": 21.9, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.92, "open": 21.9, "day": "31"} +{"volume": 1354, "symbol": "I", "ts": "2018-08-31 09:58:00", "month": "08", "high": 21.9749, "low": 21.9749, "key": "I_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 21.9749, "open": 21.9749, "day": "31"} +{"volume": 3201, "symbol": "ARQL", "ts": "2018-08-31 09:31:00", "month": "08", "high": 6.52, "low": 6.52, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.52, "open": 6.52, "day": "31"} +{"volume": 7298, "symbol": "ARQL", "ts": "2018-08-31 09:32:00", "month": "08", "high": 6.5299, "low": 6.5299, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.5299, "open": 6.5299, "day": "31"} +{"volume": 9716, "symbol": "ARQL", "ts": "2018-08-31 09:33:00", "month": "08", "high": 6.495, "low": 6.495, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.495, "open": 6.495, "day": "31"} +{"volume": 6922, "symbol": "ARQL", "ts": "2018-08-31 09:34:00", "month": "08", "high": 6.49, "low": 6.49, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.49, "open": 6.49, "day": "31"} +{"volume": 5645, "symbol": "ARQL", "ts": "2018-08-31 09:36:00", "month": "08", "high": 6.5245, "low": 6.5245, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.5245, "open": 6.5245, "day": "31"} +{"volume": 2291, "symbol": "ARQL", "ts": "2018-08-31 09:38:00", "month": "08", "high": 6.5008, "low": 6.5008, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.5008, "open": 6.5008, "day": "31"} +{"volume": 30351, "symbol": "ARQL", "ts": "2018-08-31 09:39:00", "month": "08", "high": 6.59, "low": 6.52, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.58, "open": 6.52, "day": "31"} +{"volume": 41160, "symbol": "ARQL", "ts": "2018-08-31 09:40:00", "month": "08", "high": 6.63, "low": 6.6, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.63, "open": 6.6, "day": "31"} +{"volume": 10275, "symbol": "ARQL", "ts": "2018-08-31 09:41:00", "month": "08", "high": 6.6106, "low": 6.6106, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.6106, "open": 6.6106, "day": "31"} +{"volume": 42635, "symbol": "ARQL", "ts": "2018-08-31 09:42:00", "month": "08", "high": 6.6197, "low": 6.59, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.6197, "open": 6.59, "day": "31"} +{"volume": 4935, "symbol": "ARQL", "ts": "2018-08-31 09:43:00", "month": "08", "high": 6.65, "low": 6.65, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.65, "open": 6.65, "day": "31"} +{"volume": 12963, "symbol": "ARQL", "ts": "2018-08-31 09:44:00", "month": "08", "high": 6.6771, "low": 6.64, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.64, "open": 6.6771, "day": "31"} +{"volume": 7591, "symbol": "ARQL", "ts": "2018-08-31 09:47:00", "month": "08", "high": 6.6666, "low": 6.6666, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.6666, "open": 6.6666, "day": "31"} +{"volume": 4910, "symbol": "ARQL", "ts": "2018-08-31 09:48:00", "month": "08", "high": 6.6699, "low": 6.6699, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.6699, "open": 6.6699, "day": "31"} +{"volume": 1796, "symbol": "ARQL", "ts": "2018-08-31 09:49:00", "month": "08", "high": 6.68, "low": 6.68, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.68, "open": 6.68, "day": "31"} +{"volume": 10888, "symbol": "ARQL", "ts": "2018-08-31 09:50:00", "month": "08", "high": 6.705, "low": 6.68, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.705, "open": 6.68, "day": "31"} +{"volume": 10762, "symbol": "ARQL", "ts": "2018-08-31 09:51:00", "month": "08", "high": 6.68, "low": 6.66, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.66, "open": 6.68, "day": "31"} +{"volume": 9726, "symbol": "ARQL", "ts": "2018-08-31 09:52:00", "month": "08", "high": 6.67, "low": 6.67, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.67, "open": 6.67, "day": "31"} +{"volume": 4415, "symbol": "ARQL", "ts": "2018-08-31 09:54:00", "month": "08", "high": 6.67, "low": 6.66, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.66, "open": 6.67, "day": "31"} +{"volume": 3060, "symbol": "ARQL", "ts": "2018-08-31 09:56:00", "month": "08", "high": 6.6416, "low": 6.6416, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.6416, "open": 6.6416, "day": "31"} +{"volume": 4173, "symbol": "ARQL", "ts": "2018-08-31 09:58:00", "month": "08", "high": 6.6468, "low": 6.645, "key": "ARQL_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.6468, "open": 6.645, "day": "31"} +{"volume": 40317, "symbol": "DNR", "ts": "2018-08-31 09:31:00", "month": "08", "high": 5.59, "low": 5.5852, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.59, "open": 5.59, "day": "31"} +{"volume": 75899, "symbol": "DNR", "ts": "2018-08-31 09:32:00", "month": "08", "high": 5.605, "low": 5.59, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.595, "open": 5.59, "day": "31"} +{"volume": 257943, "symbol": "DNR", "ts": "2018-08-31 09:33:00", "month": "08", "high": 5.58, "low": 5.55, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.56, "open": 5.58, "day": "31"} +{"volume": 20769, "symbol": "DNR", "ts": "2018-08-31 09:34:00", "month": "08", "high": 5.59, "low": 5.585, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.59, "open": 5.585, "day": "31"} +{"volume": 64076, "symbol": "DNR", "ts": "2018-08-31 09:35:00", "month": "08", "high": 5.58, "low": 5.56, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.58, "open": 5.56, "day": "31"} +{"volume": 64117, "symbol": "DNR", "ts": "2018-08-31 09:36:00", "month": "08", "high": 5.585, "low": 5.565, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.565, "open": 5.58, "day": "31"} +{"volume": 74268, "symbol": "DNR", "ts": "2018-08-31 09:37:00", "month": "08", "high": 5.575, "low": 5.555, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.57, "open": 5.575, "day": "31"} +{"volume": 24135, "symbol": "DNR", "ts": "2018-08-31 09:38:00", "month": "08", "high": 5.56, "low": 5.555, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.555, "open": 5.56, "day": "31"} +{"volume": 23230, "symbol": "DNR", "ts": "2018-08-31 09:39:00", "month": "08", "high": 5.54, "low": 5.54, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.54, "open": 5.54, "day": "31"} +{"volume": 36875, "symbol": "DNR", "ts": "2018-08-31 09:40:00", "month": "08", "high": 5.52, "low": 5.52, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.52, "open": 5.52, "day": "31"} +{"volume": 122394, "symbol": "DNR", "ts": "2018-08-31 09:41:00", "month": "08", "high": 5.53, "low": 5.49, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.51, "open": 5.53, "day": "31"} +{"volume": 260449, "symbol": "DNR", "ts": "2018-08-31 09:42:00", "month": "08", "high": 5.51, "low": 5.505, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.51, "open": 5.51, "day": "31"} +{"volume": 91033, "symbol": "DNR", "ts": "2018-08-31 09:43:00", "month": "08", "high": 5.52, "low": 5.5, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.52, "open": 5.51, "day": "31"} +{"volume": 40348, "symbol": "DNR", "ts": "2018-08-31 09:44:00", "month": "08", "high": 5.54, "low": 5.53, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.54, "open": 5.53, "day": "31"} +{"volume": 36839, "symbol": "DNR", "ts": "2018-08-31 09:45:00", "month": "08", "high": 5.535, "low": 5.53, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.53, "open": 5.535, "day": "31"} +{"volume": 56257, "symbol": "DNR", "ts": "2018-08-31 09:46:00", "month": "08", "high": 5.52, "low": 5.4971, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.4971, "open": 5.52, "day": "31"} +{"volume": 17721, "symbol": "DNR", "ts": "2018-08-31 09:47:00", "month": "08", "high": 5.49, "low": 5.49, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.49, "day": "31"} +{"volume": 67583, "symbol": "DNR", "ts": "2018-08-31 09:48:00", "month": "08", "high": 5.49, "low": 5.47, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.48, "open": 5.49, "day": "31"} +{"volume": 39702, "symbol": "DNR", "ts": "2018-08-31 09:49:00", "month": "08", "high": 5.49, "low": 5.47, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.47, "open": 5.49, "day": "31"} +{"volume": 54879, "symbol": "DNR", "ts": "2018-08-31 09:50:00", "month": "08", "high": 5.47, "low": 5.46, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.46, "open": 5.47, "day": "31"} +{"volume": 81246, "symbol": "DNR", "ts": "2018-08-31 09:51:00", "month": "08", "high": 5.45, "low": 5.44, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.445, "open": 5.44, "day": "31"} +{"volume": 39704, "symbol": "DNR", "ts": "2018-08-31 09:52:00", "month": "08", "high": 5.4667, "low": 5.46, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.4667, "open": 5.46, "day": "31"} +{"volume": 103759, "symbol": "DNR", "ts": "2018-08-31 09:53:00", "month": "08", "high": 5.49, "low": 5.47, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.4865, "open": 5.47, "day": "31"} +{"volume": 38122, "symbol": "DNR", "ts": "2018-08-31 09:54:00", "month": "08", "high": 5.49, "low": 5.47, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.47, "day": "31"} +{"volume": 76665, "symbol": "DNR", "ts": "2018-08-31 09:55:00", "month": "08", "high": 5.495, "low": 5.48, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.495, "open": 5.48, "day": "31"} +{"volume": 37687, "symbol": "DNR", "ts": "2018-08-31 09:56:00", "month": "08", "high": 5.5, "low": 5.49, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.495, "open": 5.49, "day": "31"} +{"volume": 13769, "symbol": "DNR", "ts": "2018-08-31 09:57:00", "month": "08", "high": 5.486, "low": 5.486, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.486, "open": 5.486, "day": "31"} +{"volume": 16748, "symbol": "DNR", "ts": "2018-08-31 09:58:00", "month": "08", "high": 5.48, "low": 5.47, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.47, "open": 5.48, "day": "31"} +{"volume": 107747, "symbol": "DNR", "ts": "2018-08-31 09:59:00", "month": "08", "high": 5.465, "low": 5.45, "key": "DNR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.45, "open": 5.46, "day": "31"} +{"volume": 5027, "symbol": "CVNA", "ts": "2018-08-31 09:32:00", "month": "08", "high": 62.09, "low": 62.09, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.09, "open": 62.09, "day": "31"} +{"volume": 1765, "symbol": "CVNA", "ts": "2018-08-31 09:33:00", "month": "08", "high": 61.87, "low": 61.87, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.87, "open": 61.87, "day": "31"} +{"volume": 381, "symbol": "CVNA", "ts": "2018-08-31 09:35:00", "month": "08", "high": 62.0, "low": 62.0, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.0, "open": 62.0, "day": "31"} +{"volume": 1191, "symbol": "CVNA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 61.8, "low": 61.8, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.8, "open": 61.8, "day": "31"} +{"volume": 995, "symbol": "CVNA", "ts": "2018-08-31 09:38:00", "month": "08", "high": 61.76, "low": 61.76, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.76, "open": 61.76, "day": "31"} +{"volume": 7841, "symbol": "CVNA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 62.39, "low": 61.9575, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.9575, "open": 62.39, "day": "31"} +{"volume": 4758, "symbol": "CVNA", "ts": "2018-08-31 09:44:00", "month": "08", "high": 61.87, "low": 61.6, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.6, "open": 61.87, "day": "31"} +{"volume": 3009, "symbol": "CVNA", "ts": "2018-08-31 09:45:00", "month": "08", "high": 61.6001, "low": 61.6001, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.6001, "open": 61.6001, "day": "31"} +{"volume": 18284, "symbol": "CVNA", "ts": "2018-08-31 09:46:00", "month": "08", "high": 61.56, "low": 61.56, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.56, "open": 61.56, "day": "31"} +{"volume": 2177, "symbol": "CVNA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 61.8371, "low": 61.8, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.8371, "open": 61.8, "day": "31"} +{"volume": 798, "symbol": "CVNA", "ts": "2018-08-31 09:51:00", "month": "08", "high": 61.92, "low": 61.92, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.92, "open": 61.92, "day": "31"} +{"volume": 364, "symbol": "CVNA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 61.76, "low": 61.76, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.76, "open": 61.76, "day": "31"} +{"volume": 3366, "symbol": "CVNA", "ts": "2018-08-31 09:53:00", "month": "08", "high": 62.08, "low": 62.08, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.08, "open": 62.08, "day": "31"} +{"volume": 3603, "symbol": "CVNA", "ts": "2018-08-31 09:54:00", "month": "08", "high": 62.08, "low": 61.98, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.98, "open": 62.08, "day": "31"} +{"volume": 1369, "symbol": "CVNA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 62.1, "low": 62.1, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.1, "open": 62.1, "day": "31"} +{"volume": 1450, "symbol": "CVNA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 62.22, "low": 62.22, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.22, "open": 62.22, "day": "31"} +{"volume": 3653, "symbol": "CVNA", "ts": "2018-08-31 09:58:00", "month": "08", "high": 62.25, "low": 62.25, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.25, "open": 62.25, "day": "31"} +{"volume": 4874, "symbol": "CVNA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 62.14, "low": 62.098, "key": "CVNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.098, "open": 62.14, "day": "31"} +{"volume": 2322, "symbol": "BOOT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 29.395, "low": 29.395, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.395, "open": 29.395, "day": "31"} +{"volume": 1478, "symbol": "BOOT", "ts": "2018-08-31 09:34:00", "month": "08", "high": 29.27, "low": 29.27, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.27, "open": 29.27, "day": "31"} +{"volume": 2311, "symbol": "BOOT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 29.35, "low": 29.35, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.35, "open": 29.35, "day": "31"} +{"volume": 1400, "symbol": "BOOT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 29.33, "low": 29.33, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.33, "open": 29.33, "day": "31"} +{"volume": 5452, "symbol": "BOOT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 29.24, "low": 29.22, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.22, "open": 29.24, "day": "31"} +{"volume": 4554, "symbol": "BOOT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 29.27, "low": 29.21, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.27, "open": 29.21, "day": "31"} +{"volume": 1930, "symbol": "BOOT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 29.32, "low": 29.32, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.32, "open": 29.32, "day": "31"} +{"volume": 1700, "symbol": "BOOT", "ts": "2018-08-31 09:44:00", "month": "08", "high": 29.45, "low": 29.45, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.45, "open": 29.45, "day": "31"} +{"volume": 4241, "symbol": "BOOT", "ts": "2018-08-31 09:45:00", "month": "08", "high": 29.4, "low": 29.4, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.4, "open": 29.4, "day": "31"} +{"volume": 1854, "symbol": "BOOT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 29.57, "low": 29.57, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.57, "open": 29.57, "day": "31"} +{"volume": 3769, "symbol": "BOOT", "ts": "2018-08-31 09:48:00", "month": "08", "high": 29.47, "low": 29.47, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.47, "open": 29.47, "day": "31"} +{"volume": 3082, "symbol": "BOOT", "ts": "2018-08-31 09:49:00", "month": "08", "high": 29.4, "low": 29.4, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.4, "open": 29.4, "day": "31"} +{"volume": 1263, "symbol": "BOOT", "ts": "2018-08-31 09:50:00", "month": "08", "high": 29.33, "low": 29.33, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.33, "open": 29.33, "day": "31"} +{"volume": 203, "symbol": "BOOT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 29.33, "low": 29.33, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.33, "open": 29.33, "day": "31"} +{"volume": 2914, "symbol": "BOOT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 29.39, "low": 29.39, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.39, "open": 29.39, "day": "31"} +{"volume": 200, "symbol": "BOOT", "ts": "2018-08-31 09:58:00", "month": "08", "high": 29.32, "low": 29.32, "key": "BOOT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.32, "open": 29.32, "day": "31"} +{"volume": 277335, "symbol": "SQ", "ts": "2018-08-31 09:31:00", "month": "08", "high": 87.9, "low": 87.77, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.8439, "open": 87.89, "day": "31"} +{"volume": 72975, "symbol": "SQ", "ts": "2018-08-31 09:32:00", "month": "08", "high": 88.3044, "low": 87.801, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.3044, "open": 87.801, "day": "31"} +{"volume": 106213, "symbol": "SQ", "ts": "2018-08-31 09:33:00", "month": "08", "high": 88.75, "low": 88.35, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.5588, "open": 88.35, "day": "31"} +{"volume": 89194, "symbol": "SQ", "ts": "2018-08-31 09:34:00", "month": "08", "high": 88.9, "low": 88.52, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.85, "open": 88.61, "day": "31"} +{"volume": 93106, "symbol": "SQ", "ts": "2018-08-31 09:35:00", "month": "08", "high": 88.818, "low": 88.59, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.6177, "open": 88.818, "day": "31"} +{"volume": 71514, "symbol": "SQ", "ts": "2018-08-31 09:36:00", "month": "08", "high": 88.91, "low": 88.5845, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.862, "open": 88.5845, "day": "31"} +{"volume": 151038, "symbol": "SQ", "ts": "2018-08-31 09:37:00", "month": "08", "high": 89.2402, "low": 88.8684, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.17, "open": 88.9106, "day": "31"} +{"volume": 107946, "symbol": "SQ", "ts": "2018-08-31 09:38:00", "month": "08", "high": 89.64, "low": 89.12, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.5, "open": 89.21, "day": "31"} +{"volume": 75323, "symbol": "SQ", "ts": "2018-08-31 09:39:00", "month": "08", "high": 89.53, "low": 89.31, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.35, "open": 89.4, "day": "31"} +{"volume": 60954, "symbol": "SQ", "ts": "2018-08-31 09:40:00", "month": "08", "high": 89.34, "low": 89.16, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.29, "open": 89.2, "day": "31"} +{"volume": 70673, "symbol": "SQ", "ts": "2018-08-31 09:41:00", "month": "08", "high": 89.21, "low": 88.95, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.0101, "open": 89.21, "day": "31"} +{"volume": 49824, "symbol": "SQ", "ts": "2018-08-31 09:42:00", "month": "08", "high": 88.9648, "low": 88.825, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.8765, "open": 88.96, "day": "31"} +{"volume": 50850, "symbol": "SQ", "ts": "2018-08-31 09:43:00", "month": "08", "high": 88.92, "low": 88.67, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.715, "open": 88.8799, "day": "31"} +{"volume": 100725, "symbol": "SQ", "ts": "2018-08-31 09:44:00", "month": "08", "high": 88.7065, "low": 88.38, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.4596, "open": 88.7065, "day": "31"} +{"volume": 61951, "symbol": "SQ", "ts": "2018-08-31 09:45:00", "month": "08", "high": 88.35, "low": 88.2, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.24, "open": 88.35, "day": "31"} +{"volume": 84054, "symbol": "SQ", "ts": "2018-08-31 09:46:00", "month": "08", "high": 88.36, "low": 88.1352, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.27, "open": 88.19, "day": "31"} +{"volume": 40531, "symbol": "SQ", "ts": "2018-08-31 09:47:00", "month": "08", "high": 88.25, "low": 88.12, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.21, "open": 88.165, "day": "31"} +{"volume": 54633, "symbol": "SQ", "ts": "2018-08-31 09:48:00", "month": "08", "high": 88.26, "low": 88.015, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.015, "open": 88.22, "day": "31"} +{"volume": 162155, "symbol": "SQ", "ts": "2018-08-31 09:49:00", "month": "08", "high": 88.173, "low": 87.76, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.15, "open": 88.0, "day": "31"} +{"volume": 104420, "symbol": "SQ", "ts": "2018-08-31 09:50:00", "month": "08", "high": 88.47, "low": 88.12, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.47, "open": 88.1755, "day": "31"} +{"volume": 96871, "symbol": "SQ", "ts": "2018-08-31 09:51:00", "month": "08", "high": 88.61, "low": 88.3601, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.4325, "open": 88.41, "day": "31"} +{"volume": 52216, "symbol": "SQ", "ts": "2018-08-31 09:52:00", "month": "08", "high": 88.51, "low": 88.36, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.51, "open": 88.43, "day": "31"} +{"volume": 51457, "symbol": "SQ", "ts": "2018-08-31 09:53:00", "month": "08", "high": 88.75, "low": 88.5474, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.5474, "open": 88.61, "day": "31"} +{"volume": 58338, "symbol": "SQ", "ts": "2018-08-31 09:54:00", "month": "08", "high": 88.8, "low": 88.637, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.637, "open": 88.74, "day": "31"} +{"volume": 53469, "symbol": "SQ", "ts": "2018-08-31 09:55:00", "month": "08", "high": 88.75, "low": 88.421, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.5, "open": 88.75, "day": "31"} +{"volume": 24597, "symbol": "SQ", "ts": "2018-08-31 09:56:00", "month": "08", "high": 88.61, "low": 88.4501, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.4501, "open": 88.61, "day": "31"} +{"volume": 54385, "symbol": "SQ", "ts": "2018-08-31 09:57:00", "month": "08", "high": 88.42, "low": 88.261, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.3301, "open": 88.42, "day": "31"} +{"volume": 100679, "symbol": "SQ", "ts": "2018-08-31 09:58:00", "month": "08", "high": 88.28, "low": 88.0, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.1508, "open": 88.28, "day": "31"} +{"volume": 37609, "symbol": "SQ", "ts": "2018-08-31 09:59:00", "month": "08", "high": 88.14, "low": 87.971, "key": "SQ_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.971, "open": 88.14, "day": "31"} +{"volume": 6912, "symbol": "EGAN", "ts": "2018-08-31 09:31:00", "month": "08", "high": 15.1, "low": 15.1, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.1, "open": 15.1, "day": "31"} +{"volume": 2767, "symbol": "EGAN", "ts": "2018-08-31 09:33:00", "month": "08", "high": 14.95, "low": 14.95, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.95, "open": 14.95, "day": "31"} +{"volume": 5500, "symbol": "EGAN", "ts": "2018-08-31 09:34:00", "month": "08", "high": 14.949, "low": 14.949, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.949, "open": 14.949, "day": "31"} +{"volume": 976, "symbol": "EGAN", "ts": "2018-08-31 09:36:00", "month": "08", "high": 14.9, "low": 14.9, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.9, "open": 14.9, "day": "31"} +{"volume": 1015, "symbol": "EGAN", "ts": "2018-08-31 09:38:00", "month": "08", "high": 15.0, "low": 15.0, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.0, "day": "31"} +{"volume": 2070, "symbol": "EGAN", "ts": "2018-08-31 09:40:00", "month": "08", "high": 15.0, "low": 15.0, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.0, "day": "31"} +{"volume": 2027, "symbol": "EGAN", "ts": "2018-08-31 09:41:00", "month": "08", "high": 15.0, "low": 15.0, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.0, "day": "31"} +{"volume": 1191, "symbol": "EGAN", "ts": "2018-08-31 09:43:00", "month": "08", "high": 14.9188, "low": 14.9188, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.9188, "open": 14.9188, "day": "31"} +{"volume": 170, "symbol": "EGAN", "ts": "2018-08-31 09:44:00", "month": "08", "high": 14.94, "low": 14.94, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.94, "day": "31"} +{"volume": 15568, "symbol": "EGAN", "ts": "2018-08-31 09:46:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 3513, "symbol": "EGAN", "ts": "2018-08-31 09:47:00", "month": "08", "high": 14.6765, "low": 14.6425, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.6765, "open": 14.6425, "day": "31"} +{"volume": 2619, "symbol": "EGAN", "ts": "2018-08-31 09:49:00", "month": "08", "high": 14.75, "low": 14.75, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.75, "open": 14.75, "day": "31"} +{"volume": 1373, "symbol": "EGAN", "ts": "2018-08-31 09:51:00", "month": "08", "high": 14.8, "low": 14.8, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.8, "open": 14.8, "day": "31"} +{"volume": 6844, "symbol": "EGAN", "ts": "2018-08-31 09:52:00", "month": "08", "high": 14.65, "low": 14.65, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.65, "open": 14.65, "day": "31"} +{"volume": 13612, "symbol": "EGAN", "ts": "2018-08-31 09:54:00", "month": "08", "high": 14.55, "low": 14.45, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.45, "open": 14.55, "day": "31"} +{"volume": 8828, "symbol": "EGAN", "ts": "2018-08-31 09:55:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 2489, "symbol": "EGAN", "ts": "2018-08-31 09:57:00", "month": "08", "high": 14.5999, "low": 14.5, "key": "EGAN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.5, "open": 14.5999, "day": "31"} +{"volume": 3649, "symbol": "RCKT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 23.77, "low": 23.77, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.77, "open": 23.77, "day": "31"} +{"volume": 802, "symbol": "RCKT", "ts": "2018-08-31 09:35:00", "month": "08", "high": 23.63, "low": 23.63, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.63, "open": 23.63, "day": "31"} +{"volume": 820, "symbol": "RCKT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 23.67, "low": 23.67, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.67, "open": 23.67, "day": "31"} +{"volume": 550, "symbol": "RCKT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 23.635, "low": 23.635, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.635, "open": 23.635, "day": "31"} +{"volume": 121, "symbol": "RCKT", "ts": "2018-08-31 09:45:00", "month": "08", "high": 23.3773, "low": 23.3773, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.3773, "open": 23.3773, "day": "31"} +{"volume": 2514, "symbol": "RCKT", "ts": "2018-08-31 09:48:00", "month": "08", "high": 23.335, "low": 23.335, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.335, "open": 23.335, "day": "31"} +{"volume": 910, "symbol": "RCKT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 23.645, "low": 23.645, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.645, "open": 23.645, "day": "31"} +{"volume": 100, "symbol": "RCKT", "ts": "2018-08-31 09:55:00", "month": "08", "high": 23.65, "low": 23.65, "key": "RCKT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.65, "open": 23.65, "day": "31"} +{"volume": 728, "symbol": "NGVC", "ts": "2018-08-31 09:47:00", "month": "08", "high": 18.91, "low": 18.91, "key": "NGVC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.91, "open": 18.91, "day": "31"} +{"volume": 1257, "symbol": "NGVC", "ts": "2018-08-31 09:51:00", "month": "08", "high": 18.92, "low": 18.92, "key": "NGVC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.92, "open": 18.92, "day": "31"} +{"volume": 471, "symbol": "NGVC", "ts": "2018-08-31 09:56:00", "month": "08", "high": 19.03, "low": 19.03, "key": "NGVC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.03, "open": 19.03, "day": "31"} +{"volume": 1949, "symbol": "NGVC", "ts": "2018-08-31 09:57:00", "month": "08", "high": 18.9101, "low": 18.9101, "key": "NGVC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.9101, "open": 18.9101, "day": "31"} +{"volume": 2431, "symbol": "NGVC", "ts": "2018-08-31 09:58:00", "month": "08", "high": 18.95, "low": 18.95, "key": "NGVC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.95, "open": 18.95, "day": "31"} +{"volume": 0, "symbol": "SHSP", "ts": "2018-08-31 09:35:00", "month": "08", "high": 13.7081, "low": 13.7081, "key": "SHSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.7081, "open": 13.7081, "day": "31"} +{"volume": 127, "symbol": "SHSP", "ts": "2018-08-31 09:48:00", "month": "08", "high": 13.92, "low": 13.92, "key": "SHSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.92, "open": 13.92, "day": "31"} +{"volume": 413, "symbol": "SHSP", "ts": "2018-08-31 09:52:00", "month": "08", "high": 13.9, "low": 13.9, "key": "SHSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 623, "symbol": "SHSP", "ts": "2018-08-31 09:59:00", "month": "08", "high": 13.9907, "low": 13.9907, "key": "SHSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9907, "open": 13.9907, "day": "31"} +{"volume": 100, "symbol": "VNCE", "ts": "2018-08-31 09:44:00", "month": "08", "high": 20.9766, "low": 20.9766, "key": "VNCE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.9766, "open": 20.9766, "day": "31"} +{"volume": 457, "symbol": "VNCE", "ts": "2018-08-31 09:47:00", "month": "08", "high": 20.96, "low": 20.96, "key": "VNCE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.96, "open": 20.96, "day": "31"} +{"volume": 8471, "symbol": "CRC", "ts": "2018-08-31 09:31:00", "month": "08", "high": 40.3329, "low": 40.3329, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.3329, "open": 40.3329, "day": "31"} +{"volume": 5253, "symbol": "CRC", "ts": "2018-08-31 09:32:00", "month": "08", "high": 40.37, "low": 40.37, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.37, "open": 40.37, "day": "31"} +{"volume": 16857, "symbol": "CRC", "ts": "2018-08-31 09:33:00", "month": "08", "high": 40.42, "low": 40.26, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.26, "open": 40.42, "day": "31"} +{"volume": 11626, "symbol": "CRC", "ts": "2018-08-31 09:34:00", "month": "08", "high": 39.953, "low": 39.91, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.953, "open": 39.91, "day": "31"} +{"volume": 14208, "symbol": "CRC", "ts": "2018-08-31 09:35:00", "month": "08", "high": 39.9, "low": 39.9, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.9, "open": 39.9, "day": "31"} +{"volume": 7920, "symbol": "CRC", "ts": "2018-08-31 09:36:00", "month": "08", "high": 40.0911, "low": 39.8407, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.0911, "open": 39.8407, "day": "31"} +{"volume": 11174, "symbol": "CRC", "ts": "2018-08-31 09:37:00", "month": "08", "high": 40.218, "low": 39.97, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.218, "open": 39.97, "day": "31"} +{"volume": 1800, "symbol": "CRC", "ts": "2018-08-31 09:38:00", "month": "08", "high": 40.26, "low": 40.26, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.26, "open": 40.26, "day": "31"} +{"volume": 2044, "symbol": "CRC", "ts": "2018-08-31 09:40:00", "month": "08", "high": 40.04, "low": 40.04, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.04, "open": 40.04, "day": "31"} +{"volume": 4833, "symbol": "CRC", "ts": "2018-08-31 09:42:00", "month": "08", "high": 39.78, "low": 39.78, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.78, "open": 39.78, "day": "31"} +{"volume": 3100, "symbol": "CRC", "ts": "2018-08-31 09:43:00", "month": "08", "high": 39.8336, "low": 39.8336, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.8336, "open": 39.8336, "day": "31"} +{"volume": 7061, "symbol": "CRC", "ts": "2018-08-31 09:44:00", "month": "08", "high": 40.01, "low": 40.01, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.01, "open": 40.01, "day": "31"} +{"volume": 1222, "symbol": "CRC", "ts": "2018-08-31 09:45:00", "month": "08", "high": 40.07, "low": 40.07, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.07, "open": 40.07, "day": "31"} +{"volume": 2140, "symbol": "CRC", "ts": "2018-08-31 09:47:00", "month": "08", "high": 40.0146, "low": 40.0146, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.0146, "open": 40.0146, "day": "31"} +{"volume": 1148, "symbol": "CRC", "ts": "2018-08-31 09:49:00", "month": "08", "high": 40.0382, "low": 40.0382, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.0382, "open": 40.0382, "day": "31"} +{"volume": 1858, "symbol": "CRC", "ts": "2018-08-31 09:50:00", "month": "08", "high": 39.97, "low": 39.97, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.97, "open": 39.97, "day": "31"} +{"volume": 1801, "symbol": "CRC", "ts": "2018-08-31 09:51:00", "month": "08", "high": 40.005, "low": 40.005, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.005, "open": 40.005, "day": "31"} +{"volume": 5083, "symbol": "CRC", "ts": "2018-08-31 09:52:00", "month": "08", "high": 40.12, "low": 40.12, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.12, "open": 40.12, "day": "31"} +{"volume": 5967, "symbol": "CRC", "ts": "2018-08-31 09:53:00", "month": "08", "high": 40.1505, "low": 40.01, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.1505, "open": 40.01, "day": "31"} +{"volume": 13552, "symbol": "CRC", "ts": "2018-08-31 09:54:00", "month": "08", "high": 40.17, "low": 40.14, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.17, "open": 40.14, "day": "31"} +{"volume": 3239, "symbol": "CRC", "ts": "2018-08-31 09:55:00", "month": "08", "high": 40.27, "low": 40.27, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.27, "open": 40.27, "day": "31"} +{"volume": 12793, "symbol": "CRC", "ts": "2018-08-31 09:56:00", "month": "08", "high": 40.4, "low": 40.27, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.4, "open": 40.27, "day": "31"} +{"volume": 2040, "symbol": "CRC", "ts": "2018-08-31 09:57:00", "month": "08", "high": 40.33, "low": 40.33, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.33, "open": 40.33, "day": "31"} +{"volume": 2600, "symbol": "CRC", "ts": "2018-08-31 09:58:00", "month": "08", "high": 40.35, "low": 40.35, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.35, "open": 40.35, "day": "31"} +{"volume": 1680, "symbol": "CRC", "ts": "2018-08-31 09:59:00", "month": "08", "high": 40.24, "low": 40.24, "key": "CRC_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 40.24, "open": 40.24, "day": "31"} +{"volume": 28044, "symbol": "VKTX", "ts": "2018-08-31 09:31:00", "month": "08", "high": 13.0, "low": 12.945, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.98, "open": 12.945, "day": "31"} +{"volume": 22429, "symbol": "VKTX", "ts": "2018-08-31 09:32:00", "month": "08", "high": 13.1, "low": 13.05, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.1, "open": 13.05, "day": "31"} +{"volume": 48140, "symbol": "VKTX", "ts": "2018-08-31 09:33:00", "month": "08", "high": 13.18, "low": 13.1, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.1, "open": 13.1, "day": "31"} +{"volume": 27710, "symbol": "VKTX", "ts": "2018-08-31 09:34:00", "month": "08", "high": 13.17, "low": 13.0963, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.11, "open": 13.17, "day": "31"} +{"volume": 21136, "symbol": "VKTX", "ts": "2018-08-31 09:35:00", "month": "08", "high": 13.16, "low": 13.15, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.16, "open": 13.15, "day": "31"} +{"volume": 29866, "symbol": "VKTX", "ts": "2018-08-31 09:36:00", "month": "08", "high": 13.15, "low": 13.0893, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.0893, "open": 13.15, "day": "31"} +{"volume": 14706, "symbol": "VKTX", "ts": "2018-08-31 09:37:00", "month": "08", "high": 13.0325, "low": 12.9108, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.9108, "open": 13.0325, "day": "31"} +{"volume": 11737, "symbol": "VKTX", "ts": "2018-08-31 09:38:00", "month": "08", "high": 13.0, "low": 12.9301, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.0, "open": 12.9301, "day": "31"} +{"volume": 3172, "symbol": "VKTX", "ts": "2018-08-31 09:39:00", "month": "08", "high": 12.9284, "low": 12.9284, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.9284, "open": 12.9284, "day": "31"} +{"volume": 11781, "symbol": "VKTX", "ts": "2018-08-31 09:40:00", "month": "08", "high": 12.9, "low": 12.88, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.9, "open": 12.88, "day": "31"} +{"volume": 4832, "symbol": "VKTX", "ts": "2018-08-31 09:41:00", "month": "08", "high": 12.91, "low": 12.91, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.91, "open": 12.91, "day": "31"} +{"volume": 10891, "symbol": "VKTX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 12.92, "low": 12.92, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.92, "open": 12.92, "day": "31"} +{"volume": 15953, "symbol": "VKTX", "ts": "2018-08-31 09:43:00", "month": "08", "high": 12.965, "low": 12.96, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.96, "open": 12.965, "day": "31"} +{"volume": 7146, "symbol": "VKTX", "ts": "2018-08-31 09:44:00", "month": "08", "high": 12.96, "low": 12.92, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.92, "open": 12.96, "day": "31"} +{"volume": 6320, "symbol": "VKTX", "ts": "2018-08-31 09:45:00", "month": "08", "high": 12.92, "low": 12.92, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.92, "open": 12.92, "day": "31"} +{"volume": 8241, "symbol": "VKTX", "ts": "2018-08-31 09:46:00", "month": "08", "high": 12.88, "low": 12.88, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.88, "open": 12.88, "day": "31"} +{"volume": 5794, "symbol": "VKTX", "ts": "2018-08-31 09:47:00", "month": "08", "high": 12.91, "low": 12.91, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.91, "open": 12.91, "day": "31"} +{"volume": 20194, "symbol": "VKTX", "ts": "2018-08-31 09:48:00", "month": "08", "high": 12.8742, "low": 12.761, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.761, "open": 12.8742, "day": "31"} +{"volume": 22886, "symbol": "VKTX", "ts": "2018-08-31 09:49:00", "month": "08", "high": 12.79, "low": 12.72, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.79, "open": 12.72, "day": "31"} +{"volume": 4001, "symbol": "VKTX", "ts": "2018-08-31 09:50:00", "month": "08", "high": 12.804, "low": 12.804, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.804, "open": 12.804, "day": "31"} +{"volume": 9142, "symbol": "VKTX", "ts": "2018-08-31 09:51:00", "month": "08", "high": 12.77, "low": 12.72, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.77, "open": 12.72, "day": "31"} +{"volume": 4995, "symbol": "VKTX", "ts": "2018-08-31 09:52:00", "month": "08", "high": 12.795, "low": 12.795, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.795, "open": 12.795, "day": "31"} +{"volume": 14957, "symbol": "VKTX", "ts": "2018-08-31 09:53:00", "month": "08", "high": 12.86, "low": 12.815, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.86, "open": 12.815, "day": "31"} +{"volume": 3326, "symbol": "VKTX", "ts": "2018-08-31 09:54:00", "month": "08", "high": 12.79, "low": 12.79, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.79, "open": 12.79, "day": "31"} +{"volume": 6432, "symbol": "VKTX", "ts": "2018-08-31 09:55:00", "month": "08", "high": 12.82, "low": 12.82, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.82, "open": 12.82, "day": "31"} +{"volume": 9490, "symbol": "VKTX", "ts": "2018-08-31 09:56:00", "month": "08", "high": 12.775, "low": 12.775, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.775, "open": 12.775, "day": "31"} +{"volume": 6528, "symbol": "VKTX", "ts": "2018-08-31 09:57:00", "month": "08", "high": 12.74, "low": 12.74, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.74, "open": 12.74, "day": "31"} +{"volume": 8132, "symbol": "VKTX", "ts": "2018-08-31 09:58:00", "month": "08", "high": 12.7278, "low": 12.7278, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.7278, "open": 12.7278, "day": "31"} +{"volume": 26358, "symbol": "VKTX", "ts": "2018-08-31 09:59:00", "month": "08", "high": 12.6954, "low": 12.6371, "key": "VKTX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.645, "open": 12.6954, "day": "31"} +{"volume": 4642, "symbol": "CDXS", "ts": "2018-08-31 09:36:00", "month": "08", "high": 17.1247, "low": 17.1247, "key": "CDXS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 17.1247, "open": 17.1247, "day": "31"} +{"volume": 9301, "symbol": "CDXS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 17.3, "low": 17.25, "key": "CDXS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 17.25, "open": 17.3, "day": "31"} +{"volume": 642, "symbol": "CDXS", "ts": "2018-08-31 09:49:00", "month": "08", "high": 17.2, "low": 17.2, "key": "CDXS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.2, "day": "31"} +{"volume": 1075, "symbol": "CDXS", "ts": "2018-08-31 09:51:00", "month": "08", "high": 17.1875, "low": 17.1875, "key": "CDXS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 17.1875, "open": 17.1875, "day": "31"} +{"volume": 1149, "symbol": "CDXS", "ts": "2018-08-31 09:59:00", "month": "08", "high": 17.05, "low": 17.05, "key": "CDXS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 17.05, "open": 17.05, "day": "31"} +{"volume": 400, "symbol": "LFVN", "ts": "2018-08-31 09:39:00", "month": "08", "high": 11.87, "low": 11.87, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.87, "open": 11.87, "day": "31"} +{"volume": 100, "symbol": "LFVN", "ts": "2018-08-31 09:41:00", "month": "08", "high": 11.989, "low": 11.989, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.989, "open": 11.989, "day": "31"} +{"volume": 201, "symbol": "LFVN", "ts": "2018-08-31 09:43:00", "month": "08", "high": 11.989, "low": 11.989, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.989, "open": 11.989, "day": "31"} +{"volume": 1695, "symbol": "LFVN", "ts": "2018-08-31 09:45:00", "month": "08", "high": 11.95, "low": 11.95, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.95, "open": 11.95, "day": "31"} +{"volume": 402, "symbol": "LFVN", "ts": "2018-08-31 09:50:00", "month": "08", "high": 11.9505, "low": 11.95, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.9505, "open": 11.95, "day": "31"} +{"volume": 7844, "symbol": "LFVN", "ts": "2018-08-31 09:56:00", "month": "08", "high": 11.9901, "low": 11.9901, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 11.9901, "open": 11.9901, "day": "31"} +{"volume": 240, "symbol": "LFVN", "ts": "2018-08-31 09:58:00", "month": "08", "high": 12.06, "low": 12.06, "key": "LFVN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 12.06, "open": 12.06, "day": "31"} +{"volume": 310, "symbol": "SRDX", "ts": "2018-08-31 09:32:00", "month": "08", "high": 76.12, "low": 76.12, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.12, "open": 76.12, "day": "31"} +{"volume": 477, "symbol": "SRDX", "ts": "2018-08-31 09:36:00", "month": "08", "high": 76.5, "low": 76.5, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.5, "open": 76.5, "day": "31"} +{"volume": 923, "symbol": "SRDX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 76.5, "low": 76.5, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.5, "open": 76.5, "day": "31"} +{"volume": 646, "symbol": "SRDX", "ts": "2018-08-31 09:49:00", "month": "08", "high": 76.5, "low": 76.5, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.5, "open": 76.5, "day": "31"} +{"volume": 292, "symbol": "SRDX", "ts": "2018-08-31 09:50:00", "month": "08", "high": 76.5, "low": 76.5, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.5, "open": 76.5, "day": "31"} +{"volume": 1018, "symbol": "SRDX", "ts": "2018-08-31 09:54:00", "month": "08", "high": 76.459, "low": 76.459, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.459, "open": 76.459, "day": "31"} +{"volume": 1023, "symbol": "SRDX", "ts": "2018-08-31 09:58:00", "month": "08", "high": 76.4, "low": 76.4, "key": "SRDX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 76.4, "open": 76.4, "day": "31"} +{"volume": 1995, "symbol": "SDPI", "ts": "2018-08-31 09:40:00", "month": "08", "high": 1.93, "low": 1.93, "key": "SDPI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1.93, "open": 1.93, "day": "31"} +{"volume": 4800, "symbol": "SDPI", "ts": "2018-08-31 09:56:00", "month": "08", "high": 1.9284, "low": 1.92, "key": "SDPI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 1.9284, "open": 1.92, "day": "31"} +{"volume": 668, "symbol": "MDB", "ts": "2018-08-31 09:31:00", "month": "08", "high": 70.71, "low": 70.71, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.71, "open": 70.71, "day": "31"} +{"volume": 894, "symbol": "MDB", "ts": "2018-08-31 09:33:00", "month": "08", "high": 70.82, "low": 70.82, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.82, "open": 70.82, "day": "31"} +{"volume": 1884, "symbol": "MDB", "ts": "2018-08-31 09:35:00", "month": "08", "high": 70.97, "low": 70.97, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.97, "open": 70.97, "day": "31"} +{"volume": 5339, "symbol": "MDB", "ts": "2018-08-31 09:36:00", "month": "08", "high": 71.17, "low": 71.17, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.17, "open": 71.17, "day": "31"} +{"volume": 484, "symbol": "MDB", "ts": "2018-08-31 09:37:00", "month": "08", "high": 71.5186, "low": 71.5186, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.5186, "open": 71.5186, "day": "31"} +{"volume": 1360, "symbol": "MDB", "ts": "2018-08-31 09:38:00", "month": "08", "high": 71.6034, "low": 71.6034, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.6034, "open": 71.6034, "day": "31"} +{"volume": 956, "symbol": "MDB", "ts": "2018-08-31 09:42:00", "month": "08", "high": 71.56, "low": 71.56, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.56, "open": 71.56, "day": "31"} +{"volume": 1860, "symbol": "MDB", "ts": "2018-08-31 09:43:00", "month": "08", "high": 71.66, "low": 71.66, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.66, "open": 71.66, "day": "31"} +{"volume": 1309, "symbol": "MDB", "ts": "2018-08-31 09:44:00", "month": "08", "high": 71.8152, "low": 71.8152, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.8152, "open": 71.8152, "day": "31"} +{"volume": 3838, "symbol": "MDB", "ts": "2018-08-31 09:45:00", "month": "08", "high": 71.4599, "low": 71.4599, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.4599, "open": 71.4599, "day": "31"} +{"volume": 1447, "symbol": "MDB", "ts": "2018-08-31 09:47:00", "month": "08", "high": 71.43, "low": 71.43, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.43, "open": 71.43, "day": "31"} +{"volume": 2215, "symbol": "MDB", "ts": "2018-08-31 09:48:00", "month": "08", "high": 71.11, "low": 71.11, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.11, "open": 71.11, "day": "31"} +{"volume": 1492, "symbol": "MDB", "ts": "2018-08-31 09:49:00", "month": "08", "high": 71.1925, "low": 71.1925, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.1925, "open": 71.1925, "day": "31"} +{"volume": 1147, "symbol": "MDB", "ts": "2018-08-31 09:50:00", "month": "08", "high": 71.47, "low": 71.47, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.47, "open": 71.47, "day": "31"} +{"volume": 435, "symbol": "MDB", "ts": "2018-08-31 09:51:00", "month": "08", "high": 71.47, "low": 71.47, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.47, "open": 71.47, "day": "31"} +{"volume": 1540, "symbol": "MDB", "ts": "2018-08-31 09:53:00", "month": "08", "high": 71.63, "low": 71.63, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.63, "open": 71.63, "day": "31"} +{"volume": 1077, "symbol": "MDB", "ts": "2018-08-31 09:55:00", "month": "08", "high": 71.52, "low": 71.52, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.52, "open": 71.52, "day": "31"} +{"volume": 2870, "symbol": "MDB", "ts": "2018-08-31 09:56:00", "month": "08", "high": 71.43, "low": 71.43, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.43, "open": 71.43, "day": "31"} +{"volume": 2163, "symbol": "MDB", "ts": "2018-08-31 09:58:00", "month": "08", "high": 71.4843, "low": 71.4843, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.4843, "open": 71.4843, "day": "31"} +{"volume": 880, "symbol": "MDB", "ts": "2018-08-31 09:59:00", "month": "08", "high": 71.349, "low": 71.349, "key": "MDB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.349, "open": 71.349, "day": "31"} +{"volume": 4039, "symbol": "LGCY", "ts": "2018-08-31 09:36:00", "month": "08", "high": 5.28, "low": 5.28, "key": "LGCY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.28, "open": 5.28, "day": "31"} +{"volume": 800, "symbol": "LGCY", "ts": "2018-08-31 09:41:00", "month": "08", "high": 5.255, "low": 5.255, "key": "LGCY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.255, "open": 5.255, "day": "31"} +{"volume": 150, "symbol": "LGCY", "ts": "2018-08-31 09:47:00", "month": "08", "high": 5.29, "low": 5.29, "key": "LGCY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.29, "open": 5.29, "day": "31"} +{"volume": 2292, "symbol": "LGCY", "ts": "2018-08-31 09:53:00", "month": "08", "high": 5.349, "low": 5.349, "key": "LGCY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.349, "open": 5.349, "day": "31"} +{"volume": 2830, "symbol": "LGCY", "ts": "2018-08-31 09:54:00", "month": "08", "high": 5.37, "low": 5.37, "key": "LGCY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.37, "open": 5.37, "day": "31"} +{"volume": 5788, "symbol": "ATTU", "ts": "2018-08-31 09:33:00", "month": "08", "high": 20.2975, "low": 20.2335, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.2975, "open": 20.2335, "day": "31"} +{"volume": 11423, "symbol": "ATTU", "ts": "2018-08-31 09:39:00", "month": "08", "high": 20.3, "low": 20.13, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.13, "open": 20.3, "day": "31"} +{"volume": 1552, "symbol": "ATTU", "ts": "2018-08-31 09:42:00", "month": "08", "high": 20.186, "low": 20.186, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.186, "open": 20.186, "day": "31"} +{"volume": 18494, "symbol": "ATTU", "ts": "2018-08-31 09:43:00", "month": "08", "high": 20.1707, "low": 20.1707, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.1707, "open": 20.1707, "day": "31"} +{"volume": 4600, "symbol": "ATTU", "ts": "2018-08-31 09:45:00", "month": "08", "high": 20.23, "low": 20.23, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.23, "open": 20.23, "day": "31"} +{"volume": 5615, "symbol": "ATTU", "ts": "2018-08-31 09:47:00", "month": "08", "high": 20.24, "low": 20.1901, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.1901, "open": 20.24, "day": "31"} +{"volume": 850, "symbol": "ATTU", "ts": "2018-08-31 09:49:00", "month": "08", "high": 20.1909, "low": 20.1909, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.1909, "open": 20.1909, "day": "31"} +{"volume": 1650, "symbol": "ATTU", "ts": "2018-08-31 09:51:00", "month": "08", "high": 20.39, "low": 20.39, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.39, "open": 20.39, "day": "31"} +{"volume": 700, "symbol": "ATTU", "ts": "2018-08-31 09:55:00", "month": "08", "high": 20.36, "low": 20.36, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.36, "open": 20.36, "day": "31"} +{"volume": 2088, "symbol": "ATTU", "ts": "2018-08-31 09:57:00", "month": "08", "high": 20.4536, "low": 20.4536, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.4536, "open": 20.4536, "day": "31"} +{"volume": 7429, "symbol": "ATTU", "ts": "2018-08-31 09:58:00", "month": "08", "high": 20.4609, "low": 20.4609, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.4609, "open": 20.4609, "day": "31"} +{"volume": 303, "symbol": "ATTU", "ts": "2018-08-31 09:59:00", "month": "08", "high": 20.4448, "low": 20.4448, "key": "ATTU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 20.4448, "open": 20.4448, "day": "31"} +{"volume": 0, "symbol": "AYX", "ts": "2018-08-31 09:31:00", "month": "08", "high": 56.2554, "low": 56.2554, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.2554, "open": 56.2554, "day": "31"} +{"volume": 272, "symbol": "AYX", "ts": "2018-08-31 09:32:00", "month": "08", "high": 56.51, "low": 56.51, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.51, "open": 56.51, "day": "31"} +{"volume": 1212, "symbol": "AYX", "ts": "2018-08-31 09:35:00", "month": "08", "high": 56.7, "low": 56.7, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.7, "open": 56.7, "day": "31"} +{"volume": 7696, "symbol": "AYX", "ts": "2018-08-31 09:37:00", "month": "08", "high": 57.09, "low": 57.09, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.09, "open": 57.09, "day": "31"} +{"volume": 567, "symbol": "AYX", "ts": "2018-08-31 09:38:00", "month": "08", "high": 57.21, "low": 57.21, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.21, "open": 57.21, "day": "31"} +{"volume": 2034, "symbol": "AYX", "ts": "2018-08-31 09:40:00", "month": "08", "high": 57.23, "low": 57.23, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.23, "open": 57.23, "day": "31"} +{"volume": 1650, "symbol": "AYX", "ts": "2018-08-31 09:41:00", "month": "08", "high": 57.31, "low": 57.31, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.31, "open": 57.31, "day": "31"} +{"volume": 781, "symbol": "AYX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 57.49, "low": 57.49, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.49, "open": 57.49, "day": "31"} +{"volume": 685, "symbol": "AYX", "ts": "2018-08-31 09:44:00", "month": "08", "high": 57.485, "low": 57.485, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.485, "open": 57.485, "day": "31"} +{"volume": 1856, "symbol": "AYX", "ts": "2018-08-31 09:45:00", "month": "08", "high": 57.2, "low": 57.2, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.2, "open": 57.2, "day": "31"} +{"volume": 2060, "symbol": "AYX", "ts": "2018-08-31 09:47:00", "month": "08", "high": 57.1999, "low": 57.01, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.01, "open": 57.1999, "day": "31"} +{"volume": 158, "symbol": "AYX", "ts": "2018-08-31 09:48:00", "month": "08", "high": 57.2999, "low": 57.2999, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.2999, "open": 57.2999, "day": "31"} +{"volume": 1048, "symbol": "AYX", "ts": "2018-08-31 09:51:00", "month": "08", "high": 57.1478, "low": 57.1478, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.1478, "open": 57.1478, "day": "31"} +{"volume": 2436, "symbol": "AYX", "ts": "2018-08-31 09:52:00", "month": "08", "high": 57.3, "low": 57.3, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.3, "open": 57.3, "day": "31"} +{"volume": 1595, "symbol": "AYX", "ts": "2018-08-31 09:53:00", "month": "08", "high": 57.2, "low": 57.2, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.2, "open": 57.2, "day": "31"} +{"volume": 102, "symbol": "AYX", "ts": "2018-08-31 09:56:00", "month": "08", "high": 57.255, "low": 57.255, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 57.255, "open": 57.255, "day": "31"} +{"volume": 6612, "symbol": "AYX", "ts": "2018-08-31 09:57:00", "month": "08", "high": 57.05, "low": 56.8, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.8, "open": 57.05, "day": "31"} +{"volume": 1682, "symbol": "AYX", "ts": "2018-08-31 09:58:00", "month": "08", "high": 56.74, "low": 56.74, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.74, "open": 56.74, "day": "31"} +{"volume": 1723, "symbol": "AYX", "ts": "2018-08-31 09:59:00", "month": "08", "high": 56.54, "low": 56.54, "key": "AYX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 56.54, "open": 56.54, "day": "31"} +{"volume": 368, "symbol": "IRMD", "ts": "2018-08-31 09:34:00", "month": "08", "high": 26.65, "low": 26.65, "key": "IRMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 26.65, "open": 26.65, "day": "31"} +{"volume": 1800, "symbol": "IRMD", "ts": "2018-08-31 09:40:00", "month": "08", "high": 26.895, "low": 26.895, "key": "IRMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 26.895, "open": 26.895, "day": "31"} +{"volume": 204, "symbol": "IRMD", "ts": "2018-08-31 09:42:00", "month": "08", "high": 26.8503, "low": 26.8503, "key": "IRMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 26.8503, "open": 26.8503, "day": "31"} +{"volume": 202, "symbol": "IRMD", "ts": "2018-08-31 09:47:00", "month": "08", "high": 26.945, "low": 26.945, "key": "IRMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 26.945, "open": 26.945, "day": "31"} +{"volume": 419, "symbol": "IRMD", "ts": "2018-08-31 09:52:00", "month": "08", "high": 26.9, "low": 26.9, "key": "IRMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 26.9, "open": 26.9, "day": "31"} +{"volume": 0, "symbol": "SEAS", "ts": "2018-08-31 09:31:00", "month": "08", "high": 28.77, "low": 28.77, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.77, "open": 28.77, "day": "31"} +{"volume": 1701, "symbol": "SEAS", "ts": "2018-08-31 09:32:00", "month": "08", "high": 28.89, "low": 28.89, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.89, "open": 28.89, "day": "31"} +{"volume": 2475, "symbol": "SEAS", "ts": "2018-08-31 09:33:00", "month": "08", "high": 28.905, "low": 28.905, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.905, "open": 28.905, "day": "31"} +{"volume": 3614, "symbol": "SEAS", "ts": "2018-08-31 09:34:00", "month": "08", "high": 28.905, "low": 28.905, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.905, "open": 28.905, "day": "31"} +{"volume": 2535, "symbol": "SEAS", "ts": "2018-08-31 09:35:00", "month": "08", "high": 28.885, "low": 28.885, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.885, "open": 28.885, "day": "31"} +{"volume": 8126, "symbol": "SEAS", "ts": "2018-08-31 09:36:00", "month": "08", "high": 28.95, "low": 28.9, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.95, "open": 28.9, "day": "31"} +{"volume": 15902, "symbol": "SEAS", "ts": "2018-08-31 09:37:00", "month": "08", "high": 28.98, "low": 28.91, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.91, "open": 28.93, "day": "31"} +{"volume": 8958, "symbol": "SEAS", "ts": "2018-08-31 09:38:00", "month": "08", "high": 28.92, "low": 28.84, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.92, "open": 28.9, "day": "31"} +{"volume": 2105, "symbol": "SEAS", "ts": "2018-08-31 09:40:00", "month": "08", "high": 28.96, "low": 28.96, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.96, "open": 28.96, "day": "31"} +{"volume": 6136, "symbol": "SEAS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 29.0, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.0, "open": 28.99, "day": "31"} +{"volume": 2345, "symbol": "SEAS", "ts": "2018-08-31 09:43:00", "month": "08", "high": 28.93, "low": 28.93, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.93, "open": 28.93, "day": "31"} +{"volume": 1507, "symbol": "SEAS", "ts": "2018-08-31 09:44:00", "month": "08", "high": 28.94, "low": 28.94, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.94, "open": 28.94, "day": "31"} +{"volume": 2269, "symbol": "SEAS", "ts": "2018-08-31 09:45:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 1400, "symbol": "SEAS", "ts": "2018-08-31 09:46:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 2554, "symbol": "SEAS", "ts": "2018-08-31 09:47:00", "month": "08", "high": 28.985, "low": 28.985, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.985, "open": 28.985, "day": "31"} +{"volume": 1246, "symbol": "SEAS", "ts": "2018-08-31 09:49:00", "month": "08", "high": 28.96, "low": 28.96, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.96, "open": 28.96, "day": "31"} +{"volume": 1730, "symbol": "SEAS", "ts": "2018-08-31 09:51:00", "month": "08", "high": 28.96, "low": 28.96, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.96, "open": 28.96, "day": "31"} +{"volume": 2061, "symbol": "SEAS", "ts": "2018-08-31 09:52:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 1870, "symbol": "SEAS", "ts": "2018-08-31 09:53:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 7531, "symbol": "SEAS", "ts": "2018-08-31 09:54:00", "month": "08", "high": 29.0, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 29.0, "open": 28.99, "day": "31"} +{"volume": 6166, "symbol": "SEAS", "ts": "2018-08-31 09:55:00", "month": "08", "high": 28.98, "low": 28.97, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.97, "open": 28.98, "day": "31"} +{"volume": 1341, "symbol": "SEAS", "ts": "2018-08-31 09:57:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 4894, "symbol": "SEAS", "ts": "2018-08-31 09:58:00", "month": "08", "high": 28.955, "low": 28.955, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.955, "open": 28.955, "day": "31"} +{"volume": 3019, "symbol": "SEAS", "ts": "2018-08-31 09:59:00", "month": "08", "high": 28.945, "low": 28.945, "key": "SEAS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 28.945, "open": 28.945, "day": "31"} +{"volume": 25490, "symbol": "RIBT", "ts": "2018-08-31 09:31:00", "month": "08", "high": 3.2604, "low": 3.2604, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.2604, "open": 3.2604, "day": "31"} +{"volume": 8232, "symbol": "RIBT", "ts": "2018-08-31 09:32:00", "month": "08", "high": 3.2508, "low": 3.2508, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.2508, "open": 3.2508, "day": "31"} +{"volume": 300, "symbol": "RIBT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 3.244, "low": 3.244, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.244, "open": 3.244, "day": "31"} +{"volume": 1399, "symbol": "RIBT", "ts": "2018-08-31 09:36:00", "month": "08", "high": 3.2375, "low": 3.2375, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.2375, "open": 3.2375, "day": "31"} +{"volume": 502, "symbol": "RIBT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 137, "symbol": "RIBT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 2068, "symbol": "RIBT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 2648, "symbol": "RIBT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 453, "symbol": "RIBT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 1087, "symbol": "RIBT", "ts": "2018-08-31 09:58:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 100, "symbol": "NSP", "ts": "2018-08-31 09:33:00", "month": "08", "high": 119.1, "low": 119.1, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 119.1, "open": 119.1, "day": "31"} +{"volume": 2017, "symbol": "NSP", "ts": "2018-08-31 09:36:00", "month": "08", "high": 117.8, "low": 117.8, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 117.8, "open": 117.8, "day": "31"} +{"volume": 738, "symbol": "NSP", "ts": "2018-08-31 09:37:00", "month": "08", "high": 117.8, "low": 117.8, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 117.8, "open": 117.8, "day": "31"} +{"volume": 1073, "symbol": "NSP", "ts": "2018-08-31 09:39:00", "month": "08", "high": 118.6, "low": 118.6, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.6, "open": 118.6, "day": "31"} +{"volume": 454, "symbol": "NSP", "ts": "2018-08-31 09:41:00", "month": "08", "high": 118.4099, "low": 118.4099, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.4099, "open": 118.4099, "day": "31"} +{"volume": 706, "symbol": "NSP", "ts": "2018-08-31 09:42:00", "month": "08", "high": 118.5, "low": 118.5, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.5, "open": 118.5, "day": "31"} +{"volume": 376, "symbol": "NSP", "ts": "2018-08-31 09:45:00", "month": "08", "high": 118.2, "low": 118.2, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.2, "open": 118.2, "day": "31"} +{"volume": 277, "symbol": "NSP", "ts": "2018-08-31 09:50:00", "month": "08", "high": 118.5, "low": 118.5, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.5, "open": 118.5, "day": "31"} +{"volume": 765, "symbol": "NSP", "ts": "2018-08-31 09:53:00", "month": "08", "high": 118.3747, "low": 118.3747, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.3747, "open": 118.3747, "day": "31"} +{"volume": 492, "symbol": "NSP", "ts": "2018-08-31 09:55:00", "month": "08", "high": 118.2, "low": 118.2, "key": "NSP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 118.2, "open": 118.2, "day": "31"} +{"volume": 6762, "symbol": "GDS", "ts": "2018-08-31 09:31:00", "month": "08", "high": 36.86, "low": 36.86, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 36.86, "open": 36.86, "day": "31"} +{"volume": 1402, "symbol": "GDS", "ts": "2018-08-31 09:32:00", "month": "08", "high": 36.74, "low": 36.74, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 36.74, "open": 36.74, "day": "31"} +{"volume": 1210, "symbol": "GDS", "ts": "2018-08-31 09:34:00", "month": "08", "high": 36.86, "low": 36.86, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 36.86, "open": 36.86, "day": "31"} +{"volume": 5092, "symbol": "GDS", "ts": "2018-08-31 09:36:00", "month": "08", "high": 37.31, "low": 37.04, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.31, "open": 37.04, "day": "31"} +{"volume": 4694, "symbol": "GDS", "ts": "2018-08-31 09:38:00", "month": "08", "high": 37.62, "low": 37.6, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.6, "open": 37.62, "day": "31"} +{"volume": 1920, "symbol": "GDS", "ts": "2018-08-31 09:40:00", "month": "08", "high": 37.78, "low": 37.78, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.78, "open": 37.78, "day": "31"} +{"volume": 2222, "symbol": "GDS", "ts": "2018-08-31 09:41:00", "month": "08", "high": 37.88, "low": 37.88, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.88, "open": 37.88, "day": "31"} +{"volume": 6525, "symbol": "GDS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 37.85, "low": 37.6411, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.6411, "open": 37.85, "day": "31"} +{"volume": 4970, "symbol": "GDS", "ts": "2018-08-31 09:44:00", "month": "08", "high": 37.7, "low": 37.7, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.7, "open": 37.7, "day": "31"} +{"volume": 1705, "symbol": "GDS", "ts": "2018-08-31 09:46:00", "month": "08", "high": 37.48, "low": 37.48, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.48, "open": 37.48, "day": "31"} +{"volume": 3524, "symbol": "GDS", "ts": "2018-08-31 09:47:00", "month": "08", "high": 37.58, "low": 37.58, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.58, "open": 37.58, "day": "31"} +{"volume": 2270, "symbol": "GDS", "ts": "2018-08-31 09:48:00", "month": "08", "high": 37.5988, "low": 37.5988, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.5988, "open": 37.5988, "day": "31"} +{"volume": 1408, "symbol": "GDS", "ts": "2018-08-31 09:50:00", "month": "08", "high": 37.5972, "low": 37.5972, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.5972, "open": 37.5972, "day": "31"} +{"volume": 5719, "symbol": "GDS", "ts": "2018-08-31 09:52:00", "month": "08", "high": 37.6595, "low": 37.6595, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.6595, "open": 37.6595, "day": "31"} +{"volume": 543, "symbol": "GDS", "ts": "2018-08-31 09:53:00", "month": "08", "high": 37.6, "low": 37.6, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.6, "open": 37.6, "day": "31"} +{"volume": 237, "symbol": "GDS", "ts": "2018-08-31 09:54:00", "month": "08", "high": 37.73, "low": 37.73, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.73, "open": 37.73, "day": "31"} +{"volume": 1582, "symbol": "GDS", "ts": "2018-08-31 09:57:00", "month": "08", "high": 37.74, "low": 37.74, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.74, "open": 37.74, "day": "31"} +{"volume": 2478, "symbol": "GDS", "ts": "2018-08-31 09:59:00", "month": "08", "high": 37.66, "low": 37.66, "key": "GDS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 37.66, "open": 37.66, "day": "31"} +{"volume": 0, "symbol": "VICR", "ts": "2018-08-31 09:31:00", "month": "08", "high": 62.1, "low": 62.1, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.1, "open": 62.1, "day": "31"} +{"volume": 1036, "symbol": "VICR", "ts": "2018-08-31 09:33:00", "month": "08", "high": 61.7684, "low": 61.7684, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.7684, "open": 61.7684, "day": "31"} +{"volume": 239, "symbol": "VICR", "ts": "2018-08-31 09:35:00", "month": "08", "high": 61.15, "low": 61.15, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.15, "open": 61.15, "day": "31"} +{"volume": 1100, "symbol": "VICR", "ts": "2018-08-31 09:39:00", "month": "08", "high": 61.75, "low": 61.75, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.75, "open": 61.75, "day": "31"} +{"volume": 2569, "symbol": "VICR", "ts": "2018-08-31 09:44:00", "month": "08", "high": 61.75, "low": 61.75, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.75, "open": 61.75, "day": "31"} +{"volume": 4591, "symbol": "VICR", "ts": "2018-08-31 09:45:00", "month": "08", "high": 61.55, "low": 61.55, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 4310, "symbol": "VICR", "ts": "2018-08-31 09:47:00", "month": "08", "high": 62.275, "low": 62.275, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.275, "open": 62.275, "day": "31"} +{"volume": 2474, "symbol": "VICR", "ts": "2018-08-31 09:49:00", "month": "08", "high": 62.2, "low": 62.2, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.2, "open": 62.2, "day": "31"} +{"volume": 1442, "symbol": "VICR", "ts": "2018-08-31 09:51:00", "month": "08", "high": 61.9736, "low": 61.9736, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.9736, "open": 61.9736, "day": "31"} +{"volume": 1024, "symbol": "VICR", "ts": "2018-08-31 09:53:00", "month": "08", "high": 62.0, "low": 62.0, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.0, "open": 62.0, "day": "31"} +{"volume": 832, "symbol": "VICR", "ts": "2018-08-31 09:56:00", "month": "08", "high": 62.0, "low": 61.95, "key": "VICR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.0, "open": 61.95, "day": "31"} +{"volume": 553184, "symbol": "CRON", "ts": "2018-08-31 09:31:00", "month": "08", "high": 8.9668, "low": 8.6001, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.9668, "open": 8.6001, "day": "31"} +{"volume": 771769, "symbol": "CRON", "ts": "2018-08-31 09:32:00", "month": "08", "high": 9.25, "low": 8.9641, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.14, "open": 8.9641, "day": "31"} +{"volume": 372906, "symbol": "CRON", "ts": "2018-08-31 09:33:00", "month": "08", "high": 9.25, "low": 9.1, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.25, "open": 9.1, "day": "31"} +{"volume": 660411, "symbol": "CRON", "ts": "2018-08-31 09:34:00", "month": "08", "high": 9.5, "low": 9.2401, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.47, "open": 9.2401, "day": "31"} +{"volume": 855530, "symbol": "CRON", "ts": "2018-08-31 09:35:00", "month": "08", "high": 9.73, "low": 9.45, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.67, "open": 9.4601, "day": "31"} +{"volume": 817442, "symbol": "CRON", "ts": "2018-08-31 09:36:00", "month": "08", "high": 9.75, "low": 9.43, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.4599, "open": 9.6896, "day": "31"} +{"volume": 418395, "symbol": "CRON", "ts": "2018-08-31 09:37:00", "month": "08", "high": 9.72, "low": 9.45, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.72, "open": 9.465, "day": "31"} +{"volume": 770651, "symbol": "CRON", "ts": "2018-08-31 09:38:00", "month": "08", "high": 9.86, "low": 9.6, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.84, "open": 9.73, "day": "31"} +{"volume": 474781, "symbol": "CRON", "ts": "2018-08-31 09:39:00", "month": "08", "high": 9.8401, "low": 9.6716, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.737, "open": 9.82, "day": "31"} +{"volume": 355112, "symbol": "CRON", "ts": "2018-08-31 09:40:00", "month": "08", "high": 9.8, "low": 9.62, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.71, "open": 9.76, "day": "31"} +{"volume": 287848, "symbol": "CRON", "ts": "2018-08-31 09:41:00", "month": "08", "high": 9.7368, "low": 9.65, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.7214, "open": 9.7134, "day": "31"} +{"volume": 478938, "symbol": "CRON", "ts": "2018-08-31 09:42:00", "month": "08", "high": 9.75, "low": 9.52, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.52, "open": 9.7299, "day": "31"} +{"volume": 449588, "symbol": "CRON", "ts": "2018-08-31 09:43:00", "month": "08", "high": 9.5217, "low": 9.371, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.52, "open": 9.5217, "day": "31"} +{"volume": 321918, "symbol": "CRON", "ts": "2018-08-31 09:44:00", "month": "08", "high": 9.63, "low": 9.4522, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.63, "open": 9.51, "day": "31"} +{"volume": 182098, "symbol": "CRON", "ts": "2018-08-31 09:45:00", "month": "08", "high": 9.6301, "low": 9.484, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.4962, "open": 9.6301, "day": "31"} +{"volume": 185948, "symbol": "CRON", "ts": "2018-08-31 09:46:00", "month": "08", "high": 9.5552, "low": 9.46, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.53, "open": 9.481, "day": "31"} +{"volume": 283311, "symbol": "CRON", "ts": "2018-08-31 09:47:00", "month": "08", "high": 9.699, "low": 9.57, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6702, "open": 9.57, "day": "31"} +{"volume": 216848, "symbol": "CRON", "ts": "2018-08-31 09:48:00", "month": "08", "high": 9.68, "low": 9.5377, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.59, "open": 9.68, "day": "31"} +{"volume": 128427, "symbol": "CRON", "ts": "2018-08-31 09:49:00", "month": "08", "high": 9.65, "low": 9.6, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.6001, "day": "31"} +{"volume": 132394, "symbol": "CRON", "ts": "2018-08-31 09:50:00", "month": "08", "high": 9.6899, "low": 9.595, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.679, "open": 9.595, "day": "31"} +{"volume": 224050, "symbol": "CRON", "ts": "2018-08-31 09:51:00", "month": "08", "high": 9.6883, "low": 9.64, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.65, "day": "31"} +{"volume": 192704, "symbol": "CRON", "ts": "2018-08-31 09:52:00", "month": "08", "high": 9.6725, "low": 9.5708, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.5708, "open": 9.6725, "day": "31"} +{"volume": 107391, "symbol": "CRON", "ts": "2018-08-31 09:53:00", "month": "08", "high": 9.64, "low": 9.58, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.61, "open": 9.5801, "day": "31"} +{"volume": 120831, "symbol": "CRON", "ts": "2018-08-31 09:54:00", "month": "08", "high": 9.649, "low": 9.57, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6215, "open": 9.595, "day": "31"} +{"volume": 164980, "symbol": "CRON", "ts": "2018-08-31 09:55:00", "month": "08", "high": 9.65, "low": 9.6, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.62, "day": "31"} +{"volume": 149232, "symbol": "CRON", "ts": "2018-08-31 09:56:00", "month": "08", "high": 9.647, "low": 9.599, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.6432, "open": 9.6001, "day": "31"} +{"volume": 142787, "symbol": "CRON", "ts": "2018-08-31 09:57:00", "month": "08", "high": 9.65, "low": 9.61, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.65, "day": "31"} +{"volume": 1227769, "symbol": "CRON", "ts": "2018-08-31 09:58:00", "month": "08", "high": 10.03, "low": 9.6301, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.975, "open": 9.6301, "day": "31"} +{"volume": 429514, "symbol": "CRON", "ts": "2018-08-31 09:59:00", "month": "08", "high": 10.1, "low": 9.92, "key": "CRON_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 10.095, "open": 9.97, "day": "31"} +{"volume": 0, "symbol": "INS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 13.13, "low": 13.13, "key": "INS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.13, "open": 13.13, "day": "31"} +{"volume": 596, "symbol": "INS", "ts": "2018-08-31 09:46:00", "month": "08", "high": 13.26, "low": 13.26, "key": "INS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.26, "open": 13.26, "day": "31"} +{"volume": 100, "symbol": "INS", "ts": "2018-08-31 09:54:00", "month": "08", "high": 13.07, "low": 13.07, "key": "INS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.07, "open": 13.07, "day": "31"} +{"volume": 849, "symbol": "INS", "ts": "2018-08-31 09:58:00", "month": "08", "high": 13.1729, "low": 13.1729, "key": "INS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.1729, "open": 13.1729, "day": "31"} +{"volume": 4900, "symbol": "SRPT", "ts": "2018-08-31 09:31:00", "month": "08", "high": 138.63, "low": 138.05, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.05, "open": 138.63, "day": "31"} +{"volume": 5511, "symbol": "SRPT", "ts": "2018-08-31 09:32:00", "month": "08", "high": 138.3225, "low": 138.155, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.3225, "open": 138.155, "day": "31"} +{"volume": 1519, "symbol": "SRPT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 138.44, "low": 138.44, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.44, "open": 138.44, "day": "31"} +{"volume": 1723, "symbol": "SRPT", "ts": "2018-08-31 09:34:00", "month": "08", "high": 138.5, "low": 138.5, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.5, "open": 138.5, "day": "31"} +{"volume": 2063, "symbol": "SRPT", "ts": "2018-08-31 09:35:00", "month": "08", "high": 138.5, "low": 138.5, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.5, "open": 138.5, "day": "31"} +{"volume": 13188, "symbol": "SRPT", "ts": "2018-08-31 09:36:00", "month": "08", "high": 137.5, "low": 137.5, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.5, "open": 137.5, "day": "31"} +{"volume": 2982, "symbol": "SRPT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 137.6, "low": 137.6, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.6, "open": 137.6, "day": "31"} +{"volume": 2850, "symbol": "SRPT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 138.21, "low": 137.755, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.755, "open": 138.21, "day": "31"} +{"volume": 10493, "symbol": "SRPT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 137.66, "low": 137.22, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.22, "open": 137.66, "day": "31"} +{"volume": 1063, "symbol": "SRPT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 137.28, "low": 137.28, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.28, "open": 137.28, "day": "31"} +{"volume": 984, "symbol": "SRPT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 137.81, "low": 137.81, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.81, "open": 137.81, "day": "31"} +{"volume": 1457, "symbol": "SRPT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 137.89, "low": 137.89, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.89, "open": 137.89, "day": "31"} +{"volume": 4452, "symbol": "SRPT", "ts": "2018-08-31 09:44:00", "month": "08", "high": 137.76, "low": 137.72, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.72, "open": 137.76, "day": "31"} +{"volume": 2433, "symbol": "SRPT", "ts": "2018-08-31 09:45:00", "month": "08", "high": 137.61, "low": 137.61, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.61, "open": 137.61, "day": "31"} +{"volume": 4370, "symbol": "SRPT", "ts": "2018-08-31 09:46:00", "month": "08", "high": 137.84, "low": 137.84, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 137.84, "open": 137.84, "day": "31"} +{"volume": 883, "symbol": "SRPT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 138.0, "low": 138.0, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.0, "open": 138.0, "day": "31"} +{"volume": 5982, "symbol": "SRPT", "ts": "2018-08-31 09:48:00", "month": "08", "high": 138.05, "low": 137.9358, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.05, "open": 137.9358, "day": "31"} +{"volume": 2253, "symbol": "SRPT", "ts": "2018-08-31 09:49:00", "month": "08", "high": 138.24, "low": 138.24, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.24, "open": 138.24, "day": "31"} +{"volume": 6635, "symbol": "SRPT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 138.5, "low": 138.3, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.5, "open": 138.3, "day": "31"} +{"volume": 8303, "symbol": "SRPT", "ts": "2018-08-31 09:52:00", "month": "08", "high": 138.95, "low": 138.95, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.95, "open": 138.95, "day": "31"} +{"volume": 2980, "symbol": "SRPT", "ts": "2018-08-31 09:53:00", "month": "08", "high": 139.0574, "low": 139.0574, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 139.0574, "open": 139.0574, "day": "31"} +{"volume": 5614, "symbol": "SRPT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 139.3, "low": 139.115, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 139.3, "open": 139.115, "day": "31"} +{"volume": 2512, "symbol": "SRPT", "ts": "2018-08-31 09:56:00", "month": "08", "high": 139.42, "low": 139.42, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 139.42, "open": 139.42, "day": "31"} +{"volume": 3421, "symbol": "SRPT", "ts": "2018-08-31 09:57:00", "month": "08", "high": 139.4, "low": 139.4, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 139.4, "open": 139.4, "day": "31"} +{"volume": 3342, "symbol": "SRPT", "ts": "2018-08-31 09:58:00", "month": "08", "high": 139.25, "low": 139.25, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 139.25, "open": 139.25, "day": "31"} +{"volume": 3721, "symbol": "SRPT", "ts": "2018-08-31 09:59:00", "month": "08", "high": 138.77, "low": 138.7685, "key": "SRPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 138.77, "open": 138.7685, "day": "31"} +{"volume": 4648, "symbol": "HEAR", "ts": "2018-08-31 09:31:00", "month": "08", "high": 22.9065, "low": 22.88, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.88, "open": 22.9065, "day": "31"} +{"volume": 5382, "symbol": "HEAR", "ts": "2018-08-31 09:32:00", "month": "08", "high": 22.95, "low": 22.95, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 22.95, "open": 22.95, "day": "31"} +{"volume": 5296, "symbol": "HEAR", "ts": "2018-08-31 09:33:00", "month": "08", "high": 23.1, "low": 23.1, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.1, "open": 23.1, "day": "31"} +{"volume": 1903, "symbol": "HEAR", "ts": "2018-08-31 09:34:00", "month": "08", "high": 23.0084, "low": 23.0084, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.0084, "open": 23.0084, "day": "31"} +{"volume": 4604, "symbol": "HEAR", "ts": "2018-08-31 09:36:00", "month": "08", "high": 23.18, "low": 23.18, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.18, "open": 23.18, "day": "31"} +{"volume": 2907, "symbol": "HEAR", "ts": "2018-08-31 09:37:00", "month": "08", "high": 23.3128, "low": 23.3128, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.3128, "open": 23.3128, "day": "31"} +{"volume": 6259, "symbol": "HEAR", "ts": "2018-08-31 09:38:00", "month": "08", "high": 23.294, "low": 23.294, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.294, "open": 23.294, "day": "31"} +{"volume": 5022, "symbol": "HEAR", "ts": "2018-08-31 09:39:00", "month": "08", "high": 23.35, "low": 23.35, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.35, "open": 23.35, "day": "31"} +{"volume": 4423, "symbol": "HEAR", "ts": "2018-08-31 09:40:00", "month": "08", "high": 23.4, "low": 23.4, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.4, "open": 23.4, "day": "31"} +{"volume": 7436, "symbol": "HEAR", "ts": "2018-08-31 09:41:00", "month": "08", "high": 23.5, "low": 23.5, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.5, "open": 23.5, "day": "31"} +{"volume": 8351, "symbol": "HEAR", "ts": "2018-08-31 09:42:00", "month": "08", "high": 23.57, "low": 23.55, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.57, "open": 23.55, "day": "31"} +{"volume": 7048, "symbol": "HEAR", "ts": "2018-08-31 09:43:00", "month": "08", "high": 23.5606, "low": 23.5606, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.5606, "open": 23.5606, "day": "31"} +{"volume": 10602, "symbol": "HEAR", "ts": "2018-08-31 09:44:00", "month": "08", "high": 23.4931, "low": 23.49, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.4931, "open": 23.49, "day": "31"} +{"volume": 5473, "symbol": "HEAR", "ts": "2018-08-31 09:46:00", "month": "08", "high": 23.5501, "low": 23.5501, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.5501, "open": 23.5501, "day": "31"} +{"volume": 8383, "symbol": "HEAR", "ts": "2018-08-31 09:47:00", "month": "08", "high": 23.6632, "low": 23.65, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.65, "open": 23.6632, "day": "31"} +{"volume": 13514, "symbol": "HEAR", "ts": "2018-08-31 09:48:00", "month": "08", "high": 23.84, "low": 23.8368, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.8368, "open": 23.84, "day": "31"} +{"volume": 16201, "symbol": "HEAR", "ts": "2018-08-31 09:49:00", "month": "08", "high": 23.98, "low": 23.93, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.98, "open": 23.93, "day": "31"} +{"volume": 8092, "symbol": "HEAR", "ts": "2018-08-31 09:50:00", "month": "08", "high": 23.82, "low": 23.77, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.77, "open": 23.82, "day": "31"} +{"volume": 3033, "symbol": "HEAR", "ts": "2018-08-31 09:51:00", "month": "08", "high": 23.82, "low": 23.82, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.82, "open": 23.82, "day": "31"} +{"volume": 12656, "symbol": "HEAR", "ts": "2018-08-31 09:52:00", "month": "08", "high": 23.94, "low": 23.85, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.94, "open": 23.85, "day": "31"} +{"volume": 8469, "symbol": "HEAR", "ts": "2018-08-31 09:53:00", "month": "08", "high": 23.98, "low": 23.98, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.98, "open": 23.98, "day": "31"} +{"volume": 6952, "symbol": "HEAR", "ts": "2018-08-31 09:54:00", "month": "08", "high": 23.9, "low": 23.82, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.82, "open": 23.9, "day": "31"} +{"volume": 9957, "symbol": "HEAR", "ts": "2018-08-31 09:56:00", "month": "08", "high": 23.94, "low": 23.92, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.94, "open": 23.92, "day": "31"} +{"volume": 18648, "symbol": "HEAR", "ts": "2018-08-31 09:58:00", "month": "08", "high": 23.895, "low": 23.5, "key": "HEAR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 23.69, "open": 23.895, "day": "31"} +{"volume": 411, "symbol": "BLFS", "ts": "2018-08-31 09:33:00", "month": "08", "high": 24.1732, "low": 24.1732, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.1732, "open": 24.1732, "day": "31"} +{"volume": 547, "symbol": "BLFS", "ts": "2018-08-31 09:37:00", "month": "08", "high": 24.1908, "low": 24.001, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.001, "open": 24.1908, "day": "31"} +{"volume": 849, "symbol": "BLFS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 24.35, "low": 24.35, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.35, "open": 24.35, "day": "31"} +{"volume": 1261, "symbol": "BLFS", "ts": "2018-08-31 09:45:00", "month": "08", "high": 24.256, "low": 24.256, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.256, "open": 24.256, "day": "31"} +{"volume": 205, "symbol": "BLFS", "ts": "2018-08-31 09:47:00", "month": "08", "high": 24.3302, "low": 24.3302, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.3302, "open": 24.3302, "day": "31"} +{"volume": 115, "symbol": "BLFS", "ts": "2018-08-31 09:48:00", "month": "08", "high": 24.1706, "low": 24.1706, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.1706, "open": 24.1706, "day": "31"} +{"volume": 230, "symbol": "BLFS", "ts": "2018-08-31 09:53:00", "month": "08", "high": 24.31, "low": 24.31, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.31, "open": 24.31, "day": "31"} +{"volume": 2721, "symbol": "BLFS", "ts": "2018-08-31 09:57:00", "month": "08", "high": 24.45, "low": 24.45, "key": "BLFS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.45, "open": 24.45, "day": "31"} +{"volume": 5350, "symbol": "LGCYO", "ts": "2018-08-31 09:36:00", "month": "08", "high": 15.0, "low": 15.0, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.0, "day": "31"} +{"volume": 2287, "symbol": "LGCYO", "ts": "2018-08-31 09:41:00", "month": "08", "high": 15.0, "low": 15.0, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.0, "day": "31"} +{"volume": 1413, "symbol": "LGCYO", "ts": "2018-08-31 09:46:00", "month": "08", "high": 15.1, "low": 15.1, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.1, "open": 15.1, "day": "31"} +{"volume": 2800, "symbol": "LGCYO", "ts": "2018-08-31 09:52:00", "month": "08", "high": 15.49, "low": 15.49, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.49, "open": 15.49, "day": "31"} +{"volume": 1300, "symbol": "LGCYO", "ts": "2018-08-31 09:53:00", "month": "08", "high": 15.4, "low": 15.4, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.4, "open": 15.4, "day": "31"} +{"volume": 200, "symbol": "LGCYO", "ts": "2018-08-31 09:58:00", "month": "08", "high": 15.41, "low": 15.41, "key": "LGCYO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.41, "open": 15.41, "day": "31"} +{"volume": 6841, "symbol": "NIHD", "ts": "2018-08-31 09:32:00", "month": "08", "high": 5.6645, "low": 5.6645, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.6645, "open": 5.6645, "day": "31"} +{"volume": 1972, "symbol": "NIHD", "ts": "2018-08-31 09:35:00", "month": "08", "high": 5.75, "low": 5.75, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.75, "open": 5.75, "day": "31"} +{"volume": 567, "symbol": "NIHD", "ts": "2018-08-31 09:36:00", "month": "08", "high": 5.715, "low": 5.715, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.715, "open": 5.715, "day": "31"} +{"volume": 1100, "symbol": "NIHD", "ts": "2018-08-31 09:38:00", "month": "08", "high": 5.7795, "low": 5.7795, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.7795, "open": 5.7795, "day": "31"} +{"volume": 4918, "symbol": "NIHD", "ts": "2018-08-31 09:40:00", "month": "08", "high": 5.76, "low": 5.76, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.76, "open": 5.76, "day": "31"} +{"volume": 5400, "symbol": "NIHD", "ts": "2018-08-31 09:41:00", "month": "08", "high": 5.75, "low": 5.75, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.75, "open": 5.75, "day": "31"} +{"volume": 2919, "symbol": "NIHD", "ts": "2018-08-31 09:42:00", "month": "08", "high": 5.75, "low": 5.75, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.75, "open": 5.75, "day": "31"} +{"volume": 2740, "symbol": "NIHD", "ts": "2018-08-31 09:43:00", "month": "08", "high": 5.73, "low": 5.73, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.73, "open": 5.73, "day": "31"} +{"volume": 800, "symbol": "NIHD", "ts": "2018-08-31 09:46:00", "month": "08", "high": 5.72, "low": 5.72, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.72, "open": 5.72, "day": "31"} +{"volume": 3245, "symbol": "NIHD", "ts": "2018-08-31 09:47:00", "month": "08", "high": 5.72, "low": 5.72, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.72, "open": 5.72, "day": "31"} +{"volume": 1016, "symbol": "NIHD", "ts": "2018-08-31 09:49:00", "month": "08", "high": 5.7001, "low": 5.7001, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.7001, "open": 5.7001, "day": "31"} +{"volume": 6018, "symbol": "NIHD", "ts": "2018-08-31 09:50:00", "month": "08", "high": 5.7, "low": 5.7, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.7, "open": 5.7, "day": "31"} +{"volume": 11354, "symbol": "NIHD", "ts": "2018-08-31 09:51:00", "month": "08", "high": 5.69, "low": 5.69, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.69, "open": 5.69, "day": "31"} +{"volume": 3689, "symbol": "NIHD", "ts": "2018-08-31 09:54:00", "month": "08", "high": 5.7, "low": 5.7, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.7, "open": 5.7, "day": "31"} +{"volume": 5940, "symbol": "NIHD", "ts": "2018-08-31 09:55:00", "month": "08", "high": 5.67, "low": 5.67, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.67, "open": 5.67, "day": "31"} +{"volume": 8688, "symbol": "NIHD", "ts": "2018-08-31 09:56:00", "month": "08", "high": 5.67, "low": 5.67, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.67, "open": 5.67, "day": "31"} +{"volume": 4767, "symbol": "NIHD", "ts": "2018-08-31 09:58:00", "month": "08", "high": 5.7, "low": 5.7, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.7, "open": 5.7, "day": "31"} +{"volume": 926, "symbol": "NIHD", "ts": "2018-08-31 09:59:00", "month": "08", "high": 5.69, "low": 5.69, "key": "NIHD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 5.69, "open": 5.69, "day": "31"} +{"volume": 16539, "symbol": "NEPT", "ts": "2018-08-31 09:31:00", "month": "08", "high": 3.8848, "low": 3.8848, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.8848, "open": 3.8848, "day": "31"} +{"volume": 37230, "symbol": "NEPT", "ts": "2018-08-31 09:32:00", "month": "08", "high": 3.92, "low": 3.9, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.92, "open": 3.9, "day": "31"} +{"volume": 7037, "symbol": "NEPT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 3.925, "low": 3.925, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.925, "open": 3.925, "day": "31"} +{"volume": 4579, "symbol": "NEPT", "ts": "2018-08-31 09:34:00", "month": "08", "high": 3.921, "low": 3.921, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.921, "open": 3.921, "day": "31"} +{"volume": 49692, "symbol": "NEPT", "ts": "2018-08-31 09:35:00", "month": "08", "high": 4.0162, "low": 3.97, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.0162, "open": 3.97, "day": "31"} +{"volume": 14951, "symbol": "NEPT", "ts": "2018-08-31 09:36:00", "month": "08", "high": 4.0199, "low": 4.0199, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.0199, "open": 4.0199, "day": "31"} +{"volume": 13045, "symbol": "NEPT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 4.05, "low": 4.04, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.05, "open": 4.04, "day": "31"} +{"volume": 12126, "symbol": "NEPT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 4.05, "low": 4.05, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.05, "open": 4.05, "day": "31"} +{"volume": 37294, "symbol": "NEPT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 4.06, "low": 4.04, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.06, "open": 4.04, "day": "31"} +{"volume": 7205, "symbol": "NEPT", "ts": "2018-08-31 09:40:00", "month": "08", "high": 4.04, "low": 4.04, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.04, "open": 4.04, "day": "31"} +{"volume": 40343, "symbol": "NEPT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 4.059, "low": 4.02, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.02, "open": 4.04, "day": "31"} +{"volume": 7148, "symbol": "NEPT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 4.02, "low": 4.02, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.02, "open": 4.02, "day": "31"} +{"volume": 7797, "symbol": "NEPT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 4.01, "low": 4.01, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.01, "open": 4.01, "day": "31"} +{"volume": 52086, "symbol": "NEPT", "ts": "2018-08-31 09:44:00", "month": "08", "high": 4.0229, "low": 3.93, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.93, "open": 4.0229, "day": "31"} +{"volume": 15229, "symbol": "NEPT", "ts": "2018-08-31 09:46:00", "month": "08", "high": 3.99, "low": 3.982, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.982, "open": 3.99, "day": "31"} +{"volume": 3493, "symbol": "NEPT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 3.9858, "low": 3.9858, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.9858, "open": 3.9858, "day": "31"} +{"volume": 14703, "symbol": "NEPT", "ts": "2018-08-31 09:49:00", "month": "08", "high": 3.999, "low": 3.98, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.98, "open": 3.999, "day": "31"} +{"volume": 12198, "symbol": "NEPT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 3.99, "low": 3.99, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.99, "open": 3.99, "day": "31"} +{"volume": 7568, "symbol": "NEPT", "ts": "2018-08-31 09:52:00", "month": "08", "high": 3.9984, "low": 3.9984, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.9984, "open": 3.9984, "day": "31"} +{"volume": 1082, "symbol": "NEPT", "ts": "2018-08-31 09:53:00", "month": "08", "high": 3.9984, "low": 3.9984, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.9984, "open": 3.9984, "day": "31"} +{"volume": 37875, "symbol": "NEPT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 4.005, "low": 3.9992, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.005, "open": 3.9992, "day": "31"} +{"volume": 10226, "symbol": "NEPT", "ts": "2018-08-31 09:56:00", "month": "08", "high": 4.0202, "low": 4.0202, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.0202, "open": 4.0202, "day": "31"} +{"volume": 4181, "symbol": "NEPT", "ts": "2018-08-31 09:57:00", "month": "08", "high": 4.01, "low": 4.01, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.01, "open": 4.01, "day": "31"} +{"volume": 2609, "symbol": "NEPT", "ts": "2018-08-31 09:58:00", "month": "08", "high": 4.0, "low": 4.0, "key": "NEPT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 4.0, "open": 4.0, "day": "31"} +{"volume": 993, "symbol": "INGN", "ts": "2018-08-31 09:31:00", "month": "08", "high": 260.18, "low": 260.18, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.18, "open": 260.18, "day": "31"} +{"volume": 959, "symbol": "INGN", "ts": "2018-08-31 09:32:00", "month": "08", "high": 260.83, "low": 260.83, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.83, "open": 260.83, "day": "31"} +{"volume": 624, "symbol": "INGN", "ts": "2018-08-31 09:35:00", "month": "08", "high": 259.441, "low": 259.441, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 259.441, "open": 259.441, "day": "31"} +{"volume": 2588, "symbol": "INGN", "ts": "2018-08-31 09:38:00", "month": "08", "high": 260.79, "low": 260.79, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.79, "open": 260.79, "day": "31"} +{"volume": 2326, "symbol": "INGN", "ts": "2018-08-31 09:40:00", "month": "08", "high": 261.96, "low": 261.96, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 261.96, "open": 261.96, "day": "31"} +{"volume": 2602, "symbol": "INGN", "ts": "2018-08-31 09:41:00", "month": "08", "high": 260.02, "low": 260.01, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.01, "open": 260.02, "day": "31"} +{"volume": 2643, "symbol": "INGN", "ts": "2018-08-31 09:44:00", "month": "08", "high": 260.0, "low": 260.0, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.0, "open": 260.0, "day": "31"} +{"volume": 372, "symbol": "INGN", "ts": "2018-08-31 09:47:00", "month": "08", "high": 260.23, "low": 260.23, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.23, "open": 260.23, "day": "31"} +{"volume": 152, "symbol": "INGN", "ts": "2018-08-31 09:48:00", "month": "08", "high": 260.49, "low": 260.49, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.49, "open": 260.49, "day": "31"} +{"volume": 641, "symbol": "INGN", "ts": "2018-08-31 09:50:00", "month": "08", "high": 259.9955, "low": 259.9955, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 259.9955, "open": 259.9955, "day": "31"} +{"volume": 601, "symbol": "INGN", "ts": "2018-08-31 09:52:00", "month": "08", "high": 260.185, "low": 260.185, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.185, "open": 260.185, "day": "31"} +{"volume": 695, "symbol": "INGN", "ts": "2018-08-31 09:56:00", "month": "08", "high": 260.52, "low": 260.52, "key": "INGN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 260.52, "open": 260.52, "day": "31"} +{"volume": 100, "symbol": "GFN", "ts": "2018-08-31 09:34:00", "month": "08", "high": 13.25, "low": 13.25, "key": "GFN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.25, "open": 13.25, "day": "31"} +{"volume": 140, "symbol": "GFN", "ts": "2018-08-31 09:42:00", "month": "08", "high": 13.3252, "low": 13.3252, "key": "GFN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.3252, "open": 13.3252, "day": "31"} +{"volume": 200, "symbol": "GFN", "ts": "2018-08-31 09:49:00", "month": "08", "high": 13.4, "low": 13.4, "key": "GFN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 0, "symbol": "GFN", "ts": "2018-08-31 09:52:00", "month": "08", "high": 13.412, "low": 13.412, "key": "GFN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.412, "open": 13.412, "day": "31"} +{"volume": 6750, "symbol": "ECYT", "ts": "2018-08-31 09:32:00", "month": "08", "high": 19.2797, "low": 19.2797, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.2797, "open": 19.2797, "day": "31"} +{"volume": 4440, "symbol": "ECYT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 19.23, "low": 19.23, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.23, "open": 19.23, "day": "31"} +{"volume": 6799, "symbol": "ECYT", "ts": "2018-08-31 09:34:00", "month": "08", "high": 19.21, "low": 19.182, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.21, "open": 19.182, "day": "31"} +{"volume": 7163, "symbol": "ECYT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 19.21, "low": 19.1907, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.1907, "open": 19.21, "day": "31"} +{"volume": 2314, "symbol": "ECYT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 19.18, "low": 19.18, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.18, "day": "31"} +{"volume": 2420, "symbol": "ECYT", "ts": "2018-08-31 09:40:00", "month": "08", "high": 19.19, "low": 19.19, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.19, "open": 19.19, "day": "31"} +{"volume": 4346, "symbol": "ECYT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 19.2055, "low": 19.2055, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.2055, "open": 19.2055, "day": "31"} +{"volume": 2882, "symbol": "ECYT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 19.2, "low": 19.2, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.2, "open": 19.2, "day": "31"} +{"volume": 7092, "symbol": "ECYT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 19.16, "low": 19.16, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.16, "open": 19.16, "day": "31"} +{"volume": 3871, "symbol": "ECYT", "ts": "2018-08-31 09:44:00", "month": "08", "high": 19.0999, "low": 19.0999, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.0999, "open": 19.0999, "day": "31"} +{"volume": 4838, "symbol": "ECYT", "ts": "2018-08-31 09:45:00", "month": "08", "high": 19.08, "low": 19.08, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.08, "open": 19.08, "day": "31"} +{"volume": 5260, "symbol": "ECYT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 19.06, "low": 19.05, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.06, "open": 19.05, "day": "31"} +{"volume": 2611, "symbol": "ECYT", "ts": "2018-08-31 09:48:00", "month": "08", "high": 19.08, "low": 19.08, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.08, "open": 19.08, "day": "31"} +{"volume": 1728, "symbol": "ECYT", "ts": "2018-08-31 09:49:00", "month": "08", "high": 19.18, "low": 19.18, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.18, "day": "31"} +{"volume": 510, "symbol": "ECYT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 19.23, "low": 19.23, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.23, "open": 19.23, "day": "31"} +{"volume": 2540, "symbol": "ECYT", "ts": "2018-08-31 09:52:00", "month": "08", "high": 19.17, "low": 19.17, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.17, "open": 19.17, "day": "31"} +{"volume": 1435, "symbol": "ECYT", "ts": "2018-08-31 09:53:00", "month": "08", "high": 19.18, "low": 19.18, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.18, "day": "31"} +{"volume": 3226, "symbol": "ECYT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 19.15, "low": 19.15, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.15, "open": 19.15, "day": "31"} +{"volume": 4627, "symbol": "ECYT", "ts": "2018-08-31 09:56:00", "month": "08", "high": 19.1, "low": 19.09, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.1, "open": 19.09, "day": "31"} +{"volume": 3296, "symbol": "ECYT", "ts": "2018-08-31 09:59:00", "month": "08", "high": 19.02, "low": 19.02, "key": "ECYT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 19.02, "open": 19.02, "day": "31"} +{"volume": 425710, "symbol": "AMD", "ts": "2018-08-31 09:31:00", "month": "08", "high": 24.9643, "low": 24.921, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.9402, "open": 24.95, "day": "31"} +{"volume": 980060, "symbol": "AMD", "ts": "2018-08-31 09:32:00", "month": "08", "high": 25.11, "low": 24.97, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.0492, "open": 24.97, "day": "31"} +{"volume": 564114, "symbol": "AMD", "ts": "2018-08-31 09:33:00", "month": "08", "high": 25.1199, "low": 25.03, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.09, "open": 25.05, "day": "31"} +{"volume": 608314, "symbol": "AMD", "ts": "2018-08-31 09:34:00", "month": "08", "high": 25.16, "low": 25.04, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.05, "open": 25.0899, "day": "31"} +{"volume": 437549, "symbol": "AMD", "ts": "2018-08-31 09:35:00", "month": "08", "high": 25.0799, "low": 24.978, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.01, "open": 25.0799, "day": "31"} +{"volume": 412583, "symbol": "AMD", "ts": "2018-08-31 09:36:00", "month": "08", "high": 25.1199, "low": 25.0099, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.1199, "open": 25.0099, "day": "31"} +{"volume": 672086, "symbol": "AMD", "ts": "2018-08-31 09:37:00", "month": "08", "high": 25.199, "low": 25.11, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.199, "open": 25.11, "day": "31"} +{"volume": 684698, "symbol": "AMD", "ts": "2018-08-31 09:38:00", "month": "08", "high": 25.229, "low": 25.16, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.17, "open": 25.195, "day": "31"} +{"volume": 196481, "symbol": "AMD", "ts": "2018-08-31 09:39:00", "month": "08", "high": 25.18, "low": 25.14, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.16, "open": 25.17, "day": "31"} +{"volume": 400820, "symbol": "AMD", "ts": "2018-08-31 09:40:00", "month": "08", "high": 25.1525, "low": 25.08, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.14, "open": 25.14, "day": "31"} +{"volume": 338848, "symbol": "AMD", "ts": "2018-08-31 09:41:00", "month": "08", "high": 25.17, "low": 25.07, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.09, "open": 25.13, "day": "31"} +{"volume": 537706, "symbol": "AMD", "ts": "2018-08-31 09:42:00", "month": "08", "high": 25.09, "low": 24.96, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.0118, "open": 25.09, "day": "31"} +{"volume": 329309, "symbol": "AMD", "ts": "2018-08-31 09:43:00", "month": "08", "high": 25.06, "low": 24.97, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.975, "open": 25.01, "day": "31"} +{"volume": 730598, "symbol": "AMD", "ts": "2018-08-31 09:44:00", "month": "08", "high": 24.99, "low": 24.8573, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.9, "open": 24.99, "day": "31"} +{"volume": 526606, "symbol": "AMD", "ts": "2018-08-31 09:45:00", "month": "08", "high": 24.94, "low": 24.85, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.8616, "open": 24.9099, "day": "31"} +{"volume": 559731, "symbol": "AMD", "ts": "2018-08-31 09:46:00", "month": "08", "high": 24.9377, "low": 24.85, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.89, "open": 24.87, "day": "31"} +{"volume": 414089, "symbol": "AMD", "ts": "2018-08-31 09:47:00", "month": "08", "high": 24.98, "low": 24.8737, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.97, "open": 24.906, "day": "31"} +{"volume": 559909, "symbol": "AMD", "ts": "2018-08-31 09:48:00", "month": "08", "high": 25.046, "low": 24.9769, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.98, "open": 24.9769, "day": "31"} +{"volume": 267599, "symbol": "AMD", "ts": "2018-08-31 09:49:00", "month": "08", "high": 25.03, "low": 24.98, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.03, "open": 24.98, "day": "31"} +{"volume": 356975, "symbol": "AMD", "ts": "2018-08-31 09:50:00", "month": "08", "high": 25.06, "low": 24.94, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.95, "open": 25.04, "day": "31"} +{"volume": 202177, "symbol": "AMD", "ts": "2018-08-31 09:51:00", "month": "08", "high": 25.0, "low": 24.931, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.97, "open": 24.9501, "day": "31"} +{"volume": 353004, "symbol": "AMD", "ts": "2018-08-31 09:52:00", "month": "08", "high": 24.9633, "low": 24.87, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.9015, "open": 24.9633, "day": "31"} +{"volume": 235914, "symbol": "AMD", "ts": "2018-08-31 09:53:00", "month": "08", "high": 24.93, "low": 24.9, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.9, "open": 24.9, "day": "31"} +{"volume": 302895, "symbol": "AMD", "ts": "2018-08-31 09:54:00", "month": "08", "high": 24.93, "low": 24.87, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.9, "open": 24.9, "day": "31"} +{"volume": 217739, "symbol": "AMD", "ts": "2018-08-31 09:55:00", "month": "08", "high": 24.94, "low": 24.875, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.94, "open": 24.8811, "day": "31"} +{"volume": 327542, "symbol": "AMD", "ts": "2018-08-31 09:56:00", "month": "08", "high": 24.97, "low": 24.89, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.89, "open": 24.9472, "day": "31"} +{"volume": 168268, "symbol": "AMD", "ts": "2018-08-31 09:57:00", "month": "08", "high": 24.935, "low": 24.89, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.91, "open": 24.89, "day": "31"} +{"volume": 579375, "symbol": "AMD", "ts": "2018-08-31 09:58:00", "month": "08", "high": 24.9, "low": 24.795, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.8, "open": 24.9, "day": "31"} +{"volume": 367212, "symbol": "AMD", "ts": "2018-08-31 09:59:00", "month": "08", "high": 24.84, "low": 24.8, "key": "AMD_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.83, "open": 24.8, "day": "31"} +{"volume": 0, "symbol": "CDMO", "ts": "2018-08-31 09:31:00", "month": "08", "high": 6.9299, "low": 6.9299, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.9299, "open": 6.9299, "day": "31"} +{"volume": 10702, "symbol": "CDMO", "ts": "2018-08-31 09:32:00", "month": "08", "high": 6.91, "low": 6.8199, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.8199, "open": 6.91, "day": "31"} +{"volume": 7960, "symbol": "CDMO", "ts": "2018-08-31 09:33:00", "month": "08", "high": 6.8115, "low": 6.8115, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.8115, "open": 6.8115, "day": "31"} +{"volume": 8200, "symbol": "CDMO", "ts": "2018-08-31 09:34:00", "month": "08", "high": 6.81, "low": 6.81, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.81, "open": 6.81, "day": "31"} +{"volume": 12008, "symbol": "CDMO", "ts": "2018-08-31 09:36:00", "month": "08", "high": 6.85, "low": 6.85, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.85, "open": 6.85, "day": "31"} +{"volume": 11076, "symbol": "CDMO", "ts": "2018-08-31 09:37:00", "month": "08", "high": 6.88, "low": 6.88, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.88, "open": 6.88, "day": "31"} +{"volume": 2088, "symbol": "CDMO", "ts": "2018-08-31 09:38:00", "month": "08", "high": 6.94, "low": 6.94, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.94, "open": 6.94, "day": "31"} +{"volume": 38283, "symbol": "CDMO", "ts": "2018-08-31 09:39:00", "month": "08", "high": 7.001, "low": 6.991, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.001, "open": 6.991, "day": "31"} +{"volume": 11026, "symbol": "CDMO", "ts": "2018-08-31 09:40:00", "month": "08", "high": 6.975, "low": 6.975, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.975, "open": 6.975, "day": "31"} +{"volume": 6287, "symbol": "CDMO", "ts": "2018-08-31 09:41:00", "month": "08", "high": 7.025, "low": 7.025, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.025, "open": 7.025, "day": "31"} +{"volume": 19455, "symbol": "CDMO", "ts": "2018-08-31 09:42:00", "month": "08", "high": 7.04, "low": 7.04, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.04, "open": 7.04, "day": "31"} +{"volume": 6994, "symbol": "CDMO", "ts": "2018-08-31 09:43:00", "month": "08", "high": 7.064, "low": 7.064, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.064, "open": 7.064, "day": "31"} +{"volume": 2480, "symbol": "CDMO", "ts": "2018-08-31 09:46:00", "month": "08", "high": 7.07, "low": 7.07, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.07, "open": 7.07, "day": "31"} +{"volume": 7305, "symbol": "CDMO", "ts": "2018-08-31 09:47:00", "month": "08", "high": 7.03, "low": 7.0203, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.03, "open": 7.0203, "day": "31"} +{"volume": 1335, "symbol": "CDMO", "ts": "2018-08-31 09:50:00", "month": "08", "high": 7.037, "low": 7.037, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.037, "open": 7.037, "day": "31"} +{"volume": 6374, "symbol": "CDMO", "ts": "2018-08-31 09:51:00", "month": "08", "high": 7.0143, "low": 7.0143, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.0143, "open": 7.0143, "day": "31"} +{"volume": 1652, "symbol": "CDMO", "ts": "2018-08-31 09:52:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 981, "symbol": "CDMO", "ts": "2018-08-31 09:54:00", "month": "08", "high": 7.05, "low": 7.05, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.05, "open": 7.05, "day": "31"} +{"volume": 3800, "symbol": "CDMO", "ts": "2018-08-31 09:56:00", "month": "08", "high": 7.021, "low": 7.021, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.021, "open": 7.021, "day": "31"} +{"volume": 8577, "symbol": "CDMO", "ts": "2018-08-31 09:58:00", "month": "08", "high": 7.07, "low": 7.02, "key": "CDMO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 7.07, "open": 7.02, "day": "31"} +{"volume": 2052, "symbol": "SFIX", "ts": "2018-08-31 09:31:00", "month": "08", "high": 38.8932, "low": 38.8932, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 38.8932, "open": 38.8932, "day": "31"} +{"volume": 15648, "symbol": "SFIX", "ts": "2018-08-31 09:32:00", "month": "08", "high": 38.76, "low": 38.6395, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 38.6395, "open": 38.76, "day": "31"} +{"volume": 9303, "symbol": "SFIX", "ts": "2018-08-31 09:33:00", "month": "08", "high": 38.8595, "low": 38.69, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 38.8595, "open": 38.69, "day": "31"} +{"volume": 19041, "symbol": "SFIX", "ts": "2018-08-31 09:34:00", "month": "08", "high": 39.09, "low": 38.99, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.09, "open": 38.99, "day": "31"} +{"volume": 22051, "symbol": "SFIX", "ts": "2018-08-31 09:35:00", "month": "08", "high": 39.11, "low": 38.9501, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 38.9501, "open": 39.11, "day": "31"} +{"volume": 3424, "symbol": "SFIX", "ts": "2018-08-31 09:36:00", "month": "08", "high": 39.2, "low": 39.2, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.2, "open": 39.2, "day": "31"} +{"volume": 7630, "symbol": "SFIX", "ts": "2018-08-31 09:37:00", "month": "08", "high": 39.1607, "low": 39.1607, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.1607, "open": 39.1607, "day": "31"} +{"volume": 9686, "symbol": "SFIX", "ts": "2018-08-31 09:38:00", "month": "08", "high": 39.2, "low": 39.2, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.2, "open": 39.2, "day": "31"} +{"volume": 46272, "symbol": "SFIX", "ts": "2018-08-31 09:39:00", "month": "08", "high": 39.55, "low": 39.27, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.55, "open": 39.27, "day": "31"} +{"volume": 21874, "symbol": "SFIX", "ts": "2018-08-31 09:40:00", "month": "08", "high": 39.7502, "low": 39.51, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.7502, "open": 39.51, "day": "31"} +{"volume": 28007, "symbol": "SFIX", "ts": "2018-08-31 09:41:00", "month": "08", "high": 39.9, "low": 39.68, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.68, "open": 39.85, "day": "31"} +{"volume": 16312, "symbol": "SFIX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 39.7386, "low": 39.56, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.56, "open": 39.7386, "day": "31"} +{"volume": 8525, "symbol": "SFIX", "ts": "2018-08-31 09:43:00", "month": "08", "high": 39.7116, "low": 39.58, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.7116, "open": 39.58, "day": "31"} +{"volume": 3270, "symbol": "SFIX", "ts": "2018-08-31 09:44:00", "month": "08", "high": 39.765, "low": 39.765, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.765, "open": 39.765, "day": "31"} +{"volume": 13966, "symbol": "SFIX", "ts": "2018-08-31 09:45:00", "month": "08", "high": 39.768, "low": 39.6805, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.768, "open": 39.6805, "day": "31"} +{"volume": 7516, "symbol": "SFIX", "ts": "2018-08-31 09:46:00", "month": "08", "high": 39.7, "low": 39.7, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.7, "open": 39.7, "day": "31"} +{"volume": 7818, "symbol": "SFIX", "ts": "2018-08-31 09:47:00", "month": "08", "high": 39.6833, "low": 39.68, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.68, "open": 39.6833, "day": "31"} +{"volume": 4318, "symbol": "SFIX", "ts": "2018-08-31 09:48:00", "month": "08", "high": 39.6925, "low": 39.6925, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.6925, "open": 39.6925, "day": "31"} +{"volume": 8024, "symbol": "SFIX", "ts": "2018-08-31 09:49:00", "month": "08", "high": 39.8, "low": 39.7175, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.7175, "open": 39.8, "day": "31"} +{"volume": 1307, "symbol": "SFIX", "ts": "2018-08-31 09:50:00", "month": "08", "high": 39.9, "low": 39.9, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.9, "open": 39.9, "day": "31"} +{"volume": 7655, "symbol": "SFIX", "ts": "2018-08-31 09:51:00", "month": "08", "high": 39.86, "low": 39.86, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.86, "open": 39.86, "day": "31"} +{"volume": 11259, "symbol": "SFIX", "ts": "2018-08-31 09:52:00", "month": "08", "high": 39.905, "low": 39.9, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.905, "open": 39.9, "day": "31"} +{"volume": 22753, "symbol": "SFIX", "ts": "2018-08-31 09:53:00", "month": "08", "high": 39.88, "low": 39.71, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.71, "open": 39.88, "day": "31"} +{"volume": 2951, "symbol": "SFIX", "ts": "2018-08-31 09:54:00", "month": "08", "high": 39.712, "low": 39.712, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.712, "open": 39.712, "day": "31"} +{"volume": 2319, "symbol": "SFIX", "ts": "2018-08-31 09:55:00", "month": "08", "high": 39.7, "low": 39.7, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.7, "open": 39.7, "day": "31"} +{"volume": 20100, "symbol": "SFIX", "ts": "2018-08-31 09:56:00", "month": "08", "high": 39.8, "low": 39.8, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.8, "open": 39.8, "day": "31"} +{"volume": 10685, "symbol": "SFIX", "ts": "2018-08-31 09:57:00", "month": "08", "high": 39.9, "low": 39.72, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.9, "open": 39.72, "day": "31"} +{"volume": 9524, "symbol": "SFIX", "ts": "2018-08-31 09:58:00", "month": "08", "high": 39.8558, "low": 39.7, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.7, "open": 39.8558, "day": "31"} +{"volume": 8873, "symbol": "SFIX", "ts": "2018-08-31 09:59:00", "month": "08", "high": 39.72, "low": 39.72, "key": "SFIX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 39.72, "open": 39.72, "day": "31"} +{"volume": 367, "symbol": "DRNA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 15.9786, "low": 15.9786, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.9786, "open": 15.9786, "day": "31"} +{"volume": 1402, "symbol": "DRNA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 16.0, "low": 16.0, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.0, "open": 16.0, "day": "31"} +{"volume": 1498, "symbol": "DRNA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 16.01, "low": 16.01, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.01, "open": 16.01, "day": "31"} +{"volume": 500, "symbol": "DRNA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 15.89, "low": 15.89, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.89, "open": 15.89, "day": "31"} +{"volume": 1200, "symbol": "DRNA", "ts": "2018-08-31 09:49:00", "month": "08", "high": 15.8827, "low": 15.8827, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 15.8827, "open": 15.8827, "day": "31"} +{"volume": 334, "symbol": "DRNA", "ts": "2018-08-31 09:51:00", "month": "08", "high": 16.0, "low": 16.0, "key": "DRNA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.0, "open": 16.0, "day": "31"} +{"volume": 100, "symbol": "RFP", "ts": "2018-08-31 09:37:00", "month": "08", "high": 13.8, "low": 13.8, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.8, "open": 13.8, "day": "31"} +{"volume": 1595, "symbol": "RFP", "ts": "2018-08-31 09:38:00", "month": "08", "high": 13.75, "low": 13.75, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.75, "open": 13.75, "day": "31"} +{"volume": 503, "symbol": "RFP", "ts": "2018-08-31 09:42:00", "month": "08", "high": 13.8, "low": 13.8, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.8, "open": 13.8, "day": "31"} +{"volume": 5845, "symbol": "RFP", "ts": "2018-08-31 09:44:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 3900, "symbol": "RFP", "ts": "2018-08-31 09:46:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 2101, "symbol": "RFP", "ts": "2018-08-31 09:47:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 1413, "symbol": "RFP", "ts": "2018-08-31 09:50:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 1639, "symbol": "RFP", "ts": "2018-08-31 09:51:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 1116, "symbol": "RFP", "ts": "2018-08-31 09:52:00", "month": "08", "high": 13.85, "low": 13.85, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.85, "open": 13.85, "day": "31"} +{"volume": 850, "symbol": "RFP", "ts": "2018-08-31 09:54:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 200, "symbol": "RFP", "ts": "2018-08-31 09:55:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 100, "symbol": "RFP", "ts": "2018-08-31 09:57:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 4606, "symbol": "RFP", "ts": "2018-08-31 09:58:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 0, "symbol": "BJRI", "ts": "2018-08-31 09:31:00", "month": "08", "high": 74.075, "low": 74.075, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.075, "open": 74.075, "day": "31"} +{"volume": 1472, "symbol": "BJRI", "ts": "2018-08-31 09:33:00", "month": "08", "high": 74.495, "low": 74.495, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.495, "open": 74.495, "day": "31"} +{"volume": 1992, "symbol": "BJRI", "ts": "2018-08-31 09:36:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 1832, "symbol": "BJRI", "ts": "2018-08-31 09:38:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 1232, "symbol": "BJRI", "ts": "2018-08-31 09:42:00", "month": "08", "high": 74.75, "low": 74.75, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.75, "open": 74.75, "day": "31"} +{"volume": 1116, "symbol": "BJRI", "ts": "2018-08-31 09:44:00", "month": "08", "high": 74.75, "low": 74.75, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.75, "open": 74.75, "day": "31"} +{"volume": 1914, "symbol": "BJRI", "ts": "2018-08-31 09:46:00", "month": "08", "high": 74.6, "low": 74.6, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.6, "open": 74.6, "day": "31"} +{"volume": 1564, "symbol": "BJRI", "ts": "2018-08-31 09:47:00", "month": "08", "high": 74.7, "low": 74.7, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.7, "open": 74.7, "day": "31"} +{"volume": 988, "symbol": "BJRI", "ts": "2018-08-31 09:49:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 6395, "symbol": "BJRI", "ts": "2018-08-31 09:51:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 2410, "symbol": "BJRI", "ts": "2018-08-31 09:52:00", "month": "08", "high": 74.4, "low": 74.4, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.4, "open": 74.4, "day": "31"} +{"volume": 144, "symbol": "BJRI", "ts": "2018-08-31 09:57:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 1193, "symbol": "BJRI", "ts": "2018-08-31 09:59:00", "month": "08", "high": 74.4, "low": 74.4, "key": "BJRI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 74.4, "open": 74.4, "day": "31"} +{"volume": 0, "symbol": "NVCR", "ts": "2018-08-31 09:31:00", "month": "08", "high": 43.65, "low": 43.65, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 43.65, "open": 43.65, "day": "31"} +{"volume": 2868, "symbol": "NVCR", "ts": "2018-08-31 09:32:00", "month": "08", "high": 43.85, "low": 43.85, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 43.85, "open": 43.85, "day": "31"} +{"volume": 9309, "symbol": "NVCR", "ts": "2018-08-31 09:35:00", "month": "08", "high": 43.7234, "low": 43.7234, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 43.7234, "open": 43.7234, "day": "31"} +{"volume": 5048, "symbol": "NVCR", "ts": "2018-08-31 09:36:00", "month": "08", "high": 44.0, "low": 44.0, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.0, "open": 44.0, "day": "31"} +{"volume": 9532, "symbol": "NVCR", "ts": "2018-08-31 09:37:00", "month": "08", "high": 44.5, "low": 44.5, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.5, "open": 44.5, "day": "31"} +{"volume": 2137, "symbol": "NVCR", "ts": "2018-08-31 09:38:00", "month": "08", "high": 44.8, "low": 44.8, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.8, "open": 44.8, "day": "31"} +{"volume": 4694, "symbol": "NVCR", "ts": "2018-08-31 09:39:00", "month": "08", "high": 44.45, "low": 44.45, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.45, "open": 44.45, "day": "31"} +{"volume": 11799, "symbol": "NVCR", "ts": "2018-08-31 09:40:00", "month": "08", "high": 44.45, "low": 44.445, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.445, "open": 44.45, "day": "31"} +{"volume": 2208, "symbol": "NVCR", "ts": "2018-08-31 09:41:00", "month": "08", "high": 44.299, "low": 44.299, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.299, "open": 44.299, "day": "31"} +{"volume": 1469, "symbol": "NVCR", "ts": "2018-08-31 09:42:00", "month": "08", "high": 44.175, "low": 44.175, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.175, "open": 44.175, "day": "31"} +{"volume": 1324, "symbol": "NVCR", "ts": "2018-08-31 09:43:00", "month": "08", "high": 44.2, "low": 44.2, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.2, "open": 44.2, "day": "31"} +{"volume": 2616, "symbol": "NVCR", "ts": "2018-08-31 09:44:00", "month": "08", "high": 44.15, "low": 44.15, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.15, "open": 44.15, "day": "31"} +{"volume": 4679, "symbol": "NVCR", "ts": "2018-08-31 09:46:00", "month": "08", "high": 44.125, "low": 44.125, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.125, "open": 44.125, "day": "31"} +{"volume": 1629, "symbol": "NVCR", "ts": "2018-08-31 09:47:00", "month": "08", "high": 43.95, "low": 43.95, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 43.95, "open": 43.95, "day": "31"} +{"volume": 3034, "symbol": "NVCR", "ts": "2018-08-31 09:48:00", "month": "08", "high": 44.2, "low": 44.2, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.2, "open": 44.2, "day": "31"} +{"volume": 1683, "symbol": "NVCR", "ts": "2018-08-31 09:49:00", "month": "08", "high": 44.2001, "low": 44.2001, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.2001, "open": 44.2001, "day": "31"} +{"volume": 904, "symbol": "NVCR", "ts": "2018-08-31 09:50:00", "month": "08", "high": 44.275, "low": 44.275, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.275, "open": 44.275, "day": "31"} +{"volume": 2012, "symbol": "NVCR", "ts": "2018-08-31 09:52:00", "month": "08", "high": 44.1, "low": 44.1, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.1, "open": 44.1, "day": "31"} +{"volume": 4318, "symbol": "NVCR", "ts": "2018-08-31 09:53:00", "month": "08", "high": 44.225, "low": 44.225, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.225, "open": 44.225, "day": "31"} +{"volume": 7694, "symbol": "NVCR", "ts": "2018-08-31 09:54:00", "month": "08", "high": 44.35, "low": 44.334, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.334, "open": 44.35, "day": "31"} +{"volume": 4479, "symbol": "NVCR", "ts": "2018-08-31 09:56:00", "month": "08", "high": 44.15, "low": 44.15, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.15, "open": 44.15, "day": "31"} +{"volume": 1892, "symbol": "NVCR", "ts": "2018-08-31 09:58:00", "month": "08", "high": 44.175, "low": 44.175, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.175, "open": 44.175, "day": "31"} +{"volume": 760, "symbol": "NVCR", "ts": "2018-08-31 09:59:00", "month": "08", "high": 44.1982, "low": 44.1982, "key": "NVCR_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 44.1982, "open": 44.1982, "day": "31"} +{"volume": 32391, "symbol": "NOG", "ts": "2018-08-31 09:31:00", "month": "08", "high": 3.41, "low": 3.38, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.38, "open": 3.41, "day": "31"} +{"volume": 2650, "symbol": "NOG", "ts": "2018-08-31 09:32:00", "month": "08", "high": 3.3941, "low": 3.3941, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.3941, "open": 3.3941, "day": "31"} +{"volume": 21430, "symbol": "NOG", "ts": "2018-08-31 09:33:00", "month": "08", "high": 3.3922, "low": 3.3922, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.3922, "open": 3.3922, "day": "31"} +{"volume": 8900, "symbol": "NOG", "ts": "2018-08-31 09:34:00", "month": "08", "high": 3.39, "low": 3.39, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.39, "open": 3.39, "day": "31"} +{"volume": 3785, "symbol": "NOG", "ts": "2018-08-31 09:35:00", "month": "08", "high": 3.38, "low": 3.38, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.38, "open": 3.38, "day": "31"} +{"volume": 399, "symbol": "NOG", "ts": "2018-08-31 09:36:00", "month": "08", "high": 3.39, "low": 3.39, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.39, "open": 3.39, "day": "31"} +{"volume": 3191, "symbol": "NOG", "ts": "2018-08-31 09:37:00", "month": "08", "high": 3.385, "low": 3.385, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.385, "open": 3.385, "day": "31"} +{"volume": 27120, "symbol": "NOG", "ts": "2018-08-31 09:38:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 20938, "symbol": "NOG", "ts": "2018-08-31 09:39:00", "month": "08", "high": 3.38, "low": 3.3701, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.3701, "open": 3.38, "day": "31"} +{"volume": 10968, "symbol": "NOG", "ts": "2018-08-31 09:41:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 1700, "symbol": "NOG", "ts": "2018-08-31 09:42:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 9964, "symbol": "NOG", "ts": "2018-08-31 09:44:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 38337, "symbol": "NOG", "ts": "2018-08-31 09:45:00", "month": "08", "high": 3.37, "low": 3.3651, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.3651, "day": "31"} +{"volume": 8330, "symbol": "NOG", "ts": "2018-08-31 09:46:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 17713, "symbol": "NOG", "ts": "2018-08-31 09:47:00", "month": "08", "high": 3.3573, "low": 3.3573, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.3573, "open": 3.3573, "day": "31"} +{"volume": 1965, "symbol": "NOG", "ts": "2018-08-31 09:49:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 2350, "symbol": "NOG", "ts": "2018-08-31 09:50:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 19928, "symbol": "NOG", "ts": "2018-08-31 09:52:00", "month": "08", "high": 3.37, "low": 3.36, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.37, "day": "31"} +{"volume": 2364, "symbol": "NOG", "ts": "2018-08-31 09:54:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 4475, "symbol": "NOG", "ts": "2018-08-31 09:55:00", "month": "08", "high": 3.351, "low": 3.351, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.351, "open": 3.351, "day": "31"} +{"volume": 8837, "symbol": "NOG", "ts": "2018-08-31 09:56:00", "month": "08", "high": 3.355, "low": 3.35, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.35, "day": "31"} +{"volume": 57148, "symbol": "NOG", "ts": "2018-08-31 09:58:00", "month": "08", "high": 3.3578, "low": 3.35, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.3578, "open": 3.35, "day": "31"} +{"volume": 4384, "symbol": "NOG", "ts": "2018-08-31 09:59:00", "month": "08", "high": 3.35, "low": 3.35, "key": "NOG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 3.35, "open": 3.35, "day": "31"} +{"volume": 450, "symbol": "PTSI", "ts": "2018-08-31 09:56:00", "month": "08", "high": 58.2311, "low": 58.2311, "key": "PTSI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 58.2311, "open": 58.2311, "day": "31"} +{"volume": 0, "symbol": "BAND", "ts": "2018-08-31 09:32:00", "month": "08", "high": 45.22, "low": 45.22, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.22, "open": 45.22, "day": "31"} +{"volume": 760, "symbol": "BAND", "ts": "2018-08-31 09:36:00", "month": "08", "high": 45.2499, "low": 45.2499, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.2499, "open": 45.2499, "day": "31"} +{"volume": 1424, "symbol": "BAND", "ts": "2018-08-31 09:41:00", "month": "08", "high": 45.13, "low": 45.13, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.13, "open": 45.13, "day": "31"} +{"volume": 4036, "symbol": "BAND", "ts": "2018-08-31 09:42:00", "month": "08", "high": 45.13, "low": 45.13, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.13, "open": 45.13, "day": "31"} +{"volume": 623, "symbol": "BAND", "ts": "2018-08-31 09:45:00", "month": "08", "high": 45.12, "low": 45.12, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.12, "open": 45.12, "day": "31"} +{"volume": 601, "symbol": "BAND", "ts": "2018-08-31 09:49:00", "month": "08", "high": 45.0204, "low": 45.0204, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.0204, "open": 45.0204, "day": "31"} +{"volume": 305, "symbol": "BAND", "ts": "2018-08-31 09:53:00", "month": "08", "high": 45.08, "low": 45.08, "key": "BAND_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 45.08, "open": 45.08, "day": "31"} +{"volume": 6108, "symbol": "AMRS", "ts": "2018-08-31 09:31:00", "month": "08", "high": 8.6075, "low": 8.6075, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.6075, "open": 8.6075, "day": "31"} +{"volume": 23751, "symbol": "AMRS", "ts": "2018-08-31 09:32:00", "month": "08", "high": 8.595, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.57, "open": 8.52, "day": "31"} +{"volume": 2913, "symbol": "AMRS", "ts": "2018-08-31 09:33:00", "month": "08", "high": 8.54, "low": 8.54, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.54, "open": 8.54, "day": "31"} +{"volume": 10595, "symbol": "AMRS", "ts": "2018-08-31 09:34:00", "month": "08", "high": 8.54, "low": 8.54, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.54, "open": 8.54, "day": "31"} +{"volume": 6217, "symbol": "AMRS", "ts": "2018-08-31 09:35:00", "month": "08", "high": 8.49, "low": 8.49, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.49, "open": 8.49, "day": "31"} +{"volume": 9352, "symbol": "AMRS", "ts": "2018-08-31 09:36:00", "month": "08", "high": 8.48, "low": 8.48, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.48, "open": 8.48, "day": "31"} +{"volume": 5905, "symbol": "AMRS", "ts": "2018-08-31 09:37:00", "month": "08", "high": 8.4498, "low": 8.4498, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.4498, "open": 8.4498, "day": "31"} +{"volume": 1611, "symbol": "AMRS", "ts": "2018-08-31 09:38:00", "month": "08", "high": 8.4209, "low": 8.4209, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.4209, "open": 8.4209, "day": "31"} +{"volume": 24077, "symbol": "AMRS", "ts": "2018-08-31 09:40:00", "month": "08", "high": 8.595, "low": 8.4999, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.595, "open": 8.4999, "day": "31"} +{"volume": 16236, "symbol": "AMRS", "ts": "2018-08-31 09:41:00", "month": "08", "high": 8.593, "low": 8.59, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.59, "open": 8.593, "day": "31"} +{"volume": 29983, "symbol": "AMRS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 8.67, "low": 8.5999, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.5999, "open": 8.62, "day": "31"} +{"volume": 17858, "symbol": "AMRS", "ts": "2018-08-31 09:43:00", "month": "08", "high": 8.5933, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.5933, "open": 8.52, "day": "31"} +{"volume": 8182, "symbol": "AMRS", "ts": "2018-08-31 09:44:00", "month": "08", "high": 8.52, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.52, "open": 8.52, "day": "31"} +{"volume": 13517, "symbol": "AMRS", "ts": "2018-08-31 09:45:00", "month": "08", "high": 8.52, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.52, "open": 8.52, "day": "31"} +{"volume": 52256, "symbol": "AMRS", "ts": "2018-08-31 09:46:00", "month": "08", "high": 8.4921, "low": 8.48, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.48, "open": 8.4921, "day": "31"} +{"volume": 4062, "symbol": "AMRS", "ts": "2018-08-31 09:48:00", "month": "08", "high": 8.52, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.52, "open": 8.52, "day": "31"} +{"volume": 7059, "symbol": "AMRS", "ts": "2018-08-31 09:49:00", "month": "08", "high": 8.5, "low": 8.5, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.5, "open": 8.5, "day": "31"} +{"volume": 8023, "symbol": "AMRS", "ts": "2018-08-31 09:50:00", "month": "08", "high": 8.52, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.52, "open": 8.52, "day": "31"} +{"volume": 1786, "symbol": "AMRS", "ts": "2018-08-31 09:51:00", "month": "08", "high": 8.53, "low": 8.53, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.53, "open": 8.53, "day": "31"} +{"volume": 6483, "symbol": "AMRS", "ts": "2018-08-31 09:52:00", "month": "08", "high": 8.5166, "low": 8.5166, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.5166, "open": 8.5166, "day": "31"} +{"volume": 6538, "symbol": "AMRS", "ts": "2018-08-31 09:53:00", "month": "08", "high": 8.56, "low": 8.56, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.56, "open": 8.56, "day": "31"} +{"volume": 8900, "symbol": "AMRS", "ts": "2018-08-31 09:54:00", "month": "08", "high": 8.56, "low": 8.51, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.51, "open": 8.56, "day": "31"} +{"volume": 4250, "symbol": "AMRS", "ts": "2018-08-31 09:55:00", "month": "08", "high": 8.52, "low": 8.52, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.52, "open": 8.52, "day": "31"} +{"volume": 8722, "symbol": "AMRS", "ts": "2018-08-31 09:56:00", "month": "08", "high": 8.46, "low": 8.46, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.46, "open": 8.46, "day": "31"} +{"volume": 6293, "symbol": "AMRS", "ts": "2018-08-31 09:57:00", "month": "08", "high": 8.47, "low": 8.45, "key": "AMRS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 8.45, "open": 8.47, "day": "31"} +{"volume": 3903, "symbol": "COUP", "ts": "2018-08-31 09:33:00", "month": "08", "high": 71.15, "low": 70.92, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.15, "open": 70.92, "day": "31"} +{"volume": 5878, "symbol": "COUP", "ts": "2018-08-31 09:35:00", "month": "08", "high": 71.07, "low": 71.07, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.07, "open": 71.07, "day": "31"} +{"volume": 2017, "symbol": "COUP", "ts": "2018-08-31 09:36:00", "month": "08", "high": 71.27, "low": 71.27, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.27, "open": 71.27, "day": "31"} +{"volume": 6715, "symbol": "COUP", "ts": "2018-08-31 09:38:00", "month": "08", "high": 71.39, "low": 71.35, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.39, "open": 71.35, "day": "31"} +{"volume": 3160, "symbol": "COUP", "ts": "2018-08-31 09:39:00", "month": "08", "high": 71.43, "low": 71.43, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.43, "open": 71.43, "day": "31"} +{"volume": 2001, "symbol": "COUP", "ts": "2018-08-31 09:40:00", "month": "08", "high": 71.35, "low": 71.35, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.35, "open": 71.35, "day": "31"} +{"volume": 2627, "symbol": "COUP", "ts": "2018-08-31 09:41:00", "month": "08", "high": 71.3747, "low": 71.3747, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.3747, "open": 71.3747, "day": "31"} +{"volume": 2592, "symbol": "COUP", "ts": "2018-08-31 09:42:00", "month": "08", "high": 71.14, "low": 71.14, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.14, "open": 71.14, "day": "31"} +{"volume": 2061, "symbol": "COUP", "ts": "2018-08-31 09:44:00", "month": "08", "high": 70.8201, "low": 70.8201, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.8201, "open": 70.8201, "day": "31"} +{"volume": 3379, "symbol": "COUP", "ts": "2018-08-31 09:46:00", "month": "08", "high": 70.63, "low": 70.63, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.63, "open": 70.63, "day": "31"} +{"volume": 2624, "symbol": "COUP", "ts": "2018-08-31 09:47:00", "month": "08", "high": 70.68, "low": 70.6564, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.6564, "open": 70.68, "day": "31"} +{"volume": 1609, "symbol": "COUP", "ts": "2018-08-31 09:50:00", "month": "08", "high": 70.8, "low": 70.8, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.8, "open": 70.8, "day": "31"} +{"volume": 15723, "symbol": "COUP", "ts": "2018-08-31 09:51:00", "month": "08", "high": 70.88, "low": 70.88, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.88, "open": 70.88, "day": "31"} +{"volume": 516, "symbol": "COUP", "ts": "2018-08-31 09:52:00", "month": "08", "high": 70.81, "low": 70.81, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.81, "open": 70.81, "day": "31"} +{"volume": 2992, "symbol": "COUP", "ts": "2018-08-31 09:55:00", "month": "08", "high": 70.82, "low": 70.82, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.82, "open": 70.82, "day": "31"} +{"volume": 208, "symbol": "COUP", "ts": "2018-08-31 09:57:00", "month": "08", "high": 70.71, "low": 70.71, "key": "COUP_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.71, "open": 70.71, "day": "31"} +{"volume": 2546, "symbol": "AAXN", "ts": "2018-08-31 09:32:00", "month": "08", "high": 66.73, "low": 66.73, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.73, "open": 66.73, "day": "31"} +{"volume": 430, "symbol": "AAXN", "ts": "2018-08-31 09:34:00", "month": "08", "high": 66.5829, "low": 66.5829, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.5829, "open": 66.5829, "day": "31"} +{"volume": 1529, "symbol": "AAXN", "ts": "2018-08-31 09:36:00", "month": "08", "high": 66.58, "low": 66.58, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.58, "open": 66.58, "day": "31"} +{"volume": 2140, "symbol": "AAXN", "ts": "2018-08-31 09:38:00", "month": "08", "high": 66.85, "low": 66.85, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.85, "open": 66.85, "day": "31"} +{"volume": 2280, "symbol": "AAXN", "ts": "2018-08-31 09:40:00", "month": "08", "high": 67.0, "low": 67.0, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 67.0, "open": 67.0, "day": "31"} +{"volume": 204, "symbol": "AAXN", "ts": "2018-08-31 09:44:00", "month": "08", "high": 67.17, "low": 67.17, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 67.17, "open": 67.17, "day": "31"} +{"volume": 1541, "symbol": "AAXN", "ts": "2018-08-31 09:46:00", "month": "08", "high": 66.92, "low": 66.92, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.92, "open": 66.92, "day": "31"} +{"volume": 463, "symbol": "AAXN", "ts": "2018-08-31 09:48:00", "month": "08", "high": 66.8293, "low": 66.8293, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.8293, "open": 66.8293, "day": "31"} +{"volume": 5108, "symbol": "AAXN", "ts": "2018-08-31 09:52:00", "month": "08", "high": 67.2, "low": 67.2, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 67.2, "open": 67.2, "day": "31"} +{"volume": 1865, "symbol": "AAXN", "ts": "2018-08-31 09:53:00", "month": "08", "high": 67.1201, "low": 67.1201, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 67.1201, "open": 67.1201, "day": "31"} +{"volume": 609, "symbol": "AAXN", "ts": "2018-08-31 09:54:00", "month": "08", "high": 67.13, "low": 67.13, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 67.13, "open": 67.13, "day": "31"} +{"volume": 847, "symbol": "AAXN", "ts": "2018-08-31 09:58:00", "month": "08", "high": 67.07, "low": 67.07, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 67.07, "open": 67.07, "day": "31"} +{"volume": 807, "symbol": "AAXN", "ts": "2018-08-31 09:59:00", "month": "08", "high": 66.87, "low": 66.87, "key": "AAXN_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 66.87, "open": 66.87, "day": "31"} +{"volume": 20975, "symbol": "OKTA", "ts": "2018-08-31 09:31:00", "month": "08", "high": 61.2, "low": 61.2, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.2, "open": 61.2, "day": "31"} +{"volume": 10188, "symbol": "OKTA", "ts": "2018-08-31 09:33:00", "month": "08", "high": 61.7, "low": 61.6, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.7, "open": 61.6, "day": "31"} +{"volume": 1751, "symbol": "OKTA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 61.725, "low": 61.725, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.725, "open": 61.725, "day": "31"} +{"volume": 4701, "symbol": "OKTA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 61.4678, "low": 61.4678, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.4678, "open": 61.4678, "day": "31"} +{"volume": 5156, "symbol": "OKTA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 61.3354, "low": 61.3354, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.3354, "open": 61.3354, "day": "31"} +{"volume": 5748, "symbol": "OKTA", "ts": "2018-08-31 09:38:00", "month": "08", "high": 61.65, "low": 61.25, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.65, "open": 61.25, "day": "31"} +{"volume": 2292, "symbol": "OKTA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 61.94, "low": 61.94, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.94, "open": 61.94, "day": "31"} +{"volume": 1417, "symbol": "OKTA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 62.06, "low": 62.06, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 62.06, "open": 62.06, "day": "31"} +{"volume": 3827, "symbol": "OKTA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 61.851, "low": 61.851, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.851, "open": 61.851, "day": "31"} +{"volume": 1340, "symbol": "OKTA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 61.7987, "low": 61.7987, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.7987, "open": 61.7987, "day": "31"} +{"volume": 424, "symbol": "OKTA", "ts": "2018-08-31 09:44:00", "month": "08", "high": 61.78, "low": 61.78, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.78, "open": 61.78, "day": "31"} +{"volume": 4195, "symbol": "OKTA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 61.455, "low": 61.455, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.455, "open": 61.455, "day": "31"} +{"volume": 1350, "symbol": "OKTA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 61.49, "low": 61.49, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.49, "open": 61.49, "day": "31"} +{"volume": 9116, "symbol": "OKTA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 61.53, "low": 61.5, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.5, "open": 61.53, "day": "31"} +{"volume": 2270, "symbol": "OKTA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 61.58, "low": 61.58, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.58, "open": 61.58, "day": "31"} +{"volume": 2912, "symbol": "OKTA", "ts": "2018-08-31 09:53:00", "month": "08", "high": 61.5, "low": 61.5, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.5, "open": 61.5, "day": "31"} +{"volume": 2043, "symbol": "OKTA", "ts": "2018-08-31 09:55:00", "month": "08", "high": 61.44, "low": 61.44, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.44, "open": 61.44, "day": "31"} +{"volume": 4532, "symbol": "OKTA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 61.44, "low": 61.44, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.44, "open": 61.44, "day": "31"} +{"volume": 5504, "symbol": "OKTA", "ts": "2018-08-31 09:58:00", "month": "08", "high": 61.37, "low": 61.22, "key": "OKTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 61.22, "open": 61.37, "day": "31"} +{"volume": 11229, "symbol": "EVBG", "ts": "2018-08-31 09:34:00", "month": "08", "high": 60.33, "low": 60.33, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 60.33, "open": 60.33, "day": "31"} +{"volume": 3039, "symbol": "EVBG", "ts": "2018-08-31 09:36:00", "month": "08", "high": 60.39, "low": 59.95, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.95, "open": 60.39, "day": "31"} +{"volume": 2065, "symbol": "EVBG", "ts": "2018-08-31 09:38:00", "month": "08", "high": 59.9, "low": 59.9, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.9, "open": 59.9, "day": "31"} +{"volume": 1323, "symbol": "EVBG", "ts": "2018-08-31 09:40:00", "month": "08", "high": 60.0, "low": 60.0, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 60.0, "open": 60.0, "day": "31"} +{"volume": 850, "symbol": "EVBG", "ts": "2018-08-31 09:41:00", "month": "08", "high": 60.0, "low": 60.0, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 60.0, "open": 60.0, "day": "31"} +{"volume": 200, "symbol": "EVBG", "ts": "2018-08-31 09:44:00", "month": "08", "high": 60.2, "low": 60.2, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 60.2, "open": 60.2, "day": "31"} +{"volume": 2773, "symbol": "EVBG", "ts": "2018-08-31 09:46:00", "month": "08", "high": 59.98, "low": 59.98, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.98, "open": 59.98, "day": "31"} +{"volume": 458, "symbol": "EVBG", "ts": "2018-08-31 09:47:00", "month": "08", "high": 60.01, "low": 60.01, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 60.01, "open": 60.01, "day": "31"} +{"volume": 222, "symbol": "EVBG", "ts": "2018-08-31 09:50:00", "month": "08", "high": 59.97, "low": 59.97, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.97, "open": 59.97, "day": "31"} +{"volume": 2152, "symbol": "EVBG", "ts": "2018-08-31 09:53:00", "month": "08", "high": 59.987, "low": 59.987, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.987, "open": 59.987, "day": "31"} +{"volume": 2221, "symbol": "EVBG", "ts": "2018-08-31 09:56:00", "month": "08", "high": 59.75, "low": 59.75, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.75, "open": 59.75, "day": "31"} +{"volume": 215, "symbol": "EVBG", "ts": "2018-08-31 09:57:00", "month": "08", "high": 59.67, "low": 59.67, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.67, "open": 59.67, "day": "31"} +{"volume": 124, "symbol": "EVBG", "ts": "2018-08-31 09:59:00", "month": "08", "high": 59.34, "low": 59.34, "key": "EVBG_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 59.34, "open": 59.34, "day": "31"} +{"volume": 0, "symbol": "RETA", "ts": "2018-08-31 09:31:00", "month": "08", "high": 86.98, "low": 86.98, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 86.98, "open": 86.98, "day": "31"} +{"volume": 1686, "symbol": "RETA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 87.19, "low": 87.19, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.19, "open": 87.19, "day": "31"} +{"volume": 3155, "symbol": "RETA", "ts": "2018-08-31 09:35:00", "month": "08", "high": 88.0, "low": 88.0, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.0, "open": 88.0, "day": "31"} +{"volume": 1534, "symbol": "RETA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 88.88, "low": 88.88, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.88, "open": 88.88, "day": "31"} +{"volume": 6489, "symbol": "RETA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 88.69, "low": 88.435, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.435, "open": 88.69, "day": "31"} +{"volume": 1222, "symbol": "RETA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 89.58, "low": 89.58, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.58, "open": 89.58, "day": "31"} +{"volume": 838, "symbol": "RETA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 89.39, "low": 89.39, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.39, "open": 89.39, "day": "31"} +{"volume": 2335, "symbol": "RETA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 89.38, "low": 89.38, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.38, "open": 89.38, "day": "31"} +{"volume": 3192, "symbol": "RETA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 89.51, "low": 89.51, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.51, "open": 89.51, "day": "31"} +{"volume": 1761, "symbol": "RETA", "ts": "2018-08-31 09:45:00", "month": "08", "high": 89.49, "low": 89.49, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.49, "open": 89.49, "day": "31"} +{"volume": 645, "symbol": "RETA", "ts": "2018-08-31 09:46:00", "month": "08", "high": 89.25, "low": 89.25, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.25, "open": 89.25, "day": "31"} +{"volume": 693, "symbol": "RETA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 89.075, "low": 89.075, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.075, "open": 89.075, "day": "31"} +{"volume": 212, "symbol": "RETA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 89.115, "low": 89.115, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 89.115, "open": 89.115, "day": "31"} +{"volume": 2323, "symbol": "RETA", "ts": "2018-08-31 09:53:00", "month": "08", "high": 88.5, "low": 88.5, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 88.5, "open": 88.5, "day": "31"} +{"volume": 3956, "symbol": "RETA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 87.6395, "low": 87.12, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.6395, "open": 87.12, "day": "31"} +{"volume": 2154, "symbol": "RETA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 87.2617, "low": 87.2617, "key": "RETA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 87.2617, "open": 87.2617, "day": "31"} +{"volume": 0, "symbol": "UIS", "ts": "2018-08-31 09:31:00", "month": "08", "high": 18.425, "low": 18.4, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.4, "open": 18.425, "day": "31"} +{"volume": 19877, "symbol": "UIS", "ts": "2018-08-31 09:32:00", "month": "08", "high": 18.5, "low": 18.45, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.45, "open": 18.5, "day": "31"} +{"volume": 43692, "symbol": "UIS", "ts": "2018-08-31 09:33:00", "month": "08", "high": 18.5, "low": 18.45, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 3280, "symbol": "UIS", "ts": "2018-08-31 09:34:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 2441, "symbol": "UIS", "ts": "2018-08-31 09:36:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 72414, "symbol": "UIS", "ts": "2018-08-31 09:37:00", "month": "08", "high": 18.55, "low": 18.5488, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.5488, "day": "31"} +{"volume": 15470, "symbol": "UIS", "ts": "2018-08-31 09:38:00", "month": "08", "high": 18.55, "low": 18.55, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.55, "day": "31"} +{"volume": 15395, "symbol": "UIS", "ts": "2018-08-31 09:39:00", "month": "08", "high": 18.55, "low": 18.55, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.55, "day": "31"} +{"volume": 31856, "symbol": "UIS", "ts": "2018-08-31 09:40:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 10283, "symbol": "UIS", "ts": "2018-08-31 09:41:00", "month": "08", "high": 18.4, "low": 18.4, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.4, "open": 18.4, "day": "31"} +{"volume": 6424, "symbol": "UIS", "ts": "2018-08-31 09:42:00", "month": "08", "high": 18.3, "low": 18.3, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.3, "open": 18.3, "day": "31"} +{"volume": 700, "symbol": "UIS", "ts": "2018-08-31 09:43:00", "month": "08", "high": 18.325, "low": 18.325, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.325, "open": 18.325, "day": "31"} +{"volume": 945, "symbol": "UIS", "ts": "2018-08-31 09:45:00", "month": "08", "high": 18.325, "low": 18.325, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.325, "open": 18.325, "day": "31"} +{"volume": 300, "symbol": "UIS", "ts": "2018-08-31 09:46:00", "month": "08", "high": 18.35, "low": 18.35, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.35, "open": 18.35, "day": "31"} +{"volume": 702, "symbol": "UIS", "ts": "2018-08-31 09:47:00", "month": "08", "high": 18.325, "low": 18.325, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.325, "open": 18.325, "day": "31"} +{"volume": 9582, "symbol": "UIS", "ts": "2018-08-31 09:48:00", "month": "08", "high": 18.35, "low": 18.35, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.35, "open": 18.35, "day": "31"} +{"volume": 4662, "symbol": "UIS", "ts": "2018-08-31 09:51:00", "month": "08", "high": 18.429, "low": 18.4, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.429, "open": 18.4, "day": "31"} +{"volume": 11812, "symbol": "UIS", "ts": "2018-08-31 09:52:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 1761, "symbol": "UIS", "ts": "2018-08-31 09:53:00", "month": "08", "high": 18.4999, "low": 18.4999, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.4999, "open": 18.4999, "day": "31"} +{"volume": 6115, "symbol": "UIS", "ts": "2018-08-31 09:54:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 3456, "symbol": "UIS", "ts": "2018-08-31 09:56:00", "month": "08", "high": 18.5, "low": 18.475, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.475, "day": "31"} +{"volume": 2003, "symbol": "UIS", "ts": "2018-08-31 09:57:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 6130, "symbol": "UIS", "ts": "2018-08-31 09:58:00", "month": "08", "high": 18.5, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.5, "open": 18.5, "day": "31"} +{"volume": 41813, "symbol": "UIS", "ts": "2018-08-31 09:59:00", "month": "08", "high": 18.575, "low": 18.5, "key": "UIS_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.5434, "day": "31"} +{"volume": 27833, "symbol": "ETSY", "ts": "2018-08-31 09:31:00", "month": "08", "high": 48.48, "low": 48.25, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.48, "open": 48.25, "day": "31"} +{"volume": 1731, "symbol": "ETSY", "ts": "2018-08-31 09:33:00", "month": "08", "high": 48.48, "low": 48.48, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.48, "open": 48.48, "day": "31"} +{"volume": 2824, "symbol": "ETSY", "ts": "2018-08-31 09:34:00", "month": "08", "high": 48.44, "low": 48.44, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.44, "open": 48.44, "day": "31"} +{"volume": 3020, "symbol": "ETSY", "ts": "2018-08-31 09:35:00", "month": "08", "high": 48.69, "low": 48.69, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.69, "open": 48.69, "day": "31"} +{"volume": 1187, "symbol": "ETSY", "ts": "2018-08-31 09:37:00", "month": "08", "high": 48.5875, "low": 48.5875, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.5875, "open": 48.5875, "day": "31"} +{"volume": 4648, "symbol": "ETSY", "ts": "2018-08-31 09:38:00", "month": "08", "high": 48.79, "low": 48.74, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.79, "open": 48.74, "day": "31"} +{"volume": 1184, "symbol": "ETSY", "ts": "2018-08-31 09:40:00", "month": "08", "high": 48.7334, "low": 48.7334, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.7334, "open": 48.7334, "day": "31"} +{"volume": 1912, "symbol": "ETSY", "ts": "2018-08-31 09:41:00", "month": "08", "high": 48.819, "low": 48.819, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.819, "open": 48.819, "day": "31"} +{"volume": 3472, "symbol": "ETSY", "ts": "2018-08-31 09:43:00", "month": "08", "high": 48.7709, "low": 48.59, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.59, "open": 48.7709, "day": "31"} +{"volume": 4856, "symbol": "ETSY", "ts": "2018-08-31 09:45:00", "month": "08", "high": 48.32, "low": 48.32, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.32, "open": 48.32, "day": "31"} +{"volume": 3239, "symbol": "ETSY", "ts": "2018-08-31 09:46:00", "month": "08", "high": 48.24, "low": 48.24, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.24, "open": 48.24, "day": "31"} +{"volume": 14480, "symbol": "ETSY", "ts": "2018-08-31 09:47:00", "month": "08", "high": 48.37, "low": 48.22, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.37, "open": 48.22, "day": "31"} +{"volume": 4754, "symbol": "ETSY", "ts": "2018-08-31 09:48:00", "month": "08", "high": 48.38, "low": 48.38, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.38, "open": 48.38, "day": "31"} +{"volume": 4604, "symbol": "ETSY", "ts": "2018-08-31 09:49:00", "month": "08", "high": 48.36, "low": 48.36, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.36, "open": 48.36, "day": "31"} +{"volume": 2601, "symbol": "ETSY", "ts": "2018-08-31 09:50:00", "month": "08", "high": 48.39, "low": 48.39, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.39, "open": 48.39, "day": "31"} +{"volume": 1488, "symbol": "ETSY", "ts": "2018-08-31 09:51:00", "month": "08", "high": 48.54, "low": 48.54, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.54, "open": 48.54, "day": "31"} +{"volume": 3652, "symbol": "ETSY", "ts": "2018-08-31 09:52:00", "month": "08", "high": 48.6199, "low": 48.6199, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.6199, "open": 48.6199, "day": "31"} +{"volume": 526, "symbol": "ETSY", "ts": "2018-08-31 09:54:00", "month": "08", "high": 48.6, "low": 48.6, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.6, "open": 48.6, "day": "31"} +{"volume": 1778, "symbol": "ETSY", "ts": "2018-08-31 09:56:00", "month": "08", "high": 48.56, "low": 48.56, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.56, "open": 48.56, "day": "31"} +{"volume": 15250, "symbol": "ETSY", "ts": "2018-08-31 09:57:00", "month": "08", "high": 48.39, "low": 48.39, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.39, "open": 48.39, "day": "31"} +{"volume": 1203, "symbol": "ETSY", "ts": "2018-08-31 09:59:00", "month": "08", "high": 48.21, "low": 48.21, "key": "ETSY_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 48.21, "open": 48.21, "day": "31"} +{"volume": 24648, "symbol": "TWLO", "ts": "2018-08-31 09:31:00", "month": "08", "high": 80.32, "low": 80.32, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.32, "open": 80.32, "day": "31"} +{"volume": 1477, "symbol": "TWLO", "ts": "2018-08-31 09:32:00", "month": "08", "high": 80.355, "low": 80.355, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.355, "open": 80.355, "day": "31"} +{"volume": 1766, "symbol": "TWLO", "ts": "2018-08-31 09:33:00", "month": "08", "high": 80.39, "low": 80.39, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.39, "open": 80.39, "day": "31"} +{"volume": 10513, "symbol": "TWLO", "ts": "2018-08-31 09:35:00", "month": "08", "high": 80.65, "low": 80.5613, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.5613, "open": 80.65, "day": "31"} +{"volume": 2010, "symbol": "TWLO", "ts": "2018-08-31 09:36:00", "month": "08", "high": 80.5767, "low": 80.5767, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.5767, "open": 80.5767, "day": "31"} +{"volume": 1508, "symbol": "TWLO", "ts": "2018-08-31 09:38:00", "month": "08", "high": 80.6197, "low": 80.6197, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.6197, "open": 80.6197, "day": "31"} +{"volume": 11108, "symbol": "TWLO", "ts": "2018-08-31 09:39:00", "month": "08", "high": 81.08, "low": 80.781, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.781, "open": 80.8, "day": "31"} +{"volume": 4542, "symbol": "TWLO", "ts": "2018-08-31 09:40:00", "month": "08", "high": 80.9954, "low": 80.9954, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.9954, "open": 80.9954, "day": "31"} +{"volume": 3821, "symbol": "TWLO", "ts": "2018-08-31 09:41:00", "month": "08", "high": 81.01, "low": 81.01, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 81.01, "open": 81.01, "day": "31"} +{"volume": 5066, "symbol": "TWLO", "ts": "2018-08-31 09:42:00", "month": "08", "high": 80.906, "low": 80.7413, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.7413, "open": 80.906, "day": "31"} +{"volume": 2230, "symbol": "TWLO", "ts": "2018-08-31 09:43:00", "month": "08", "high": 80.48, "low": 80.48, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.48, "open": 80.48, "day": "31"} +{"volume": 3693, "symbol": "TWLO", "ts": "2018-08-31 09:44:00", "month": "08", "high": 80.23, "low": 80.23, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.23, "open": 80.23, "day": "31"} +{"volume": 1691, "symbol": "TWLO", "ts": "2018-08-31 09:45:00", "month": "08", "high": 80.15, "low": 80.15, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.15, "open": 80.15, "day": "31"} +{"volume": 3056, "symbol": "TWLO", "ts": "2018-08-31 09:46:00", "month": "08", "high": 80.53, "low": 80.53, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.53, "open": 80.53, "day": "31"} +{"volume": 3532, "symbol": "TWLO", "ts": "2018-08-31 09:47:00", "month": "08", "high": 80.5, "low": 80.5, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.5, "open": 80.5, "day": "31"} +{"volume": 2035, "symbol": "TWLO", "ts": "2018-08-31 09:48:00", "month": "08", "high": 80.41, "low": 80.41, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.41, "open": 80.41, "day": "31"} +{"volume": 1708, "symbol": "TWLO", "ts": "2018-08-31 09:49:00", "month": "08", "high": 80.44, "low": 80.44, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.44, "open": 80.44, "day": "31"} +{"volume": 733, "symbol": "TWLO", "ts": "2018-08-31 09:51:00", "month": "08", "high": 80.435, "low": 80.435, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.435, "open": 80.435, "day": "31"} +{"volume": 1004, "symbol": "TWLO", "ts": "2018-08-31 09:54:00", "month": "08", "high": 80.515, "low": 80.515, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.515, "open": 80.515, "day": "31"} +{"volume": 924, "symbol": "TWLO", "ts": "2018-08-31 09:55:00", "month": "08", "high": 80.5, "low": 80.5, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.5, "open": 80.5, "day": "31"} +{"volume": 2109, "symbol": "TWLO", "ts": "2018-08-31 09:56:00", "month": "08", "high": 80.6, "low": 80.6, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.6, "open": 80.6, "day": "31"} +{"volume": 37556, "symbol": "TWLO", "ts": "2018-08-31 09:57:00", "month": "08", "high": 80.46, "low": 80.24, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.24, "open": 80.46, "day": "31"} +{"volume": 3316, "symbol": "TWLO", "ts": "2018-08-31 09:58:00", "month": "08", "high": 80.13, "low": 80.13, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.13, "open": 80.13, "day": "31"} +{"volume": 3062, "symbol": "TWLO", "ts": "2018-08-31 09:59:00", "month": "08", "high": 80.2213, "low": 80.2213, "key": "TWLO_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 80.2213, "open": 80.2213, "day": "31"} +{"volume": 2899, "symbol": "AMED", "ts": "2018-08-31 09:31:00", "month": "08", "high": 120.7556, "low": 120.7556, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 120.7556, "open": 120.7556, "day": "31"} +{"volume": 1467, "symbol": "AMED", "ts": "2018-08-31 09:32:00", "month": "08", "high": 120.89, "low": 120.89, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 120.89, "open": 120.89, "day": "31"} +{"volume": 1745, "symbol": "AMED", "ts": "2018-08-31 09:33:00", "month": "08", "high": 121.93, "low": 121.93, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.93, "open": 121.93, "day": "31"} +{"volume": 1801, "symbol": "AMED", "ts": "2018-08-31 09:36:00", "month": "08", "high": 122.0, "low": 122.0, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.0, "open": 122.0, "day": "31"} +{"volume": 2005, "symbol": "AMED", "ts": "2018-08-31 09:37:00", "month": "08", "high": 122.05, "low": 122.05, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.05, "open": 122.05, "day": "31"} +{"volume": 2138, "symbol": "AMED", "ts": "2018-08-31 09:38:00", "month": "08", "high": 122.27, "low": 122.27, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.27, "open": 122.27, "day": "31"} +{"volume": 1671, "symbol": "AMED", "ts": "2018-08-31 09:40:00", "month": "08", "high": 122.32, "low": 122.32, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.32, "open": 122.32, "day": "31"} +{"volume": 2012, "symbol": "AMED", "ts": "2018-08-31 09:41:00", "month": "08", "high": 122.19, "low": 122.19, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.19, "open": 122.19, "day": "31"} +{"volume": 169, "symbol": "AMED", "ts": "2018-08-31 09:42:00", "month": "08", "high": 122.27, "low": 122.27, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.27, "open": 122.27, "day": "31"} +{"volume": 2010, "symbol": "AMED", "ts": "2018-08-31 09:46:00", "month": "08", "high": 122.0675, "low": 122.0675, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.0675, "open": 122.0675, "day": "31"} +{"volume": 833, "symbol": "AMED", "ts": "2018-08-31 09:47:00", "month": "08", "high": 122.0, "low": 122.0, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 122.0, "open": 122.0, "day": "31"} +{"volume": 521, "symbol": "AMED", "ts": "2018-08-31 09:49:00", "month": "08", "high": 121.881, "low": 121.881, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.881, "open": 121.881, "day": "31"} +{"volume": 1384, "symbol": "AMED", "ts": "2018-08-31 09:51:00", "month": "08", "high": 121.79, "low": 121.79, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.79, "open": 121.79, "day": "31"} +{"volume": 2371, "symbol": "AMED", "ts": "2018-08-31 09:52:00", "month": "08", "high": 121.81, "low": 121.81, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.81, "open": 121.81, "day": "31"} +{"volume": 633, "symbol": "AMED", "ts": "2018-08-31 09:54:00", "month": "08", "high": 121.7923, "low": 121.7923, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.7923, "open": 121.7923, "day": "31"} +{"volume": 436, "symbol": "AMED", "ts": "2018-08-31 09:55:00", "month": "08", "high": 121.9, "low": 121.9, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.9, "open": 121.9, "day": "31"} +{"volume": 1314, "symbol": "AMED", "ts": "2018-08-31 09:57:00", "month": "08", "high": 121.59, "low": 121.59, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.59, "open": 121.59, "day": "31"} +{"volume": 830, "symbol": "AMED", "ts": "2018-08-31 09:58:00", "month": "08", "high": 121.785, "low": 121.785, "key": "AMED_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 121.785, "open": 121.785, "day": "31"} +{"volume": 0, "symbol": "USAT", "ts": "2018-08-31 09:31:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 3632, "symbol": "USAT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 1286, "symbol": "USAT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 16.0641, "low": 16.0641, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.0641, "open": 16.0641, "day": "31"} +{"volume": 3135, "symbol": "USAT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 16.1, "low": 16.1, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.1, "open": 16.1, "day": "31"} +{"volume": 2823, "symbol": "USAT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 16.15, "low": 16.1, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.15, "open": 16.1, "day": "31"} +{"volume": 2643, "symbol": "USAT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 600, "symbol": "USAT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 100, "symbol": "USAT", "ts": "2018-08-31 09:50:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 100, "symbol": "USAT", "ts": "2018-08-31 09:52:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 200, "symbol": "USAT", "ts": "2018-08-31 09:53:00", "month": "08", "high": 16.01, "low": 16.01, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.01, "open": 16.01, "day": "31"} +{"volume": 869, "symbol": "USAT", "ts": "2018-08-31 09:55:00", "month": "08", "high": 16.005, "low": 16.005, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.005, "open": 16.005, "day": "31"} +{"volume": 3417, "symbol": "USAT", "ts": "2018-08-31 09:59:00", "month": "08", "high": 16.0, "low": 16.0, "key": "USAT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 16.0, "open": 16.0, "day": "31"} +{"volume": 0, "symbol": "RGNX", "ts": "2018-08-31 09:31:00", "month": "08", "high": 73.85, "low": 73.85, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.85, "open": 73.85, "day": "31"} +{"volume": 3083, "symbol": "RGNX", "ts": "2018-08-31 09:33:00", "month": "08", "high": 73.825, "low": 73.825, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.825, "open": 73.825, "day": "31"} +{"volume": 2818, "symbol": "RGNX", "ts": "2018-08-31 09:34:00", "month": "08", "high": 73.25, "low": 73.25, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.25, "open": 73.25, "day": "31"} +{"volume": 3116, "symbol": "RGNX", "ts": "2018-08-31 09:35:00", "month": "08", "high": 73.45, "low": 73.45, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.45, "open": 73.45, "day": "31"} +{"volume": 3129, "symbol": "RGNX", "ts": "2018-08-31 09:37:00", "month": "08", "high": 73.35, "low": 73.35, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.35, "open": 73.35, "day": "31"} +{"volume": 1687, "symbol": "RGNX", "ts": "2018-08-31 09:38:00", "month": "08", "high": 73.15, "low": 73.15, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 73.15, "open": 73.15, "day": "31"} +{"volume": 5071, "symbol": "RGNX", "ts": "2018-08-31 09:39:00", "month": "08", "high": 72.65, "low": 72.65, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 72.65, "open": 72.65, "day": "31"} +{"volume": 2527, "symbol": "RGNX", "ts": "2018-08-31 09:40:00", "month": "08", "high": 72.2186, "low": 72.2186, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 72.2186, "open": 72.2186, "day": "31"} +{"volume": 3594, "symbol": "RGNX", "ts": "2018-08-31 09:41:00", "month": "08", "high": 71.95, "low": 71.95, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.95, "open": 71.95, "day": "31"} +{"volume": 7132, "symbol": "RGNX", "ts": "2018-08-31 09:42:00", "month": "08", "high": 71.65, "low": 71.65, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.65, "open": 71.65, "day": "31"} +{"volume": 8621, "symbol": "RGNX", "ts": "2018-08-31 09:43:00", "month": "08", "high": 71.425, "low": 71.2384, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.2384, "open": 71.425, "day": "31"} +{"volume": 3032, "symbol": "RGNX", "ts": "2018-08-31 09:45:00", "month": "08", "high": 70.9, "low": 70.9, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.9, "open": 70.9, "day": "31"} +{"volume": 2247, "symbol": "RGNX", "ts": "2018-08-31 09:47:00", "month": "08", "high": 70.95, "low": 70.95, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.95, "open": 70.95, "day": "31"} +{"volume": 2072, "symbol": "RGNX", "ts": "2018-08-31 09:48:00", "month": "08", "high": 70.85, "low": 70.85, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.85, "open": 70.85, "day": "31"} +{"volume": 1100, "symbol": "RGNX", "ts": "2018-08-31 09:49:00", "month": "08", "high": 70.995, "low": 70.995, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.995, "open": 70.995, "day": "31"} +{"volume": 2421, "symbol": "RGNX", "ts": "2018-08-31 09:50:00", "month": "08", "high": 71.3, "low": 71.3, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.3, "open": 71.3, "day": "31"} +{"volume": 3820, "symbol": "RGNX", "ts": "2018-08-31 09:52:00", "month": "08", "high": 71.5, "low": 71.35, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.5, "open": 71.35, "day": "31"} +{"volume": 3711, "symbol": "RGNX", "ts": "2018-08-31 09:54:00", "month": "08", "high": 71.725, "low": 71.675, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.725, "open": 71.675, "day": "31"} +{"volume": 5526, "symbol": "RGNX", "ts": "2018-08-31 09:56:00", "month": "08", "high": 71.3, "low": 71.3, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.3, "open": 71.3, "day": "31"} +{"volume": 1648, "symbol": "RGNX", "ts": "2018-08-31 09:57:00", "month": "08", "high": 71.0, "low": 71.0, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 71.0, "open": 71.0, "day": "31"} +{"volume": 2676, "symbol": "RGNX", "ts": "2018-08-31 09:58:00", "month": "08", "high": 70.55, "low": 70.55, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.55, "open": 70.55, "day": "31"} +{"volume": 3843, "symbol": "RGNX", "ts": "2018-08-31 09:59:00", "month": "08", "high": 70.4, "low": 70.4, "key": "RGNX_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 70.4, "open": 70.4, "day": "31"} +{"volume": 3051, "symbol": "HAE", "ts": "2018-08-31 09:33:00", "month": "08", "high": 109.0, "low": 109.0, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.0, "open": 109.0, "day": "31"} +{"volume": 759, "symbol": "HAE", "ts": "2018-08-31 09:34:00", "month": "08", "high": 109.2998, "low": 109.2998, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.2998, "open": 109.2998, "day": "31"} +{"volume": 1205, "symbol": "HAE", "ts": "2018-08-31 09:36:00", "month": "08", "high": 109.03, "low": 109.03, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.03, "open": 109.03, "day": "31"} +{"volume": 254, "symbol": "HAE", "ts": "2018-08-31 09:37:00", "month": "08", "high": 109.0, "low": 109.0, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.0, "open": 109.0, "day": "31"} +{"volume": 777, "symbol": "HAE", "ts": "2018-08-31 09:40:00", "month": "08", "high": 109.4, "low": 109.4, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.4, "open": 109.4, "day": "31"} +{"volume": 884, "symbol": "HAE", "ts": "2018-08-31 09:41:00", "month": "08", "high": 109.7, "low": 109.7, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.7, "open": 109.7, "day": "31"} +{"volume": 1329, "symbol": "HAE", "ts": "2018-08-31 09:47:00", "month": "08", "high": 110.01, "low": 110.01, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 110.01, "open": 110.01, "day": "31"} +{"volume": 1776, "symbol": "HAE", "ts": "2018-08-31 09:50:00", "month": "08", "high": 109.87, "low": 109.87, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.87, "open": 109.87, "day": "31"} +{"volume": 445, "symbol": "HAE", "ts": "2018-08-31 09:54:00", "month": "08", "high": 109.76, "low": 109.76, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.76, "open": 109.76, "day": "31"} +{"volume": 251, "symbol": "HAE", "ts": "2018-08-31 09:57:00", "month": "08", "high": 109.82, "low": 109.82, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.82, "open": 109.82, "day": "31"} +{"volume": 1058, "symbol": "HAE", "ts": "2018-08-31 09:59:00", "month": "08", "high": 109.475, "low": 109.475, "key": "HAE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 109.475, "open": 109.475, "day": "31"} +{"volume": 2218, "symbol": "NVTA", "ts": "2018-08-31 09:33:00", "month": "08", "high": 14.09, "low": 14.09, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.09, "open": 14.09, "day": "31"} +{"volume": 2097, "symbol": "NVTA", "ts": "2018-08-31 09:34:00", "month": "08", "high": 14.1325, "low": 14.1325, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.1325, "open": 14.1325, "day": "31"} +{"volume": 4453, "symbol": "NVTA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 14.24, "low": 14.24, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.24, "open": 14.24, "day": "31"} +{"volume": 2266, "symbol": "NVTA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 14.27, "low": 14.27, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.27, "open": 14.27, "day": "31"} +{"volume": 3090, "symbol": "NVTA", "ts": "2018-08-31 09:38:00", "month": "08", "high": 14.3, "low": 14.3, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.3, "open": 14.3, "day": "31"} +{"volume": 3576, "symbol": "NVTA", "ts": "2018-08-31 09:39:00", "month": "08", "high": 14.32, "low": 14.32, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.32, "open": 14.32, "day": "31"} +{"volume": 3006, "symbol": "NVTA", "ts": "2018-08-31 09:40:00", "month": "08", "high": 14.32, "low": 14.32, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.32, "open": 14.32, "day": "31"} +{"volume": 7579, "symbol": "NVTA", "ts": "2018-08-31 09:41:00", "month": "08", "high": 14.38, "low": 14.38, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.38, "open": 14.38, "day": "31"} +{"volume": 2210, "symbol": "NVTA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 14.401, "low": 14.401, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.401, "open": 14.401, "day": "31"} +{"volume": 2955, "symbol": "NVTA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 14.43, "low": 14.43, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.43, "open": 14.43, "day": "31"} +{"volume": 1801, "symbol": "NVTA", "ts": "2018-08-31 09:44:00", "month": "08", "high": 14.48, "low": 14.48, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.48, "open": 14.48, "day": "31"} +{"volume": 11211, "symbol": "NVTA", "ts": "2018-08-31 09:46:00", "month": "08", "high": 14.55, "low": 14.47, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.47, "day": "31"} +{"volume": 4243, "symbol": "NVTA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 14.49, "low": 14.49, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.49, "open": 14.49, "day": "31"} +{"volume": 7618, "symbol": "NVTA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 14.58, "low": 14.58, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.58, "open": 14.58, "day": "31"} +{"volume": 1493, "symbol": "NVTA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 14.59, "low": 14.59, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.59, "open": 14.59, "day": "31"} +{"volume": 15888, "symbol": "NVTA", "ts": "2018-08-31 09:51:00", "month": "08", "high": 14.6978, "low": 14.63, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.6978, "open": 14.63, "day": "31"} +{"volume": 4300, "symbol": "NVTA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 14.63, "low": 14.63, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.63, "open": 14.63, "day": "31"} +{"volume": 2126, "symbol": "NVTA", "ts": "2018-08-31 09:53:00", "month": "08", "high": 14.69, "low": 14.69, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.69, "open": 14.69, "day": "31"} +{"volume": 3696, "symbol": "NVTA", "ts": "2018-08-31 09:54:00", "month": "08", "high": 14.71, "low": 14.71, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.71, "open": 14.71, "day": "31"} +{"volume": 3883, "symbol": "NVTA", "ts": "2018-08-31 09:55:00", "month": "08", "high": 14.74, "low": 14.74, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.74, "open": 14.74, "day": "31"} +{"volume": 4996, "symbol": "NVTA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 14.73, "low": 14.73, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.73, "open": 14.73, "day": "31"} +{"volume": 3133, "symbol": "NVTA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 14.73, "low": 14.73, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.73, "open": 14.73, "day": "31"} +{"volume": 12531, "symbol": "NVTA", "ts": "2018-08-31 09:58:00", "month": "08", "high": 14.76, "low": 14.73, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.76, "open": 14.73, "day": "31"} +{"volume": 6760, "symbol": "NVTA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 14.78, "low": 14.77, "key": "NVTA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 14.78, "open": 14.77, "day": "31"} +{"volume": 0, "symbol": "TPB", "ts": "2018-08-31 09:31:00", "month": "08", "high": 33.87, "low": 33.87, "key": "TPB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 33.87, "open": 33.87, "day": "31"} +{"volume": 1415, "symbol": "TPB", "ts": "2018-08-31 09:33:00", "month": "08", "high": 33.85, "low": 33.85, "key": "TPB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 33.85, "open": 33.85, "day": "31"} +{"volume": 221, "symbol": "TPB", "ts": "2018-08-31 09:38:00", "month": "08", "high": 33.85, "low": 33.85, "key": "TPB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 33.85, "open": 33.85, "day": "31"} +{"volume": 210, "symbol": "TPB", "ts": "2018-08-31 09:39:00", "month": "08", "high": 33.85, "low": 33.85, "key": "TPB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 33.85, "open": 33.85, "day": "31"} +{"volume": 548, "symbol": "TPB", "ts": "2018-08-31 09:43:00", "month": "08", "high": 34.0, "low": 34.0, "key": "TPB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 34.0, "open": 34.0, "day": "31"} +{"volume": 8788, "symbol": "TPB", "ts": "2018-08-31 09:58:00", "month": "08", "high": 33.9275, "low": 33.9275, "key": "TPB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 33.9275, "open": 33.9275, "day": "31"} +{"volume": 1437, "symbol": "NTRA", "ts": "2018-08-31 09:32:00", "month": "08", "high": 27.46, "low": 27.46, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.46, "open": 27.46, "day": "31"} +{"volume": 1520, "symbol": "NTRA", "ts": "2018-08-31 09:36:00", "month": "08", "high": 27.4734, "low": 27.4734, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.4734, "open": 27.4734, "day": "31"} +{"volume": 400, "symbol": "NTRA", "ts": "2018-08-31 09:37:00", "month": "08", "high": 27.6, "low": 27.6, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.6, "open": 27.6, "day": "31"} +{"volume": 1032, "symbol": "NTRA", "ts": "2018-08-31 09:42:00", "month": "08", "high": 27.72, "low": 27.72, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.72, "open": 27.72, "day": "31"} +{"volume": 1920, "symbol": "NTRA", "ts": "2018-08-31 09:43:00", "month": "08", "high": 27.52, "low": 27.52, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.52, "open": 27.52, "day": "31"} +{"volume": 5400, "symbol": "NTRA", "ts": "2018-08-31 09:45:00", "month": "08", "high": 27.47, "low": 27.47, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.47, "open": 27.47, "day": "31"} +{"volume": 1670, "symbol": "NTRA", "ts": "2018-08-31 09:47:00", "month": "08", "high": 27.45, "low": 27.45, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.45, "open": 27.45, "day": "31"} +{"volume": 8156, "symbol": "NTRA", "ts": "2018-08-31 09:48:00", "month": "08", "high": 27.51, "low": 27.46, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.46, "open": 27.51, "day": "31"} +{"volume": 7511, "symbol": "NTRA", "ts": "2018-08-31 09:49:00", "month": "08", "high": 27.58, "low": 27.51, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.51, "open": 27.58, "day": "31"} +{"volume": 2700, "symbol": "NTRA", "ts": "2018-08-31 09:50:00", "month": "08", "high": 27.44, "low": 27.44, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.44, "open": 27.44, "day": "31"} +{"volume": 7885, "symbol": "NTRA", "ts": "2018-08-31 09:51:00", "month": "08", "high": 27.48, "low": 27.43, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.43, "open": 27.48, "day": "31"} +{"volume": 1050, "symbol": "NTRA", "ts": "2018-08-31 09:52:00", "month": "08", "high": 27.41, "low": 27.41, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.41, "open": 27.41, "day": "31"} +{"volume": 3329, "symbol": "NTRA", "ts": "2018-08-31 09:54:00", "month": "08", "high": 27.36, "low": 27.36, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.36, "open": 27.36, "day": "31"} +{"volume": 2897, "symbol": "NTRA", "ts": "2018-08-31 09:56:00", "month": "08", "high": 27.48, "low": 27.48, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.48, "open": 27.48, "day": "31"} +{"volume": 940, "symbol": "NTRA", "ts": "2018-08-31 09:57:00", "month": "08", "high": 27.42, "low": 27.42, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.42, "open": 27.42, "day": "31"} +{"volume": 1326, "symbol": "NTRA", "ts": "2018-08-31 09:59:00", "month": "08", "high": 27.45, "low": 27.45, "key": "NTRA_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 27.45, "open": 27.45, "day": "31"} +{"volume": 40238, "symbol": "WTI", "ts": "2018-08-31 09:31:00", "month": "08", "high": 6.825, "low": 6.82, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.825, "open": 6.82, "day": "31"} +{"volume": 9718, "symbol": "WTI", "ts": "2018-08-31 09:32:00", "month": "08", "high": 6.85, "low": 6.85, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.85, "open": 6.85, "day": "31"} +{"volume": 5315, "symbol": "WTI", "ts": "2018-08-31 09:33:00", "month": "08", "high": 6.84, "low": 6.84, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.84, "open": 6.84, "day": "31"} +{"volume": 17353, "symbol": "WTI", "ts": "2018-08-31 09:34:00", "month": "08", "high": 6.8503, "low": 6.81, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.8503, "open": 6.81, "day": "31"} +{"volume": 15342, "symbol": "WTI", "ts": "2018-08-31 09:35:00", "month": "08", "high": 6.86, "low": 6.85, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.85, "open": 6.86, "day": "31"} +{"volume": 9001, "symbol": "WTI", "ts": "2018-08-31 09:36:00", "month": "08", "high": 6.8533, "low": 6.8533, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.8533, "open": 6.8533, "day": "31"} +{"volume": 4220, "symbol": "WTI", "ts": "2018-08-31 09:37:00", "month": "08", "high": 6.8301, "low": 6.8301, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.8301, "open": 6.8301, "day": "31"} +{"volume": 52892, "symbol": "WTI", "ts": "2018-08-31 09:38:00", "month": "08", "high": 6.825, "low": 6.78, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.78, "open": 6.825, "day": "31"} +{"volume": 3381, "symbol": "WTI", "ts": "2018-08-31 09:39:00", "month": "08", "high": 6.7901, "low": 6.7901, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.7901, "open": 6.7901, "day": "31"} +{"volume": 26282, "symbol": "WTI", "ts": "2018-08-31 09:40:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 6678, "symbol": "WTI", "ts": "2018-08-31 09:41:00", "month": "08", "high": 6.72, "low": 6.72, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.72, "open": 6.72, "day": "31"} +{"volume": 15809, "symbol": "WTI", "ts": "2018-08-31 09:42:00", "month": "08", "high": 6.73, "low": 6.73, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.73, "open": 6.73, "day": "31"} +{"volume": 2336, "symbol": "WTI", "ts": "2018-08-31 09:43:00", "month": "08", "high": 6.75, "low": 6.75, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.75, "open": 6.75, "day": "31"} +{"volume": 14074, "symbol": "WTI", "ts": "2018-08-31 09:44:00", "month": "08", "high": 6.77, "low": 6.75, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.75, "open": 6.77, "day": "31"} +{"volume": 11145, "symbol": "WTI", "ts": "2018-08-31 09:45:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 9433, "symbol": "WTI", "ts": "2018-08-31 09:46:00", "month": "08", "high": 6.75, "low": 6.75, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.75, "open": 6.75, "day": "31"} +{"volume": 16830, "symbol": "WTI", "ts": "2018-08-31 09:47:00", "month": "08", "high": 6.7454, "low": 6.7454, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.7454, "open": 6.7454, "day": "31"} +{"volume": 18420, "symbol": "WTI", "ts": "2018-08-31 09:48:00", "month": "08", "high": 6.77, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.77, "day": "31"} +{"volume": 7967, "symbol": "WTI", "ts": "2018-08-31 09:50:00", "month": "08", "high": 6.7601, "low": 6.7601, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.7601, "open": 6.7601, "day": "31"} +{"volume": 5160, "symbol": "WTI", "ts": "2018-08-31 09:51:00", "month": "08", "high": 6.75, "low": 6.75, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.75, "open": 6.75, "day": "31"} +{"volume": 6500, "symbol": "WTI", "ts": "2018-08-31 09:52:00", "month": "08", "high": 6.75, "low": 6.75, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.75, "open": 6.75, "day": "31"} +{"volume": 3005, "symbol": "WTI", "ts": "2018-08-31 09:53:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 1898, "symbol": "WTI", "ts": "2018-08-31 09:54:00", "month": "08", "high": 6.77, "low": 6.77, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.77, "open": 6.77, "day": "31"} +{"volume": 4584, "symbol": "WTI", "ts": "2018-08-31 09:55:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 7915, "symbol": "WTI", "ts": "2018-08-31 09:56:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 3864, "symbol": "WTI", "ts": "2018-08-31 09:57:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 11586, "symbol": "WTI", "ts": "2018-08-31 09:58:00", "month": "08", "high": 6.76, "low": 6.76, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.76, "open": 6.76, "day": "31"} +{"volume": 7476, "symbol": "WTI", "ts": "2018-08-31 09:59:00", "month": "08", "high": 6.765, "low": 6.765, "key": "WTI_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 6.765, "open": 6.765, "day": "31"} +{"volume": 1409, "symbol": "RCM", "ts": "2018-08-31 09:33:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 2703, "symbol": "RCM", "ts": "2018-08-31 09:34:00", "month": "08", "high": 10.0, "low": 10.0, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 10.0, "open": 10.0, "day": "31"} +{"volume": 6364, "symbol": "RCM", "ts": "2018-08-31 09:35:00", "month": "08", "high": 9.95, "low": 9.95, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.95, "open": 9.95, "day": "31"} +{"volume": 2270, "symbol": "RCM", "ts": "2018-08-31 09:37:00", "month": "08", "high": 9.97, "low": 9.97, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.97, "open": 9.97, "day": "31"} +{"volume": 6400, "symbol": "RCM", "ts": "2018-08-31 09:39:00", "month": "08", "high": 10.0, "low": 10.0, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 10.0, "open": 10.0, "day": "31"} +{"volume": 2710, "symbol": "RCM", "ts": "2018-08-31 09:40:00", "month": "08", "high": 10.025, "low": 10.025, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 10.025, "open": 10.025, "day": "31"} +{"volume": 1460, "symbol": "RCM", "ts": "2018-08-31 09:41:00", "month": "08", "high": 10.03, "low": 10.03, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 10.03, "open": 10.03, "day": "31"} +{"volume": 4231, "symbol": "RCM", "ts": "2018-08-31 09:43:00", "month": "08", "high": 10.05, "low": 10.015, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 10.015, "open": 10.05, "day": "31"} +{"volume": 3002, "symbol": "RCM", "ts": "2018-08-31 09:44:00", "month": "08", "high": 9.94, "low": 9.94, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 9.94, "day": "31"} +{"volume": 3102, "symbol": "RCM", "ts": "2018-08-31 09:48:00", "month": "08", "high": 9.96, "low": 9.96, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.96, "open": 9.96, "day": "31"} +{"volume": 1401, "symbol": "RCM", "ts": "2018-08-31 09:51:00", "month": "08", "high": 9.9718, "low": 9.9718, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.9718, "open": 9.9718, "day": "31"} +{"volume": 1789, "symbol": "RCM", "ts": "2018-08-31 09:52:00", "month": "08", "high": 9.975, "low": 9.975, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.975, "open": 9.975, "day": "31"} +{"volume": 400, "symbol": "RCM", "ts": "2018-08-31 09:56:00", "month": "08", "high": 9.97, "low": 9.97, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.97, "open": 9.97, "day": "31"} +{"volume": 1269, "symbol": "RCM", "ts": "2018-08-31 09:57:00", "month": "08", "high": 9.95, "low": 9.95, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.95, "open": 9.95, "day": "31"} +{"volume": 2225, "symbol": "RCM", "ts": "2018-08-31 09:58:00", "month": "08", "high": 9.915, "low": 9.915, "key": "RCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 9.915, "open": 9.915, "day": "31"} +{"volume": 319668, "symbol": "LULU", "ts": "2018-08-31 09:31:00", "month": "08", "high": 156.13, "low": 153.8, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.13, "open": 153.8, "day": "31"} +{"volume": 225842, "symbol": "LULU", "ts": "2018-08-31 09:32:00", "month": "08", "high": 156.0, "low": 155.5, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.63, "open": 156.0, "day": "31"} +{"volume": 158431, "symbol": "LULU", "ts": "2018-08-31 09:33:00", "month": "08", "high": 155.794, "low": 153.97, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 154.08, "open": 155.794, "day": "31"} +{"volume": 132066, "symbol": "LULU", "ts": "2018-08-31 09:34:00", "month": "08", "high": 154.84, "low": 153.37, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 154.72, "open": 153.9796, "day": "31"} +{"volume": 143364, "symbol": "LULU", "ts": "2018-08-31 09:35:00", "month": "08", "high": 155.73, "low": 154.68, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.4, "open": 154.795, "day": "31"} +{"volume": 141086, "symbol": "LULU", "ts": "2018-08-31 09:36:00", "month": "08", "high": 155.58, "low": 155.11, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.2639, "open": 155.46, "day": "31"} +{"volume": 92506, "symbol": "LULU", "ts": "2018-08-31 09:37:00", "month": "08", "high": 155.5, "low": 155.2994, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.5, "open": 155.32, "day": "31"} +{"volume": 86170, "symbol": "LULU", "ts": "2018-08-31 09:38:00", "month": "08", "high": 155.5, "low": 155.2022, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.3712, "open": 155.44, "day": "31"} +{"volume": 147526, "symbol": "LULU", "ts": "2018-08-31 09:39:00", "month": "08", "high": 155.94, "low": 155.38, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.38, "open": 155.425, "day": "31"} +{"volume": 106326, "symbol": "LULU", "ts": "2018-08-31 09:40:00", "month": "08", "high": 155.645, "low": 154.35, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 154.72, "open": 155.39, "day": "31"} +{"volume": 46619, "symbol": "LULU", "ts": "2018-08-31 09:41:00", "month": "08", "high": 155.36, "low": 154.67, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 154.985, "open": 154.67, "day": "31"} +{"volume": 50226, "symbol": "LULU", "ts": "2018-08-31 09:42:00", "month": "08", "high": 155.29, "low": 154.649, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.29, "open": 154.9225, "day": "31"} +{"volume": 279641, "symbol": "LULU", "ts": "2018-08-31 09:43:00", "month": "08", "high": 156.0, "low": 155.1, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 155.98, "open": 155.1, "day": "31"} +{"volume": 225337, "symbol": "LULU", "ts": "2018-08-31 09:44:00", "month": "08", "high": 156.89, "low": 155.9549, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.2805, "open": 155.9549, "day": "31"} +{"volume": 85941, "symbol": "LULU", "ts": "2018-08-31 09:45:00", "month": "08", "high": 156.77, "low": 156.27, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.5125, "open": 156.27, "day": "31"} +{"volume": 90679, "symbol": "LULU", "ts": "2018-08-31 09:46:00", "month": "08", "high": 156.66, "low": 156.431, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.5694, "open": 156.504, "day": "31"} +{"volume": 93578, "symbol": "LULU", "ts": "2018-08-31 09:47:00", "month": "08", "high": 156.66, "low": 156.11, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.26, "open": 156.5712, "day": "31"} +{"volume": 65052, "symbol": "LULU", "ts": "2018-08-31 09:48:00", "month": "08", "high": 156.621, "low": 156.2, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.32, "open": 156.2986, "day": "31"} +{"volume": 139525, "symbol": "LULU", "ts": "2018-08-31 09:49:00", "month": "08", "high": 157.2299, "low": 156.27, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 156.95, "open": 156.27, "day": "31"} +{"volume": 111996, "symbol": "LULU", "ts": "2018-08-31 09:50:00", "month": "08", "high": 157.5653, "low": 157.02, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 157.5653, "open": 157.02, "day": "31"} +{"volume": 108752, "symbol": "LULU", "ts": "2018-08-31 09:51:00", "month": "08", "high": 157.81, "low": 157.465, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 157.81, "open": 157.6, "day": "31"} +{"volume": 150325, "symbol": "LULU", "ts": "2018-08-31 09:52:00", "month": "08", "high": 158.45, "low": 157.85, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 158.42, "open": 157.85, "day": "31"} +{"volume": 109302, "symbol": "LULU", "ts": "2018-08-31 09:53:00", "month": "08", "high": 158.55, "low": 158.1, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 158.55, "open": 158.42, "day": "31"} +{"volume": 92013, "symbol": "LULU", "ts": "2018-08-31 09:54:00", "month": "08", "high": 158.7, "low": 158.405, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 158.62, "open": 158.58, "day": "31"} +{"volume": 107586, "symbol": "LULU", "ts": "2018-08-31 09:55:00", "month": "08", "high": 158.7, "low": 158.4505, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 158.5607, "open": 158.67, "day": "31"} +{"volume": 137041, "symbol": "LULU", "ts": "2018-08-31 09:56:00", "month": "08", "high": 158.47, "low": 157.9588, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 157.9588, "open": 158.45, "day": "31"} +{"volume": 68524, "symbol": "LULU", "ts": "2018-08-31 09:57:00", "month": "08", "high": 158.15, "low": 157.9, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 158.14, "open": 157.98, "day": "31"} +{"volume": 58725, "symbol": "LULU", "ts": "2018-08-31 09:58:00", "month": "08", "high": 158.53, "low": 158.085, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 158.295, "open": 158.1, "day": "31"} +{"volume": 91515, "symbol": "LULU", "ts": "2018-08-31 09:59:00", "month": "08", "high": 157.935, "low": 157.28, "key": "LULU_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 157.66, "open": 157.935, "day": "31"} +{"volume": 136323, "symbol": "YEXT", "ts": "2018-08-31 09:31:00", "month": "08", "high": 24.85, "low": 23.8, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.175, "open": 24.85, "day": "31"} +{"volume": 15134, "symbol": "YEXT", "ts": "2018-08-31 09:32:00", "month": "08", "high": 24.5001, "low": 24.27, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.5001, "open": 24.27, "day": "31"} +{"volume": 18694, "symbol": "YEXT", "ts": "2018-08-31 09:33:00", "month": "08", "high": 25.12, "low": 24.88, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.94, "open": 24.88, "day": "31"} +{"volume": 23377, "symbol": "YEXT", "ts": "2018-08-31 09:34:00", "month": "08", "high": 25.3814, "low": 25.158, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.28, "open": 25.158, "day": "31"} +{"volume": 20324, "symbol": "YEXT", "ts": "2018-08-31 09:35:00", "month": "08", "high": 25.34, "low": 25.31, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.32, "open": 25.34, "day": "31"} +{"volume": 15386, "symbol": "YEXT", "ts": "2018-08-31 09:36:00", "month": "08", "high": 25.284, "low": 25.06, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.2265, "open": 25.284, "day": "31"} +{"volume": 8298, "symbol": "YEXT", "ts": "2018-08-31 09:37:00", "month": "08", "high": 25.33, "low": 25.33, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.33, "open": 25.33, "day": "31"} +{"volume": 17173, "symbol": "YEXT", "ts": "2018-08-31 09:38:00", "month": "08", "high": 25.35, "low": 25.27, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.27, "open": 25.35, "day": "31"} +{"volume": 10490, "symbol": "YEXT", "ts": "2018-08-31 09:39:00", "month": "08", "high": 25.32, "low": 25.15, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.15, "open": 25.32, "day": "31"} +{"volume": 18234, "symbol": "YEXT", "ts": "2018-08-31 09:40:00", "month": "08", "high": 25.3, "low": 25.3, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.3, "day": "31"} +{"volume": 15285, "symbol": "YEXT", "ts": "2018-08-31 09:41:00", "month": "08", "high": 25.33, "low": 25.3, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.33, "open": 25.3, "day": "31"} +{"volume": 15990, "symbol": "YEXT", "ts": "2018-08-31 09:42:00", "month": "08", "high": 25.58, "low": 25.32, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.58, "open": 25.36, "day": "31"} +{"volume": 12210, "symbol": "YEXT", "ts": "2018-08-31 09:43:00", "month": "08", "high": 25.7799, "low": 25.633, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.7799, "open": 25.633, "day": "31"} +{"volume": 7971, "symbol": "YEXT", "ts": "2018-08-31 09:44:00", "month": "08", "high": 25.5, "low": 25.4, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.5, "open": 25.4, "day": "31"} +{"volume": 4507, "symbol": "YEXT", "ts": "2018-08-31 09:45:00", "month": "08", "high": 25.5, "low": 25.5, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.5, "open": 25.5, "day": "31"} +{"volume": 14233, "symbol": "YEXT", "ts": "2018-08-31 09:46:00", "month": "08", "high": 25.4022, "low": 25.22, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.25, "open": 25.4022, "day": "31"} +{"volume": 11328, "symbol": "YEXT", "ts": "2018-08-31 09:47:00", "month": "08", "high": 25.02, "low": 24.93, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 24.93, "open": 25.02, "day": "31"} +{"volume": 8462, "symbol": "YEXT", "ts": "2018-08-31 09:48:00", "month": "08", "high": 25.09, "low": 25.02, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.09, "open": 25.02, "day": "31"} +{"volume": 3171, "symbol": "YEXT", "ts": "2018-08-31 09:49:00", "month": "08", "high": 25.1101, "low": 25.1101, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.1101, "open": 25.1101, "day": "31"} +{"volume": 6312, "symbol": "YEXT", "ts": "2018-08-31 09:50:00", "month": "08", "high": 25.22, "low": 25.19, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.19, "open": 25.22, "day": "31"} +{"volume": 16763, "symbol": "YEXT", "ts": "2018-08-31 09:51:00", "month": "08", "high": 25.24, "low": 25.13, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.13, "open": 25.18, "day": "31"} +{"volume": 1828, "symbol": "YEXT", "ts": "2018-08-31 09:52:00", "month": "08", "high": 25.25, "low": 25.25, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.25, "open": 25.25, "day": "31"} +{"volume": 5952, "symbol": "YEXT", "ts": "2018-08-31 09:53:00", "month": "08", "high": 25.365, "low": 25.365, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.365, "open": 25.365, "day": "31"} +{"volume": 17718, "symbol": "YEXT", "ts": "2018-08-31 09:54:00", "month": "08", "high": 25.4, "low": 25.3548, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.3548, "open": 25.4, "day": "31"} +{"volume": 2031, "symbol": "YEXT", "ts": "2018-08-31 09:55:00", "month": "08", "high": 25.385, "low": 25.385, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.385, "open": 25.385, "day": "31"} +{"volume": 106990, "symbol": "YEXT", "ts": "2018-08-31 09:56:00", "month": "08", "high": 25.26, "low": 25.24, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.24, "open": 25.26, "day": "31"} +{"volume": 2420, "symbol": "YEXT", "ts": "2018-08-31 09:57:00", "month": "08", "high": 25.26, "low": 25.26, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.26, "open": 25.26, "day": "31"} +{"volume": 2010, "symbol": "YEXT", "ts": "2018-08-31 09:58:00", "month": "08", "high": 25.24, "low": 25.24, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.24, "open": 25.24, "day": "31"} +{"volume": 1588, "symbol": "YEXT", "ts": "2018-08-31 09:59:00", "month": "08", "high": 25.2, "low": 25.2, "key": "YEXT_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 25.2, "open": 25.2, "day": "31"} +{"volume": 2352, "symbol": "GRUB", "ts": "2018-08-31 09:32:00", "month": "08", "high": 142.73, "low": 142.73, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 142.73, "open": 142.73, "day": "31"} +{"volume": 2055, "symbol": "GRUB", "ts": "2018-08-31 09:34:00", "month": "08", "high": 143.2432, "low": 143.2432, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.2432, "open": 143.2432, "day": "31"} +{"volume": 2333, "symbol": "GRUB", "ts": "2018-08-31 09:35:00", "month": "08", "high": 143.2622, "low": 143.2622, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.2622, "open": 143.2622, "day": "31"} +{"volume": 1179, "symbol": "GRUB", "ts": "2018-08-31 09:37:00", "month": "08", "high": 143.5, "low": 143.5, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.5, "open": 143.5, "day": "31"} +{"volume": 5235, "symbol": "GRUB", "ts": "2018-08-31 09:38:00", "month": "08", "high": 144.31, "low": 144.19, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.31, "open": 144.19, "day": "31"} +{"volume": 2123, "symbol": "GRUB", "ts": "2018-08-31 09:39:00", "month": "08", "high": 143.8358, "low": 143.8358, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.8358, "open": 143.8358, "day": "31"} +{"volume": 1576, "symbol": "GRUB", "ts": "2018-08-31 09:40:00", "month": "08", "high": 144.1863, "low": 144.1863, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.1863, "open": 144.1863, "day": "31"} +{"volume": 3164, "symbol": "GRUB", "ts": "2018-08-31 09:42:00", "month": "08", "high": 143.9874, "low": 143.9874, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.9874, "open": 143.9874, "day": "31"} +{"volume": 1705, "symbol": "GRUB", "ts": "2018-08-31 09:43:00", "month": "08", "high": 143.7113, "low": 143.7113, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.7113, "open": 143.7113, "day": "31"} +{"volume": 2514, "symbol": "GRUB", "ts": "2018-08-31 09:45:00", "month": "08", "high": 143.39, "low": 143.39, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.39, "open": 143.39, "day": "31"} +{"volume": 2228, "symbol": "GRUB", "ts": "2018-08-31 09:47:00", "month": "08", "high": 143.34, "low": 143.34, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.34, "open": 143.34, "day": "31"} +{"volume": 551, "symbol": "GRUB", "ts": "2018-08-31 09:48:00", "month": "08", "high": 143.23, "low": 143.23, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.23, "open": 143.23, "day": "31"} +{"volume": 1833, "symbol": "GRUB", "ts": "2018-08-31 09:50:00", "month": "08", "high": 143.57, "low": 143.57, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.57, "open": 143.57, "day": "31"} +{"volume": 1665, "symbol": "GRUB", "ts": "2018-08-31 09:54:00", "month": "08", "high": 143.4, "low": 143.4, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.4, "open": 143.4, "day": "31"} +{"volume": 3717, "symbol": "GRUB", "ts": "2018-08-31 09:55:00", "month": "08", "high": 143.62, "low": 143.58, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.58, "open": 143.62, "day": "31"} +{"volume": 1892, "symbol": "GRUB", "ts": "2018-08-31 09:57:00", "month": "08", "high": 143.36, "low": 143.36, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 143.36, "open": 143.36, "day": "31"} +{"volume": 1866, "symbol": "GRUB", "ts": "2018-08-31 09:58:00", "month": "08", "high": 142.57, "low": 142.57, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 142.57, "open": 142.57, "day": "31"} +{"volume": 1760, "symbol": "GRUB", "ts": "2018-08-31 09:59:00", "month": "08", "high": 142.79, "low": 142.79, "key": "GRUB_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 142.79, "open": 142.79, "day": "31"} +{"volume": 536, "symbol": "DXCM", "ts": "2018-08-31 09:34:00", "month": "08", "high": 144.4794, "low": 144.4794, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.4794, "open": 144.4794, "day": "31"} +{"volume": 725, "symbol": "DXCM", "ts": "2018-08-31 09:36:00", "month": "08", "high": 144.79, "low": 144.79, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.79, "open": 144.79, "day": "31"} +{"volume": 17864, "symbol": "DXCM", "ts": "2018-08-31 09:38:00", "month": "08", "high": 145.26, "low": 144.905, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.26, "open": 144.905, "day": "31"} +{"volume": 7501, "symbol": "DXCM", "ts": "2018-08-31 09:40:00", "month": "08", "high": 145.518, "low": 145.41, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.518, "open": 145.41, "day": "31"} +{"volume": 1189, "symbol": "DXCM", "ts": "2018-08-31 09:41:00", "month": "08", "high": 145.49, "low": 145.49, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.49, "open": 145.49, "day": "31"} +{"volume": 3274, "symbol": "DXCM", "ts": "2018-08-31 09:42:00", "month": "08", "high": 145.62, "low": 145.62, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.62, "open": 145.62, "day": "31"} +{"volume": 8442, "symbol": "DXCM", "ts": "2018-08-31 09:43:00", "month": "08", "high": 145.623, "low": 145.623, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.623, "open": 145.623, "day": "31"} +{"volume": 1902, "symbol": "DXCM", "ts": "2018-08-31 09:44:00", "month": "08", "high": 145.6265, "low": 145.6265, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.6265, "open": 145.6265, "day": "31"} +{"volume": 8483, "symbol": "DXCM", "ts": "2018-08-31 09:46:00", "month": "08", "high": 145.9458, "low": 145.11, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.9458, "open": 145.11, "day": "31"} +{"volume": 1013, "symbol": "DXCM", "ts": "2018-08-31 09:48:00", "month": "08", "high": 145.8855, "low": 145.8855, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.8855, "open": 145.8855, "day": "31"} +{"volume": 1185, "symbol": "DXCM", "ts": "2018-08-31 09:49:00", "month": "08", "high": 145.93, "low": 145.93, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.93, "open": 145.93, "day": "31"} +{"volume": 1863, "symbol": "DXCM", "ts": "2018-08-31 09:51:00", "month": "08", "high": 145.67, "low": 145.67, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.67, "open": 145.67, "day": "31"} +{"volume": 1147, "symbol": "DXCM", "ts": "2018-08-31 09:52:00", "month": "08", "high": 145.06, "low": 145.06, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.06, "open": 145.06, "day": "31"} +{"volume": 2758, "symbol": "DXCM", "ts": "2018-08-31 09:54:00", "month": "08", "high": 145.1628, "low": 145.1628, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 145.1628, "open": 145.1628, "day": "31"} +{"volume": 3958, "symbol": "DXCM", "ts": "2018-08-31 09:56:00", "month": "08", "high": 145.3256, "low": 144.94, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.94, "open": 145.3256, "day": "31"} +{"volume": 3102, "symbol": "DXCM", "ts": "2018-08-31 09:57:00", "month": "08", "high": 144.5, "low": 144.5, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.5, "open": 144.5, "day": "31"} +{"volume": 3073, "symbol": "DXCM", "ts": "2018-08-31 09:58:00", "month": "08", "high": 144.75, "low": 144.75, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.75, "open": 144.75, "day": "31"} +{"volume": 2307, "symbol": "DXCM", "ts": "2018-08-31 09:59:00", "month": "08", "high": 144.7481, "low": 144.7481, "key": "DXCM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 144.7481, "open": 144.7481, "day": "31"} +{"volume": 1006, "symbol": "QURE", "ts": "2018-08-31 09:35:00", "month": "08", "high": 42.51, "low": 42.51, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.51, "open": 42.51, "day": "31"} +{"volume": 1644, "symbol": "QURE", "ts": "2018-08-31 09:37:00", "month": "08", "high": 42.45, "low": 42.45, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.45, "open": 42.45, "day": "31"} +{"volume": 2403, "symbol": "QURE", "ts": "2018-08-31 09:39:00", "month": "08", "high": 42.6, "low": 42.6, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.6, "open": 42.6, "day": "31"} +{"volume": 900, "symbol": "QURE", "ts": "2018-08-31 09:40:00", "month": "08", "high": 42.59, "low": 42.59, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.59, "open": 42.59, "day": "31"} +{"volume": 1991, "symbol": "QURE", "ts": "2018-08-31 09:44:00", "month": "08", "high": 42.43, "low": 42.425, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.43, "open": 42.425, "day": "31"} +{"volume": 260, "symbol": "QURE", "ts": "2018-08-31 09:47:00", "month": "08", "high": 42.43, "low": 42.43, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.43, "open": 42.43, "day": "31"} +{"volume": 121, "symbol": "QURE", "ts": "2018-08-31 09:51:00", "month": "08", "high": 42.51, "low": 42.51, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.51, "open": 42.51, "day": "31"} +{"volume": 2161, "symbol": "QURE", "ts": "2018-08-31 09:56:00", "month": "08", "high": 42.53, "low": 42.53, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.53, "open": 42.53, "day": "31"} +{"volume": 700, "symbol": "QURE", "ts": "2018-08-31 09:57:00", "month": "08", "high": 42.33, "low": 42.33, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.33, "open": 42.33, "day": "31"} +{"volume": 125, "symbol": "QURE", "ts": "2018-08-31 09:58:00", "month": "08", "high": 42.32, "low": 42.32, "key": "QURE_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 42.32, "open": 42.32, "day": "31"} +{"volume": 19282, "symbol": "CRM", "ts": "2018-08-31 09:31:00", "month": "08", "high": 152.3132, "low": 152.24, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.24, "open": 152.26, "day": "31"} +{"volume": 33766, "symbol": "CRM", "ts": "2018-08-31 09:32:00", "month": "08", "high": 152.3099, "low": 152.0, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.02, "open": 152.3099, "day": "31"} +{"volume": 13859, "symbol": "CRM", "ts": "2018-08-31 09:33:00", "month": "08", "high": 152.5, "low": 152.26, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.5, "open": 152.26, "day": "31"} +{"volume": 17410, "symbol": "CRM", "ts": "2018-08-31 09:34:00", "month": "08", "high": 152.725, "low": 152.445, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.47, "open": 152.445, "day": "31"} +{"volume": 23529, "symbol": "CRM", "ts": "2018-08-31 09:35:00", "month": "08", "high": 152.9, "low": 152.48, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.9, "open": 152.48, "day": "31"} +{"volume": 27341, "symbol": "CRM", "ts": "2018-08-31 09:36:00", "month": "08", "high": 153.15, "low": 152.75, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.15, "open": 152.75, "day": "31"} +{"volume": 13769, "symbol": "CRM", "ts": "2018-08-31 09:37:00", "month": "08", "high": 153.165, "low": 153.0, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.165, "open": 153.1, "day": "31"} +{"volume": 54519, "symbol": "CRM", "ts": "2018-08-31 09:38:00", "month": "08", "high": 153.47, "low": 153.27, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.4655, "open": 153.315, "day": "31"} +{"volume": 42828, "symbol": "CRM", "ts": "2018-08-31 09:39:00", "month": "08", "high": 153.6, "low": 153.28, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.58, "open": 153.385, "day": "31"} +{"volume": 64383, "symbol": "CRM", "ts": "2018-08-31 09:40:00", "month": "08", "high": 153.85, "low": 153.58, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.6775, "open": 153.58, "day": "31"} +{"volume": 16600, "symbol": "CRM", "ts": "2018-08-31 09:41:00", "month": "08", "high": 153.72, "low": 153.56, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.56, "open": 153.72, "day": "31"} +{"volume": 16638, "symbol": "CRM", "ts": "2018-08-31 09:42:00", "month": "08", "high": 153.51, "low": 153.41, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.51, "open": 153.5, "day": "31"} +{"volume": 21235, "symbol": "CRM", "ts": "2018-08-31 09:43:00", "month": "08", "high": 153.58, "low": 153.41, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.54, "open": 153.51, "day": "31"} +{"volume": 10516, "symbol": "CRM", "ts": "2018-08-31 09:44:00", "month": "08", "high": 153.44, "low": 153.39, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.39, "open": 153.44, "day": "31"} +{"volume": 20000, "symbol": "CRM", "ts": "2018-08-31 09:45:00", "month": "08", "high": 153.325, "low": 153.1026, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.1026, "open": 153.325, "day": "31"} +{"volume": 13247, "symbol": "CRM", "ts": "2018-08-31 09:46:00", "month": "08", "high": 153.3725, "low": 153.12, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.28, "open": 153.12, "day": "31"} +{"volume": 22251, "symbol": "CRM", "ts": "2018-08-31 09:47:00", "month": "08", "high": 153.37, "low": 153.28, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.36, "open": 153.28, "day": "31"} +{"volume": 44039, "symbol": "CRM", "ts": "2018-08-31 09:48:00", "month": "08", "high": 153.405, "low": 153.18, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.18, "open": 153.405, "day": "31"} +{"volume": 15131, "symbol": "CRM", "ts": "2018-08-31 09:49:00", "month": "08", "high": 153.19, "low": 153.12, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.12, "open": 153.18, "day": "31"} +{"volume": 14616, "symbol": "CRM", "ts": "2018-08-31 09:50:00", "month": "08", "high": 153.03, "low": 152.84, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.84, "open": 153.03, "day": "31"} +{"volume": 17750, "symbol": "CRM", "ts": "2018-08-31 09:51:00", "month": "08", "high": 153.03, "low": 152.84, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.8625, "open": 152.88, "day": "31"} +{"volume": 7624, "symbol": "CRM", "ts": "2018-08-31 09:52:00", "month": "08", "high": 152.8912, "low": 152.88, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.8912, "open": 152.88, "day": "31"} +{"volume": 12355, "symbol": "CRM", "ts": "2018-08-31 09:53:00", "month": "08", "high": 153.05, "low": 152.94, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.02, "open": 152.94, "day": "31"} +{"volume": 12606, "symbol": "CRM", "ts": "2018-08-31 09:54:00", "month": "08", "high": 153.02, "low": 152.95, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.95, "open": 152.97, "day": "31"} +{"volume": 21343, "symbol": "CRM", "ts": "2018-08-31 09:55:00", "month": "08", "high": 153.06, "low": 153.0485, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.06, "open": 153.06, "day": "31"} +{"volume": 8590, "symbol": "CRM", "ts": "2018-08-31 09:56:00", "month": "08", "high": 153.1565, "low": 153.12, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.12, "open": 153.1565, "day": "31"} +{"volume": 21740, "symbol": "CRM", "ts": "2018-08-31 09:57:00", "month": "08", "high": 153.17, "low": 153.05, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 153.05, "open": 153.17, "day": "31"} +{"volume": 8849, "symbol": "CRM", "ts": "2018-08-31 09:58:00", "month": "08", "high": 152.89, "low": 152.88, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.89, "open": 152.88, "day": "31"} +{"volume": 11268, "symbol": "CRM", "ts": "2018-08-31 09:59:00", "month": "08", "high": 152.84, "low": 152.83, "key": "CRM_2018-08-31 09", "year": 2018, "date": "2018/08/31", "close": 152.83, "open": 152.84, "day": "31"} +{"volume": 18722, "symbol": "CRM", "ts": "2018-08-31 10:00:00", "month": "08", "high": 152.835, "low": 152.66, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.66, "open": 152.835, "day": "31"} +{"volume": 37528, "symbol": "MSFT", "ts": "2018-08-31 10:01:00", "month": "08", "high": 112.22, "low": 112.18, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.2, "open": 112.19, "day": "31"} +{"volume": 58075, "symbol": "MSFT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 112.33, "low": 112.24, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.33, "open": 112.24, "day": "31"} +{"volume": 68491, "symbol": "MSFT", "ts": "2018-08-31 10:03:00", "month": "08", "high": 112.43, "low": 112.3445, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.43, "open": 112.3445, "day": "31"} +{"volume": 31008, "symbol": "MSFT", "ts": "2018-08-31 10:04:00", "month": "08", "high": 112.445, "low": 112.4, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.445, "open": 112.42, "day": "31"} +{"volume": 68844, "symbol": "MSFT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 112.4815, "low": 112.45, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.4815, "open": 112.45, "day": "31"} +{"volume": 73828, "symbol": "MSFT", "ts": "2018-08-31 10:06:00", "month": "08", "high": 112.54, "low": 112.48, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.51, "open": 112.49, "day": "31"} +{"volume": 51152, "symbol": "MSFT", "ts": "2018-08-31 10:07:00", "month": "08", "high": 112.58, "low": 112.525, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.58, "open": 112.5354, "day": "31"} +{"volume": 63894, "symbol": "MSFT", "ts": "2018-08-31 10:08:00", "month": "08", "high": 112.57, "low": 112.495, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.52, "open": 112.57, "day": "31"} +{"volume": 68274, "symbol": "MSFT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 112.5015, "low": 112.47, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.5, "open": 112.5015, "day": "31"} +{"volume": 51981, "symbol": "MSFT", "ts": "2018-08-31 10:10:00", "month": "08", "high": 112.57, "low": 112.51, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.55, "open": 112.52, "day": "31"} +{"volume": 31834, "symbol": "MSFT", "ts": "2018-08-31 10:11:00", "month": "08", "high": 112.53, "low": 112.475, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.475, "open": 112.53, "day": "31"} +{"volume": 42325, "symbol": "MSFT", "ts": "2018-08-31 10:12:00", "month": "08", "high": 112.5, "low": 112.469, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.5, "open": 112.48, "day": "31"} +{"volume": 24447, "symbol": "MSFT", "ts": "2018-08-31 10:13:00", "month": "08", "high": 112.535, "low": 112.49, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.535, "open": 112.5, "day": "31"} +{"volume": 41663, "symbol": "MSFT", "ts": "2018-08-31 10:14:00", "month": "08", "high": 112.575, "low": 112.53, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.575, "open": 112.54, "day": "31"} +{"volume": 70535, "symbol": "MSFT", "ts": "2018-08-31 10:15:00", "month": "08", "high": 112.645, "low": 112.57, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.64, "open": 112.58, "day": "31"} +{"volume": 68872, "symbol": "MSFT", "ts": "2018-08-31 10:16:00", "month": "08", "high": 112.705, "low": 112.65, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.66, "open": 112.65, "day": "31"} +{"volume": 45782, "symbol": "MSFT", "ts": "2018-08-31 10:17:00", "month": "08", "high": 112.71, "low": 112.66, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.71, "open": 112.67, "day": "31"} +{"volume": 33181, "symbol": "MSFT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 112.71, "low": 112.66, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.66, "open": 112.71, "day": "31"} +{"volume": 41362, "symbol": "MSFT", "ts": "2018-08-31 10:19:00", "month": "08", "high": 112.65, "low": 112.6039, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.63, "open": 112.65, "day": "31"} +{"volume": 41529, "symbol": "MSFT", "ts": "2018-08-31 10:20:00", "month": "08", "high": 112.68, "low": 112.63, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.649, "open": 112.63, "day": "31"} +{"volume": 26429, "symbol": "MSFT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 112.6482, "low": 112.61, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.64, "open": 112.645, "day": "31"} +{"volume": 49725, "symbol": "MSFT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 112.71, "low": 112.65, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.71, "open": 112.65, "day": "31"} +{"volume": 41912, "symbol": "MSFT", "ts": "2018-08-31 10:23:00", "month": "08", "high": 112.715, "low": 112.68, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.69, "open": 112.715, "day": "31"} +{"volume": 55791, "symbol": "MSFT", "ts": "2018-08-31 10:24:00", "month": "08", "high": 112.75, "low": 112.68, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.75, "open": 112.68, "day": "31"} +{"volume": 40885, "symbol": "MSFT", "ts": "2018-08-31 10:25:00", "month": "08", "high": 112.75, "low": 112.705, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.74, "open": 112.75, "day": "31"} +{"volume": 92768, "symbol": "MSFT", "ts": "2018-08-31 10:26:00", "month": "08", "high": 112.7553, "low": 112.73, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.73, "open": 112.735, "day": "31"} +{"volume": 26508, "symbol": "MSFT", "ts": "2018-08-31 10:27:00", "month": "08", "high": 112.73, "low": 112.7, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.72, "open": 112.73, "day": "31"} +{"volume": 63273, "symbol": "MSFT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 112.7, "low": 112.59, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.62, "open": 112.7, "day": "31"} +{"volume": 29165, "symbol": "MSFT", "ts": "2018-08-31 10:29:00", "month": "08", "high": 112.65, "low": 112.61, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.64, "open": 112.62, "day": "31"} +{"volume": 94426, "symbol": "AAPL", "ts": "2018-08-31 10:01:00", "month": "08", "high": 227.789, "low": 227.7, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.74, "open": 227.7, "day": "31"} +{"volume": 88432, "symbol": "AAPL", "ts": "2018-08-31 10:02:00", "month": "08", "high": 227.94, "low": 227.72, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.92, "open": 227.721, "day": "31"} +{"volume": 96336, "symbol": "AAPL", "ts": "2018-08-31 10:03:00", "month": "08", "high": 228.143, "low": 227.86, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.097, "open": 227.9, "day": "31"} +{"volume": 262493, "symbol": "AAPL", "ts": "2018-08-31 10:04:00", "month": "08", "high": 228.4, "low": 228.11, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.4, "open": 228.12, "day": "31"} +{"volume": 213946, "symbol": "AAPL", "ts": "2018-08-31 10:05:00", "month": "08", "high": 228.49, "low": 228.331, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.49, "open": 228.42, "day": "31"} +{"volume": 244995, "symbol": "AAPL", "ts": "2018-08-31 10:06:00", "month": "08", "high": 228.65, "low": 228.46, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.64, "open": 228.481, "day": "31"} +{"volume": 167705, "symbol": "AAPL", "ts": "2018-08-31 10:07:00", "month": "08", "high": 228.74, "low": 228.6, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.71, "open": 228.63, "day": "31"} +{"volume": 190090, "symbol": "AAPL", "ts": "2018-08-31 10:08:00", "month": "08", "high": 228.75, "low": 228.4963, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.5501, "open": 228.72, "day": "31"} +{"volume": 174312, "symbol": "AAPL", "ts": "2018-08-31 10:09:00", "month": "08", "high": 228.5, "low": 228.14, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.235, "open": 228.5, "day": "31"} +{"volume": 138091, "symbol": "AAPL", "ts": "2018-08-31 10:10:00", "month": "08", "high": 228.56, "low": 228.227, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.52, "open": 228.227, "day": "31"} +{"volume": 114616, "symbol": "AAPL", "ts": "2018-08-31 10:11:00", "month": "08", "high": 228.63, "low": 228.44, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.44, "open": 228.5278, "day": "31"} +{"volume": 103235, "symbol": "AAPL", "ts": "2018-08-31 10:12:00", "month": "08", "high": 228.65, "low": 228.44, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.6, "open": 228.44, "day": "31"} +{"volume": 126021, "symbol": "AAPL", "ts": "2018-08-31 10:13:00", "month": "08", "high": 228.84, "low": 228.61, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.84, "open": 228.61, "day": "31"} +{"volume": 109585, "symbol": "AAPL", "ts": "2018-08-31 10:14:00", "month": "08", "high": 228.85, "low": 228.76, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.7808, "open": 228.81, "day": "31"} +{"volume": 136458, "symbol": "AAPL", "ts": "2018-08-31 10:15:00", "month": "08", "high": 228.87, "low": 228.6227, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.635, "open": 228.82, "day": "31"} +{"volume": 159033, "symbol": "AAPL", "ts": "2018-08-31 10:16:00", "month": "08", "high": 228.64, "low": 228.34, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.36, "open": 228.63, "day": "31"} +{"volume": 149453, "symbol": "AAPL", "ts": "2018-08-31 10:17:00", "month": "08", "high": 228.54, "low": 228.35, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.51, "open": 228.4, "day": "31"} +{"volume": 122093, "symbol": "AAPL", "ts": "2018-08-31 10:18:00", "month": "08", "high": 228.5632, "low": 228.28, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.28, "open": 228.52, "day": "31"} +{"volume": 148654, "symbol": "AAPL", "ts": "2018-08-31 10:19:00", "month": "08", "high": 228.31, "low": 228.07, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.07, "open": 228.25, "day": "31"} +{"volume": 199067, "symbol": "AAPL", "ts": "2018-08-31 10:20:00", "month": "08", "high": 228.22, "low": 228.0, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.01, "open": 228.05, "day": "31"} +{"volume": 161211, "symbol": "AAPL", "ts": "2018-08-31 10:21:00", "month": "08", "high": 228.0525, "low": 227.83, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.83, "open": 228.01, "day": "31"} +{"volume": 198135, "symbol": "AAPL", "ts": "2018-08-31 10:22:00", "month": "08", "high": 227.88, "low": 227.65, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.77, "open": 227.8255, "day": "31"} +{"volume": 137763, "symbol": "AAPL", "ts": "2018-08-31 10:23:00", "month": "08", "high": 227.8, "low": 227.55, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.6, "open": 227.78, "day": "31"} +{"volume": 222538, "symbol": "AAPL", "ts": "2018-08-31 10:24:00", "month": "08", "high": 227.605, "low": 227.36, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.4291, "open": 227.59, "day": "31"} +{"volume": 246211, "symbol": "AAPL", "ts": "2018-08-31 10:25:00", "month": "08", "high": 227.78, "low": 227.42, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.6493, "open": 227.42, "day": "31"} +{"volume": 147549, "symbol": "AAPL", "ts": "2018-08-31 10:26:00", "month": "08", "high": 227.87, "low": 227.63, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.82, "open": 227.63, "day": "31"} +{"volume": 144493, "symbol": "AAPL", "ts": "2018-08-31 10:27:00", "month": "08", "high": 228.0, "low": 227.82, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.98, "open": 227.82, "day": "31"} +{"volume": 123150, "symbol": "AAPL", "ts": "2018-08-31 10:28:00", "month": "08", "high": 227.96, "low": 227.5858, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.65, "open": 227.96, "day": "31"} +{"volume": 85613, "symbol": "AAPL", "ts": "2018-08-31 10:29:00", "month": "08", "high": 227.71, "low": 227.521, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.63, "open": 227.61, "day": "31"} +{"volume": 3911, "symbol": "GOOG", "ts": "2018-08-31 10:01:00", "month": "08", "high": 1229.3635, "low": 1228.7, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.7, "open": 1229.3635, "day": "31"} +{"volume": 1948, "symbol": "GOOG", "ts": "2018-08-31 10:02:00", "month": "08", "high": 1229.41, "low": 1229.41, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.41, "open": 1229.41, "day": "31"} +{"volume": 13450, "symbol": "GOOG", "ts": "2018-08-31 10:03:00", "month": "08", "high": 1230.14, "low": 1229.8101, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.14, "open": 1229.9, "day": "31"} +{"volume": 5239, "symbol": "GOOG", "ts": "2018-08-31 10:04:00", "month": "08", "high": 1230.13, "low": 1230.12, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.13, "open": 1230.12, "day": "31"} +{"volume": 2195, "symbol": "GOOG", "ts": "2018-08-31 10:05:00", "month": "08", "high": 1230.6, "low": 1230.6, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.6, "open": 1230.6, "day": "31"} +{"volume": 3531, "symbol": "GOOG", "ts": "2018-08-31 10:06:00", "month": "08", "high": 1230.8199, "low": 1230.64, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.8199, "open": 1230.64, "day": "31"} +{"volume": 4867, "symbol": "GOOG", "ts": "2018-08-31 10:07:00", "month": "08", "high": 1231.9399, "low": 1231.63, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1231.9399, "open": 1231.63, "day": "31"} +{"volume": 5182, "symbol": "GOOG", "ts": "2018-08-31 10:08:00", "month": "08", "high": 1232.25, "low": 1231.87, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1231.87, "open": 1232.25, "day": "31"} +{"volume": 1525, "symbol": "GOOG", "ts": "2018-08-31 10:09:00", "month": "08", "high": 1232.1329, "low": 1232.1329, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1232.1329, "open": 1232.1329, "day": "31"} +{"volume": 4478, "symbol": "GOOG", "ts": "2018-08-31 10:10:00", "month": "08", "high": 1233.24, "low": 1232.96, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.24, "open": 1232.96, "day": "31"} +{"volume": 1581, "symbol": "GOOG", "ts": "2018-08-31 10:11:00", "month": "08", "high": 1233.0291, "low": 1233.0291, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.0291, "open": 1233.0291, "day": "31"} +{"volume": 1348, "symbol": "GOOG", "ts": "2018-08-31 10:12:00", "month": "08", "high": 1233.255, "low": 1233.255, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.255, "open": 1233.255, "day": "31"} +{"volume": 3876, "symbol": "GOOG", "ts": "2018-08-31 10:13:00", "month": "08", "high": 1233.99, "low": 1233.6899, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.99, "open": 1233.6899, "day": "31"} +{"volume": 3286, "symbol": "GOOG", "ts": "2018-08-31 10:14:00", "month": "08", "high": 1234.11, "low": 1234.11, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1234.11, "open": 1234.11, "day": "31"} +{"volume": 2289, "symbol": "GOOG", "ts": "2018-08-31 10:15:00", "month": "08", "high": 1233.97, "low": 1233.97, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.97, "open": 1233.97, "day": "31"} +{"volume": 4505, "symbol": "GOOG", "ts": "2018-08-31 10:16:00", "month": "08", "high": 1234.915, "low": 1233.9, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.9, "open": 1234.915, "day": "31"} +{"volume": 8413, "symbol": "GOOG", "ts": "2018-08-31 10:17:00", "month": "08", "high": 1233.75, "low": 1233.3013, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.75, "open": 1233.6, "day": "31"} +{"volume": 1983, "symbol": "GOOG", "ts": "2018-08-31 10:19:00", "month": "08", "high": 1233.5114, "low": 1233.5114, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.5114, "open": 1233.5114, "day": "31"} +{"volume": 1367, "symbol": "GOOG", "ts": "2018-08-31 10:20:00", "month": "08", "high": 1233.0551, "low": 1233.0551, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.0551, "open": 1233.0551, "day": "31"} +{"volume": 3019, "symbol": "GOOG", "ts": "2018-08-31 10:21:00", "month": "08", "high": 1232.38, "low": 1232.38, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1232.38, "open": 1232.38, "day": "31"} +{"volume": 2686, "symbol": "GOOG", "ts": "2018-08-31 10:22:00", "month": "08", "high": 1233.245, "low": 1232.67, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.245, "open": 1232.67, "day": "31"} +{"volume": 1586, "symbol": "GOOG", "ts": "2018-08-31 10:24:00", "month": "08", "high": 1233.3199, "low": 1233.3199, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1233.3199, "open": 1233.3199, "day": "31"} +{"volume": 3426, "symbol": "GOOG", "ts": "2018-08-31 10:25:00", "month": "08", "high": 1232.5583, "low": 1231.9041, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1231.9041, "open": 1232.5583, "day": "31"} +{"volume": 1714, "symbol": "GOOG", "ts": "2018-08-31 10:26:00", "month": "08", "high": 1231.425, "low": 1231.425, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1231.425, "open": 1231.425, "day": "31"} +{"volume": 673, "symbol": "GOOG", "ts": "2018-08-31 10:27:00", "month": "08", "high": 1231.425, "low": 1231.425, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1231.425, "open": 1231.425, "day": "31"} +{"volume": 2354, "symbol": "GOOG", "ts": "2018-08-31 10:28:00", "month": "08", "high": 1230.98, "low": 1230.98, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.98, "open": 1230.98, "day": "31"} +{"volume": 3391, "symbol": "GOOG", "ts": "2018-08-31 10:29:00", "month": "08", "high": 1230.1899, "low": 1230.085, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.085, "open": 1230.1899, "day": "31"} +{"volume": 72750, "symbol": "FB", "ts": "2018-08-31 10:01:00", "month": "08", "high": 175.39, "low": 175.16, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.23, "open": 175.38, "day": "31"} +{"volume": 106707, "symbol": "FB", "ts": "2018-08-31 10:02:00", "month": "08", "high": 175.5, "low": 175.23, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.39, "open": 175.23, "day": "31"} +{"volume": 56137, "symbol": "FB", "ts": "2018-08-31 10:03:00", "month": "08", "high": 175.485, "low": 175.39, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.43, "open": 175.39, "day": "31"} +{"volume": 62835, "symbol": "FB", "ts": "2018-08-31 10:04:00", "month": "08", "high": 175.42, "low": 175.3, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.3, "open": 175.42, "day": "31"} +{"volume": 38054, "symbol": "FB", "ts": "2018-08-31 10:05:00", "month": "08", "high": 175.315, "low": 175.28, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.3, "open": 175.31, "day": "31"} +{"volume": 51409, "symbol": "FB", "ts": "2018-08-31 10:06:00", "month": "08", "high": 175.35, "low": 175.22, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.32, "open": 175.29, "day": "31"} +{"volume": 41899, "symbol": "FB", "ts": "2018-08-31 10:07:00", "month": "08", "high": 175.44, "low": 175.33, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.44, "open": 175.37, "day": "31"} +{"volume": 49652, "symbol": "FB", "ts": "2018-08-31 10:08:00", "month": "08", "high": 175.4666, "low": 175.32, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.355, "open": 175.44, "day": "31"} +{"volume": 58170, "symbol": "FB", "ts": "2018-08-31 10:09:00", "month": "08", "high": 175.55, "low": 175.3632, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.48, "open": 175.3632, "day": "31"} +{"volume": 100248, "symbol": "FB", "ts": "2018-08-31 10:10:00", "month": "08", "high": 175.83, "low": 175.5, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.7751, "open": 175.5, "day": "31"} +{"volume": 58687, "symbol": "FB", "ts": "2018-08-31 10:11:00", "month": "08", "high": 175.8769, "low": 175.7, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.8316, "open": 175.72, "day": "31"} +{"volume": 88007, "symbol": "FB", "ts": "2018-08-31 10:12:00", "month": "08", "high": 175.9316, "low": 175.75, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.87, "open": 175.84, "day": "31"} +{"volume": 67303, "symbol": "FB", "ts": "2018-08-31 10:13:00", "month": "08", "high": 175.8601, "low": 175.73, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.815, "open": 175.8347, "day": "31"} +{"volume": 69210, "symbol": "FB", "ts": "2018-08-31 10:14:00", "month": "08", "high": 175.86, "low": 175.69, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.7973, "open": 175.86, "day": "31"} +{"volume": 57883, "symbol": "FB", "ts": "2018-08-31 10:15:00", "month": "08", "high": 175.88, "low": 175.775, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.8699, "open": 175.775, "day": "31"} +{"volume": 52001, "symbol": "FB", "ts": "2018-08-31 10:16:00", "month": "08", "high": 175.82, "low": 175.7215, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.745, "open": 175.82, "day": "31"} +{"volume": 59725, "symbol": "FB", "ts": "2018-08-31 10:17:00", "month": "08", "high": 175.74, "low": 175.6607, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.68, "open": 175.7302, "day": "31"} +{"volume": 33394, "symbol": "FB", "ts": "2018-08-31 10:18:00", "month": "08", "high": 175.71, "low": 175.63, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.63, "open": 175.71, "day": "31"} +{"volume": 46969, "symbol": "FB", "ts": "2018-08-31 10:19:00", "month": "08", "high": 175.65, "low": 175.6, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.64, "open": 175.6, "day": "31"} +{"volume": 40064, "symbol": "FB", "ts": "2018-08-31 10:20:00", "month": "08", "high": 175.63, "low": 175.53, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.53, "open": 175.63, "day": "31"} +{"volume": 45244, "symbol": "FB", "ts": "2018-08-31 10:21:00", "month": "08", "high": 175.55, "low": 175.515, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.52, "open": 175.515, "day": "31"} +{"volume": 64651, "symbol": "FB", "ts": "2018-08-31 10:22:00", "month": "08", "high": 175.63, "low": 175.53, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.58, "open": 175.53, "day": "31"} +{"volume": 34855, "symbol": "FB", "ts": "2018-08-31 10:23:00", "month": "08", "high": 175.7099, "low": 175.6, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.66, "open": 175.6, "day": "31"} +{"volume": 52139, "symbol": "FB", "ts": "2018-08-31 10:24:00", "month": "08", "high": 175.71, "low": 175.63, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.695, "open": 175.68, "day": "31"} +{"volume": 30381, "symbol": "FB", "ts": "2018-08-31 10:25:00", "month": "08", "high": 175.66, "low": 175.62, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.64, "open": 175.64, "day": "31"} +{"volume": 60383, "symbol": "FB", "ts": "2018-08-31 10:26:00", "month": "08", "high": 175.85, "low": 175.65, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.75, "open": 175.65, "day": "31"} +{"volume": 48945, "symbol": "FB", "ts": "2018-08-31 10:27:00", "month": "08", "high": 175.96, "low": 175.779, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.96, "open": 175.779, "day": "31"} +{"volume": 150186, "symbol": "FB", "ts": "2018-08-31 10:28:00", "month": "08", "high": 176.245, "low": 175.922, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.17, "open": 175.97, "day": "31"} +{"volume": 93925, "symbol": "FB", "ts": "2018-08-31 10:29:00", "month": "08", "high": 176.3, "low": 176.18, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.2617, "open": 176.1801, "day": "31"} +{"volume": 44494, "symbol": "NFLX", "ts": "2018-08-31 10:01:00", "month": "08", "high": 373.5, "low": 373.26, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.4334, "open": 373.36, "day": "31"} +{"volume": 20002, "symbol": "NFLX", "ts": "2018-08-31 10:02:00", "month": "08", "high": 373.83, "low": 373.39, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.83, "open": 373.39, "day": "31"} +{"volume": 24877, "symbol": "NFLX", "ts": "2018-08-31 10:03:00", "month": "08", "high": 373.61, "low": 373.41, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.45, "open": 373.5575, "day": "31"} +{"volume": 20062, "symbol": "NFLX", "ts": "2018-08-31 10:04:00", "month": "08", "high": 373.73, "low": 373.5095, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.7, "open": 373.5445, "day": "31"} +{"volume": 26476, "symbol": "NFLX", "ts": "2018-08-31 10:05:00", "month": "08", "high": 373.8655, "low": 373.57, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.6, "open": 373.82, "day": "31"} +{"volume": 25778, "symbol": "NFLX", "ts": "2018-08-31 10:06:00", "month": "08", "high": 373.78, "low": 373.55, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.78, "open": 373.585, "day": "31"} +{"volume": 40445, "symbol": "NFLX", "ts": "2018-08-31 10:07:00", "month": "08", "high": 374.32, "low": 373.8205, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.22, "open": 373.99, "day": "31"} +{"volume": 24558, "symbol": "NFLX", "ts": "2018-08-31 10:08:00", "month": "08", "high": 374.28, "low": 374.13, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.25, "open": 374.28, "day": "31"} +{"volume": 25606, "symbol": "NFLX", "ts": "2018-08-31 10:09:00", "month": "08", "high": 374.1225, "low": 373.57, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.57, "open": 374.1225, "day": "31"} +{"volume": 17249, "symbol": "NFLX", "ts": "2018-08-31 10:10:00", "month": "08", "high": 374.4, "low": 373.8999, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.4, "open": 373.8999, "day": "31"} +{"volume": 14486, "symbol": "NFLX", "ts": "2018-08-31 10:11:00", "month": "08", "high": 374.0627, "low": 373.9764, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.9764, "open": 374.06, "day": "31"} +{"volume": 8518, "symbol": "NFLX", "ts": "2018-08-31 10:12:00", "month": "08", "high": 373.9995, "low": 373.9, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.9, "open": 373.9995, "day": "31"} +{"volume": 7904, "symbol": "NFLX", "ts": "2018-08-31 10:13:00", "month": "08", "high": 374.18, "low": 373.95, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.18, "open": 374.0, "day": "31"} +{"volume": 31475, "symbol": "NFLX", "ts": "2018-08-31 10:14:00", "month": "08", "high": 374.6989, "low": 374.212, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.6898, "open": 374.212, "day": "31"} +{"volume": 20154, "symbol": "NFLX", "ts": "2018-08-31 10:15:00", "month": "08", "high": 374.75, "low": 374.54, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.54, "open": 374.75, "day": "31"} +{"volume": 29543, "symbol": "NFLX", "ts": "2018-08-31 10:16:00", "month": "08", "high": 374.69, "low": 374.4394, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.52, "open": 374.6203, "day": "31"} +{"volume": 28351, "symbol": "NFLX", "ts": "2018-08-31 10:17:00", "month": "08", "high": 374.8786, "low": 374.55, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.741, "open": 374.55, "day": "31"} +{"volume": 26011, "symbol": "NFLX", "ts": "2018-08-31 10:18:00", "month": "08", "high": 374.7129, "low": 374.4, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.4, "open": 374.71, "day": "31"} +{"volume": 28505, "symbol": "NFLX", "ts": "2018-08-31 10:19:00", "month": "08", "high": 374.6483, "low": 374.42, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.52, "open": 374.58, "day": "31"} +{"volume": 23703, "symbol": "NFLX", "ts": "2018-08-31 10:20:00", "month": "08", "high": 374.6, "low": 374.39, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.39, "open": 374.5669, "day": "31"} +{"volume": 44927, "symbol": "NFLX", "ts": "2018-08-31 10:21:00", "month": "08", "high": 374.515, "low": 374.2, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.22, "open": 374.2832, "day": "31"} +{"volume": 15841, "symbol": "NFLX", "ts": "2018-08-31 10:22:00", "month": "08", "high": 374.6, "low": 374.3656, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.51, "open": 374.3656, "day": "31"} +{"volume": 24119, "symbol": "NFLX", "ts": "2018-08-31 10:23:00", "month": "08", "high": 374.8, "low": 374.4, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.7454, "open": 374.4, "day": "31"} +{"volume": 56783, "symbol": "NFLX", "ts": "2018-08-31 10:24:00", "month": "08", "high": 375.09, "low": 374.8, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.94, "open": 374.8, "day": "31"} +{"volume": 27726, "symbol": "NFLX", "ts": "2018-08-31 10:25:00", "month": "08", "high": 375.13, "low": 374.79, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.79, "open": 375.12, "day": "31"} +{"volume": 27610, "symbol": "NFLX", "ts": "2018-08-31 10:26:00", "month": "08", "high": 374.7, "low": 374.54, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.64, "open": 374.6273, "day": "31"} +{"volume": 33919, "symbol": "NFLX", "ts": "2018-08-31 10:27:00", "month": "08", "high": 374.7199, "low": 374.46, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.55, "open": 374.58, "day": "31"} +{"volume": 39746, "symbol": "NFLX", "ts": "2018-08-31 10:28:00", "month": "08", "high": 374.64, "low": 373.85, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.9436, "open": 374.64, "day": "31"} +{"volume": 29786, "symbol": "NFLX", "ts": "2018-08-31 10:29:00", "month": "08", "high": 374.1369, "low": 373.726, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.1369, "open": 373.77, "day": "31"} +{"volume": 26204, "symbol": "TSLA", "ts": "2018-08-31 10:01:00", "month": "08", "high": 301.45, "low": 300.83, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.24, "open": 300.83, "day": "31"} +{"volume": 13488, "symbol": "TSLA", "ts": "2018-08-31 10:02:00", "month": "08", "high": 301.42, "low": 301.11, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.11, "open": 301.42, "day": "31"} +{"volume": 18394, "symbol": "TSLA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 301.435, "low": 301.02, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.27, "open": 301.22, "day": "31"} +{"volume": 14942, "symbol": "TSLA", "ts": "2018-08-31 10:04:00", "month": "08", "high": 301.13, "low": 300.72, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.72, "open": 300.88, "day": "31"} +{"volume": 8471, "symbol": "TSLA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 301.2, "low": 300.9241, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.9241, "open": 301.19, "day": "31"} +{"volume": 16290, "symbol": "TSLA", "ts": "2018-08-31 10:06:00", "month": "08", "high": 301.38, "low": 300.9759, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.27, "open": 300.9759, "day": "31"} +{"volume": 7847, "symbol": "TSLA", "ts": "2018-08-31 10:07:00", "month": "08", "high": 301.36, "low": 301.3296, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.36, "open": 301.3296, "day": "31"} +{"volume": 14718, "symbol": "TSLA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 301.15, "low": 300.76, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.96, "open": 301.15, "day": "31"} +{"volume": 18943, "symbol": "TSLA", "ts": "2018-08-31 10:09:00", "month": "08", "high": 300.87, "low": 300.25, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.39, "open": 300.87, "day": "31"} +{"volume": 21836, "symbol": "TSLA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 300.94, "low": 300.55, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.94, "open": 300.55, "day": "31"} +{"volume": 9723, "symbol": "TSLA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 300.85, "low": 300.7, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.84, "open": 300.7, "day": "31"} +{"volume": 14158, "symbol": "TSLA", "ts": "2018-08-31 10:12:00", "month": "08", "high": 300.993, "low": 300.7101, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.7101, "open": 300.993, "day": "31"} +{"volume": 12590, "symbol": "TSLA", "ts": "2018-08-31 10:13:00", "month": "08", "high": 301.32, "low": 300.99, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.32, "open": 300.99, "day": "31"} +{"volume": 15974, "symbol": "TSLA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 301.305, "low": 300.738, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.738, "open": 301.305, "day": "31"} +{"volume": 15769, "symbol": "TSLA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 301.25, "low": 300.9999, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.25, "open": 300.9999, "day": "31"} +{"volume": 13852, "symbol": "TSLA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 301.31, "low": 301.0053, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.31, "open": 301.0053, "day": "31"} +{"volume": 25796, "symbol": "TSLA", "ts": "2018-08-31 10:17:00", "month": "08", "high": 301.75, "low": 301.38, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.63, "open": 301.3937, "day": "31"} +{"volume": 50439, "symbol": "TSLA", "ts": "2018-08-31 10:18:00", "month": "08", "high": 302.27, "low": 301.69, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.69, "open": 301.8, "day": "31"} +{"volume": 34047, "symbol": "TSLA", "ts": "2018-08-31 10:19:00", "month": "08", "high": 301.8405, "low": 301.5, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.535, "open": 301.7, "day": "31"} +{"volume": 9270, "symbol": "TSLA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 301.84, "low": 301.62, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.62, "open": 301.68, "day": "31"} +{"volume": 9267, "symbol": "TSLA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 301.8572, "low": 301.6687, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.8572, "open": 301.6687, "day": "31"} +{"volume": 30858, "symbol": "TSLA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 301.89, "low": 301.61, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.61, "open": 301.73, "day": "31"} +{"volume": 9759, "symbol": "TSLA", "ts": "2018-08-31 10:23:00", "month": "08", "high": 301.885, "low": 301.65, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.885, "open": 301.65, "day": "31"} +{"volume": 9953, "symbol": "TSLA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 301.99, "low": 301.72, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.86, "open": 301.72, "day": "31"} +{"volume": 8662, "symbol": "TSLA", "ts": "2018-08-31 10:25:00", "month": "08", "high": 301.8165, "low": 301.58, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.58, "open": 301.8165, "day": "31"} +{"volume": 28700, "symbol": "TSLA", "ts": "2018-08-31 10:26:00", "month": "08", "high": 301.5764, "low": 300.88, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.88, "open": 301.5764, "day": "31"} +{"volume": 16211, "symbol": "TSLA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 301.02, "low": 300.85, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 301.01, "open": 300.96, "day": "31"} +{"volume": 19019, "symbol": "TSLA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 300.71, "low": 300.52, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.6172, "open": 300.66, "day": "31"} +{"volume": 4613, "symbol": "TSLA", "ts": "2018-08-31 10:29:00", "month": "08", "high": 300.99, "low": 300.89, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.89, "open": 300.99, "day": "31"} +{"volume": 34156, "symbol": "F", "ts": "2018-08-31 10:01:00", "month": "08", "high": 9.6554, "low": 9.6514, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6514, "open": 9.6554, "day": "31"} +{"volume": 52538, "symbol": "F", "ts": "2018-08-31 10:02:00", "month": "08", "high": 9.655, "low": 9.65, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.655, "day": "31"} +{"volume": 48642, "symbol": "F", "ts": "2018-08-31 10:03:00", "month": "08", "high": 9.655, "low": 9.65, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.6536, "day": "31"} +{"volume": 110638, "symbol": "F", "ts": "2018-08-31 10:04:00", "month": "08", "high": 9.65, "low": 9.65, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.65, "day": "31"} +{"volume": 62133, "symbol": "F", "ts": "2018-08-31 10:05:00", "month": "08", "high": 9.65, "low": 9.641, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6432, "open": 9.641, "day": "31"} +{"volume": 15526, "symbol": "F", "ts": "2018-08-31 10:06:00", "month": "08", "high": 9.65, "low": 9.6444, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.6444, "day": "31"} +{"volume": 284067, "symbol": "F", "ts": "2018-08-31 10:07:00", "month": "08", "high": 9.655, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.649, "open": 9.645, "day": "31"} +{"volume": 52214, "symbol": "F", "ts": "2018-08-31 10:08:00", "month": "08", "high": 9.645, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6408, "open": 9.645, "day": "31"} +{"volume": 59077, "symbol": "F", "ts": "2018-08-31 10:09:00", "month": "08", "high": 9.65, "low": 9.645, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.65, "open": 9.65, "day": "31"} +{"volume": 31605, "symbol": "F", "ts": "2018-08-31 10:10:00", "month": "08", "high": 9.65, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.64, "open": 9.645, "day": "31"} +{"volume": 29332, "symbol": "F", "ts": "2018-08-31 10:11:00", "month": "08", "high": 9.646, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.645, "open": 9.64, "day": "31"} +{"volume": 17001, "symbol": "F", "ts": "2018-08-31 10:12:00", "month": "08", "high": 9.645, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.64, "open": 9.645, "day": "31"} +{"volume": 204684, "symbol": "F", "ts": "2018-08-31 10:13:00", "month": "08", "high": 9.645, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.64, "open": 9.64, "day": "31"} +{"volume": 157456, "symbol": "F", "ts": "2018-08-31 10:14:00", "month": "08", "high": 9.64, "low": 9.63, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.64, "open": 9.6355, "day": "31"} +{"volume": 198316, "symbol": "F", "ts": "2018-08-31 10:15:00", "month": "08", "high": 9.66, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.655, "open": 9.64, "day": "31"} +{"volume": 36671, "symbol": "F", "ts": "2018-08-31 10:16:00", "month": "08", "high": 9.66, "low": 9.65, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.655, "open": 9.65, "day": "31"} +{"volume": 31228, "symbol": "F", "ts": "2018-08-31 10:17:00", "month": "08", "high": 9.66, "low": 9.655, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6594, "open": 9.66, "day": "31"} +{"volume": 49687, "symbol": "F", "ts": "2018-08-31 10:18:00", "month": "08", "high": 9.66, "low": 9.655, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.66, "open": 9.66, "day": "31"} +{"volume": 170937, "symbol": "F", "ts": "2018-08-31 10:19:00", "month": "08", "high": 9.675, "low": 9.655, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.675, "open": 9.66, "day": "31"} +{"volume": 244534, "symbol": "F", "ts": "2018-08-31 10:20:00", "month": "08", "high": 9.68, "low": 9.67, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.67, "open": 9.675, "day": "31"} +{"volume": 357595, "symbol": "F", "ts": "2018-08-31 10:21:00", "month": "08", "high": 9.67, "low": 9.655, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.655, "open": 9.67, "day": "31"} +{"volume": 248357, "symbol": "F", "ts": "2018-08-31 10:22:00", "month": "08", "high": 9.655, "low": 9.645, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.645, "open": 9.651, "day": "31"} +{"volume": 23928, "symbol": "F", "ts": "2018-08-31 10:23:00", "month": "08", "high": 9.6477, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.645, "open": 9.64, "day": "31"} +{"volume": 22427, "symbol": "F", "ts": "2018-08-31 10:24:00", "month": "08", "high": 9.649, "low": 9.64, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.64, "open": 9.649, "day": "31"} +{"volume": 90599, "symbol": "F", "ts": "2018-08-31 10:25:00", "month": "08", "high": 9.64, "low": 9.6338, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6338, "open": 9.64, "day": "31"} +{"volume": 13989, "symbol": "F", "ts": "2018-08-31 10:26:00", "month": "08", "high": 9.63, "low": 9.63, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.63, "open": 9.63, "day": "31"} +{"volume": 75866, "symbol": "F", "ts": "2018-08-31 10:27:00", "month": "08", "high": 9.63, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.625, "open": 9.63, "day": "31"} +{"volume": 55690, "symbol": "F", "ts": "2018-08-31 10:28:00", "month": "08", "high": 9.62, "low": 9.61, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6151, "open": 9.62, "day": "31"} +{"volume": 164743, "symbol": "F", "ts": "2018-08-31 10:29:00", "month": "08", "high": 9.63, "low": 9.616, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.62, "open": 9.616, "day": "31"} +{"volume": 12667, "symbol": "AMZN", "ts": "2018-08-31 10:01:00", "month": "08", "high": 2018.27, "low": 2017.5601, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.6899, "open": 2017.5601, "day": "31"} +{"volume": 9601, "symbol": "AMZN", "ts": "2018-08-31 10:02:00", "month": "08", "high": 2018.74, "low": 2018.2227, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.63, "open": 2018.2227, "day": "31"} +{"volume": 9578, "symbol": "AMZN", "ts": "2018-08-31 10:03:00", "month": "08", "high": 2019.09, "low": 2018.52, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.03, "open": 2019.0, "day": "31"} +{"volume": 6093, "symbol": "AMZN", "ts": "2018-08-31 10:04:00", "month": "08", "high": 2018.9543, "low": 2018.6031, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.9543, "open": 2018.85, "day": "31"} +{"volume": 6128, "symbol": "AMZN", "ts": "2018-08-31 10:05:00", "month": "08", "high": 2018.8658, "low": 2018.6899, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.8658, "open": 2018.6899, "day": "31"} +{"volume": 6354, "symbol": "AMZN", "ts": "2018-08-31 10:06:00", "month": "08", "high": 2018.9399, "low": 2018.54, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.9399, "open": 2018.54, "day": "31"} +{"volume": 10666, "symbol": "AMZN", "ts": "2018-08-31 10:07:00", "month": "08", "high": 2019.59, "low": 2018.91, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.59, "open": 2018.91, "day": "31"} +{"volume": 12864, "symbol": "AMZN", "ts": "2018-08-31 10:08:00", "month": "08", "high": 2019.8, "low": 2018.08, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.08, "open": 2019.8, "day": "31"} +{"volume": 9047, "symbol": "AMZN", "ts": "2018-08-31 10:09:00", "month": "08", "high": 2017.9301, "low": 2017.0, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.1, "open": 2017.6899, "day": "31"} +{"volume": 9356, "symbol": "AMZN", "ts": "2018-08-31 10:10:00", "month": "08", "high": 2018.9509, "low": 2017.5916, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.9509, "open": 2017.5916, "day": "31"} +{"volume": 10899, "symbol": "AMZN", "ts": "2018-08-31 10:11:00", "month": "08", "high": 2019.8, "low": 2019.4, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.6193, "open": 2019.4, "day": "31"} +{"volume": 12139, "symbol": "AMZN", "ts": "2018-08-31 10:12:00", "month": "08", "high": 2019.8, "low": 2018.6801, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.6801, "open": 2019.45, "day": "31"} +{"volume": 19967, "symbol": "AMZN", "ts": "2018-08-31 10:13:00", "month": "08", "high": 2020.25, "low": 2019.5, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2020.25, "open": 2019.5, "day": "31"} +{"volume": 29883, "symbol": "AMZN", "ts": "2018-08-31 10:14:00", "month": "08", "high": 2021.1, "low": 2019.76, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.1, "open": 2020.015, "day": "31"} +{"volume": 35205, "symbol": "AMZN", "ts": "2018-08-31 10:15:00", "month": "08", "high": 2022.0, "low": 2020.13, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.5343, "open": 2021.4, "day": "31"} +{"volume": 14255, "symbol": "AMZN", "ts": "2018-08-31 10:16:00", "month": "08", "high": 2021.615, "low": 2021.27, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.2811, "open": 2021.615, "day": "31"} +{"volume": 14636, "symbol": "AMZN", "ts": "2018-08-31 10:17:00", "month": "08", "high": 2021.76, "low": 2020.76, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.7075, "open": 2020.76, "day": "31"} +{"volume": 15779, "symbol": "AMZN", "ts": "2018-08-31 10:18:00", "month": "08", "high": 2022.1801, "low": 2021.21, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.6777, "open": 2022.1801, "day": "31"} +{"volume": 11355, "symbol": "AMZN", "ts": "2018-08-31 10:19:00", "month": "08", "high": 2021.425, "low": 2020.98, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.1534, "open": 2021.425, "day": "31"} +{"volume": 16354, "symbol": "AMZN", "ts": "2018-08-31 10:20:00", "month": "08", "high": 2021.08, "low": 2020.02, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2020.02, "open": 2021.03, "day": "31"} +{"volume": 21121, "symbol": "AMZN", "ts": "2018-08-31 10:21:00", "month": "08", "high": 2020.02, "low": 2018.0, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.0, "open": 2020.02, "day": "31"} +{"volume": 16678, "symbol": "AMZN", "ts": "2018-08-31 10:22:00", "month": "08", "high": 2020.28, "low": 2019.1835, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2020.0551, "open": 2019.4301, "day": "31"} +{"volume": 6489, "symbol": "AMZN", "ts": "2018-08-31 10:23:00", "month": "08", "high": 2019.84, "low": 2019.395, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.395, "open": 2019.84, "day": "31"} +{"volume": 11948, "symbol": "AMZN", "ts": "2018-08-31 10:24:00", "month": "08", "high": 2019.98, "low": 2018.89, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.98, "open": 2019.8199, "day": "31"} +{"volume": 22765, "symbol": "AMZN", "ts": "2018-08-31 10:25:00", "month": "08", "high": 2018.59, "low": 2017.09, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.09, "open": 2018.59, "day": "31"} +{"volume": 13083, "symbol": "AMZN", "ts": "2018-08-31 10:26:00", "month": "08", "high": 2017.575, "low": 2017.12, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.1935, "open": 2017.12, "day": "31"} +{"volume": 12526, "symbol": "AMZN", "ts": "2018-08-31 10:27:00", "month": "08", "high": 2017.2834, "low": 2016.89, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.2834, "open": 2017.0, "day": "31"} +{"volume": 14602, "symbol": "AMZN", "ts": "2018-08-31 10:28:00", "month": "08", "high": 2016.78, "low": 2016.12, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.395, "open": 2016.78, "day": "31"} +{"volume": 16995, "symbol": "AMZN", "ts": "2018-08-31 10:29:00", "month": "08", "high": 2016.2229, "low": 2015.13, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.2229, "open": 2016.0, "day": "31"} +{"volume": 18611, "symbol": "NVDA", "ts": "2018-08-31 10:01:00", "month": "08", "high": 279.72, "low": 279.5, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.6416, "open": 279.5, "day": "31"} +{"volume": 31087, "symbol": "NVDA", "ts": "2018-08-31 10:02:00", "month": "08", "high": 279.8604, "low": 279.73, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.84, "open": 279.73, "day": "31"} +{"volume": 22926, "symbol": "NVDA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 279.9, "low": 279.61, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.83, "open": 279.9, "day": "31"} +{"volume": 15711, "symbol": "NVDA", "ts": "2018-08-31 10:04:00", "month": "08", "high": 279.8456, "low": 279.64, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.64, "open": 279.81, "day": "31"} +{"volume": 7114, "symbol": "NVDA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 279.75, "low": 279.71, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.75, "open": 279.71, "day": "31"} +{"volume": 11779, "symbol": "NVDA", "ts": "2018-08-31 10:06:00", "month": "08", "high": 279.76, "low": 279.74, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.74, "open": 279.76, "day": "31"} +{"volume": 22620, "symbol": "NVDA", "ts": "2018-08-31 10:07:00", "month": "08", "high": 279.84, "low": 279.68, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.7, "open": 279.84, "day": "31"} +{"volume": 17202, "symbol": "NVDA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 279.699, "low": 279.58, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.58, "open": 279.58, "day": "31"} +{"volume": 21527, "symbol": "NVDA", "ts": "2018-08-31 10:09:00", "month": "08", "high": 279.49, "low": 279.33, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.33, "open": 279.49, "day": "31"} +{"volume": 29104, "symbol": "NVDA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 279.33, "low": 279.1751, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.1751, "open": 279.25, "day": "31"} +{"volume": 27148, "symbol": "NVDA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 279.47, "low": 279.17, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.47, "open": 279.17, "day": "31"} +{"volume": 19609, "symbol": "NVDA", "ts": "2018-08-31 10:12:00", "month": "08", "high": 279.5, "low": 279.29, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.29, "open": 279.5, "day": "31"} +{"volume": 11964, "symbol": "NVDA", "ts": "2018-08-31 10:13:00", "month": "08", "high": 279.36, "low": 279.3324, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.3324, "open": 279.3599, "day": "31"} +{"volume": 15247, "symbol": "NVDA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 279.4625, "low": 279.32, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.36, "open": 279.4625, "day": "31"} +{"volume": 9988, "symbol": "NVDA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 279.5, "low": 279.4582, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.5, "open": 279.4582, "day": "31"} +{"volume": 9970, "symbol": "NVDA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 279.48, "low": 279.4372, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.4372, "open": 279.48, "day": "31"} +{"volume": 6172, "symbol": "NVDA", "ts": "2018-08-31 10:17:00", "month": "08", "high": 279.43, "low": 279.4238, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.4238, "open": 279.43, "day": "31"} +{"volume": 11607, "symbol": "NVDA", "ts": "2018-08-31 10:18:00", "month": "08", "high": 279.47, "low": 279.4201, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.4495, "open": 279.47, "day": "31"} +{"volume": 15580, "symbol": "NVDA", "ts": "2018-08-31 10:19:00", "month": "08", "high": 279.31, "low": 279.25, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.29, "open": 279.31, "day": "31"} +{"volume": 11404, "symbol": "NVDA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 279.47, "low": 279.34, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.39, "open": 279.34, "day": "31"} +{"volume": 5914, "symbol": "NVDA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 279.49, "low": 279.42, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.42, "open": 279.49, "day": "31"} +{"volume": 15290, "symbol": "NVDA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 279.65, "low": 279.49, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.65, "open": 279.49, "day": "31"} +{"volume": 71792, "symbol": "NVDA", "ts": "2018-08-31 10:23:00", "month": "08", "high": 280.15, "low": 279.69, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.1273, "open": 279.69, "day": "31"} +{"volume": 61785, "symbol": "NVDA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 280.4001, "low": 280.19, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.362, "open": 280.199, "day": "31"} +{"volume": 12330, "symbol": "NVDA", "ts": "2018-08-31 10:25:00", "month": "08", "high": 280.26, "low": 280.2, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.2, "open": 280.26, "day": "31"} +{"volume": 19239, "symbol": "NVDA", "ts": "2018-08-31 10:26:00", "month": "08", "high": 280.21, "low": 279.89, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.89, "open": 280.19, "day": "31"} +{"volume": 22687, "symbol": "NVDA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 280.18, "low": 279.96, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.16, "open": 279.96, "day": "31"} +{"volume": 20289, "symbol": "NVDA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 280.15, "low": 280.0, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.0, "open": 280.15, "day": "31"} +{"volume": 18345, "symbol": "NVDA", "ts": "2018-08-31 10:29:00", "month": "08", "high": 280.055, "low": 279.965, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.055, "open": 280.03, "day": "31"} +{"volume": 57675, "symbol": "INTC", "ts": "2018-08-31 10:01:00", "month": "08", "high": 48.15, "low": 48.1051, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.1051, "open": 48.14, "day": "31"} +{"volume": 52171, "symbol": "INTC", "ts": "2018-08-31 10:02:00", "month": "08", "high": 48.12, "low": 48.055, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.06, "open": 48.12, "day": "31"} +{"volume": 65334, "symbol": "INTC", "ts": "2018-08-31 10:03:00", "month": "08", "high": 48.07, "low": 48.045, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.055, "open": 48.065, "day": "31"} +{"volume": 81036, "symbol": "INTC", "ts": "2018-08-31 10:04:00", "month": "08", "high": 48.06, "low": 48.035, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.045, "open": 48.04, "day": "31"} +{"volume": 44500, "symbol": "INTC", "ts": "2018-08-31 10:05:00", "month": "08", "high": 48.095, "low": 48.055, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.095, "open": 48.055, "day": "31"} +{"volume": 19434, "symbol": "INTC", "ts": "2018-08-31 10:06:00", "month": "08", "high": 48.13, "low": 48.1075, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.13, "open": 48.1075, "day": "31"} +{"volume": 45501, "symbol": "INTC", "ts": "2018-08-31 10:07:00", "month": "08", "high": 48.19, "low": 48.13, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.19, "open": 48.13, "day": "31"} +{"volume": 44261, "symbol": "INTC", "ts": "2018-08-31 10:08:00", "month": "08", "high": 48.18, "low": 48.15, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.15, "open": 48.18, "day": "31"} +{"volume": 34878, "symbol": "INTC", "ts": "2018-08-31 10:09:00", "month": "08", "high": 48.14, "low": 48.13, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.13, "open": 48.14, "day": "31"} +{"volume": 73191, "symbol": "INTC", "ts": "2018-08-31 10:10:00", "month": "08", "high": 48.21, "low": 48.145, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.21, "open": 48.145, "day": "31"} +{"volume": 20414, "symbol": "INTC", "ts": "2018-08-31 10:11:00", "month": "08", "high": 48.22, "low": 48.2, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.21, "open": 48.2, "day": "31"} +{"volume": 26758, "symbol": "INTC", "ts": "2018-08-31 10:12:00", "month": "08", "high": 48.21, "low": 48.19, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.19, "open": 48.21, "day": "31"} +{"volume": 42848, "symbol": "INTC", "ts": "2018-08-31 10:13:00", "month": "08", "high": 48.23, "low": 48.2, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.23, "open": 48.2, "day": "31"} +{"volume": 33851, "symbol": "INTC", "ts": "2018-08-31 10:14:00", "month": "08", "high": 48.27, "low": 48.22, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.27, "open": 48.22, "day": "31"} +{"volume": 61699, "symbol": "INTC", "ts": "2018-08-31 10:15:00", "month": "08", "high": 48.31, "low": 48.2754, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.31, "open": 48.28, "day": "31"} +{"volume": 28399, "symbol": "INTC", "ts": "2018-08-31 10:16:00", "month": "08", "high": 48.2801, "low": 48.26, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.2801, "open": 48.28, "day": "31"} +{"volume": 13196, "symbol": "INTC", "ts": "2018-08-31 10:17:00", "month": "08", "high": 48.29, "low": 48.285, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.29, "open": 48.285, "day": "31"} +{"volume": 25142, "symbol": "INTC", "ts": "2018-08-31 10:18:00", "month": "08", "high": 48.29, "low": 48.285, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.29, "open": 48.285, "day": "31"} +{"volume": 23529, "symbol": "INTC", "ts": "2018-08-31 10:19:00", "month": "08", "high": 48.28, "low": 48.26, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.27, "open": 48.28, "day": "31"} +{"volume": 26706, "symbol": "INTC", "ts": "2018-08-31 10:20:00", "month": "08", "high": 48.2825, "low": 48.26, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.2825, "open": 48.26, "day": "31"} +{"volume": 23825, "symbol": "INTC", "ts": "2018-08-31 10:21:00", "month": "08", "high": 48.2999, "low": 48.285, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.2999, "open": 48.285, "day": "31"} +{"volume": 29899, "symbol": "INTC", "ts": "2018-08-31 10:22:00", "month": "08", "high": 48.32, "low": 48.3, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.32, "open": 48.3, "day": "31"} +{"volume": 36529, "symbol": "INTC", "ts": "2018-08-31 10:23:00", "month": "08", "high": 48.325, "low": 48.3, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.32, "open": 48.325, "day": "31"} +{"volume": 41439, "symbol": "INTC", "ts": "2018-08-31 10:24:00", "month": "08", "high": 48.365, "low": 48.34, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.365, "open": 48.34, "day": "31"} +{"volume": 41020, "symbol": "INTC", "ts": "2018-08-31 10:25:00", "month": "08", "high": 48.34, "low": 48.315, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.33, "open": 48.34, "day": "31"} +{"volume": 41350, "symbol": "INTC", "ts": "2018-08-31 10:26:00", "month": "08", "high": 48.325, "low": 48.29, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.29, "open": 48.325, "day": "31"} +{"volume": 25849, "symbol": "INTC", "ts": "2018-08-31 10:27:00", "month": "08", "high": 48.31, "low": 48.29, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.31, "open": 48.291, "day": "31"} +{"volume": 33016, "symbol": "INTC", "ts": "2018-08-31 10:28:00", "month": "08", "high": 48.31, "low": 48.28, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.29, "open": 48.31, "day": "31"} +{"volume": 40769, "symbol": "INTC", "ts": "2018-08-31 10:29:00", "month": "08", "high": 48.32, "low": 48.29, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.31, "open": 48.29, "day": "31"} +{"volume": 17451, "symbol": "TNDM", "ts": "2018-08-31 10:01:00", "month": "08", "high": 45.08, "low": 44.81, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.0168, "open": 44.81, "day": "31"} +{"volume": 5619, "symbol": "TNDM", "ts": "2018-08-31 10:02:00", "month": "08", "high": 45.0, "low": 44.88, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.88, "open": 45.0, "day": "31"} +{"volume": 2974, "symbol": "TNDM", "ts": "2018-08-31 10:03:00", "month": "08", "high": 44.87, "low": 44.87, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.87, "open": 44.87, "day": "31"} +{"volume": 12782, "symbol": "TNDM", "ts": "2018-08-31 10:04:00", "month": "08", "high": 44.9992, "low": 44.99, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.99, "open": 44.99, "day": "31"} +{"volume": 4770, "symbol": "TNDM", "ts": "2018-08-31 10:05:00", "month": "08", "high": 44.95, "low": 44.95, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.95, "open": 44.95, "day": "31"} +{"volume": 13966, "symbol": "TNDM", "ts": "2018-08-31 10:06:00", "month": "08", "high": 44.95, "low": 44.7621, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7621, "open": 44.95, "day": "31"} +{"volume": 2370, "symbol": "TNDM", "ts": "2018-08-31 10:07:00", "month": "08", "high": 44.7393, "low": 44.7393, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7393, "open": 44.7393, "day": "31"} +{"volume": 9318, "symbol": "TNDM", "ts": "2018-08-31 10:08:00", "month": "08", "high": 44.62, "low": 44.61, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.61, "open": 44.62, "day": "31"} +{"volume": 13714, "symbol": "TNDM", "ts": "2018-08-31 10:09:00", "month": "08", "high": 44.72, "low": 44.7, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.715, "open": 44.7, "day": "31"} +{"volume": 7553, "symbol": "TNDM", "ts": "2018-08-31 10:10:00", "month": "08", "high": 44.9, "low": 44.84, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.9, "open": 44.84, "day": "31"} +{"volume": 1912, "symbol": "TNDM", "ts": "2018-08-31 10:11:00", "month": "08", "high": 44.855, "low": 44.855, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.855, "open": 44.855, "day": "31"} +{"volume": 6039, "symbol": "TNDM", "ts": "2018-08-31 10:12:00", "month": "08", "high": 44.73, "low": 44.73, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.73, "open": 44.73, "day": "31"} +{"volume": 4396, "symbol": "TNDM", "ts": "2018-08-31 10:13:00", "month": "08", "high": 44.835, "low": 44.77, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.77, "open": 44.835, "day": "31"} +{"volume": 9789, "symbol": "TNDM", "ts": "2018-08-31 10:14:00", "month": "08", "high": 44.8289, "low": 44.82, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.8289, "open": 44.82, "day": "31"} +{"volume": 4017, "symbol": "TNDM", "ts": "2018-08-31 10:15:00", "month": "08", "high": 44.84, "low": 44.84, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.84, "open": 44.84, "day": "31"} +{"volume": 3420, "symbol": "TNDM", "ts": "2018-08-31 10:16:00", "month": "08", "high": 44.78, "low": 44.78, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.78, "open": 44.78, "day": "31"} +{"volume": 10458, "symbol": "TNDM", "ts": "2018-08-31 10:17:00", "month": "08", "high": 44.949, "low": 44.8178, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.949, "open": 44.8178, "day": "31"} +{"volume": 7599, "symbol": "TNDM", "ts": "2018-08-31 10:18:00", "month": "08", "high": 45.0, "low": 44.95, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.0, "open": 44.95, "day": "31"} +{"volume": 5156, "symbol": "TNDM", "ts": "2018-08-31 10:19:00", "month": "08", "high": 44.97, "low": 44.88, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.97, "open": 44.88, "day": "31"} +{"volume": 6211, "symbol": "TNDM", "ts": "2018-08-31 10:20:00", "month": "08", "high": 45.0, "low": 44.96, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.96, "open": 45.0, "day": "31"} +{"volume": 1807, "symbol": "TNDM", "ts": "2018-08-31 10:21:00", "month": "08", "high": 44.955, "low": 44.955, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.955, "open": 44.955, "day": "31"} +{"volume": 4468, "symbol": "TNDM", "ts": "2018-08-31 10:22:00", "month": "08", "high": 44.925, "low": 44.925, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.925, "open": 44.925, "day": "31"} +{"volume": 4663, "symbol": "TNDM", "ts": "2018-08-31 10:23:00", "month": "08", "high": 44.95, "low": 44.91, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.95, "open": 44.91, "day": "31"} +{"volume": 8985, "symbol": "TNDM", "ts": "2018-08-31 10:24:00", "month": "08", "high": 44.9, "low": 44.89, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.89, "open": 44.9, "day": "31"} +{"volume": 17056, "symbol": "TNDM", "ts": "2018-08-31 10:25:00", "month": "08", "high": 44.99, "low": 44.97, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.99, "open": 44.97, "day": "31"} +{"volume": 9016, "symbol": "TNDM", "ts": "2018-08-31 10:26:00", "month": "08", "high": 44.9843, "low": 44.96, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.9843, "open": 44.96, "day": "31"} +{"volume": 3002, "symbol": "TNDM", "ts": "2018-08-31 10:27:00", "month": "08", "high": 45.0, "low": 45.0, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.0, "open": 45.0, "day": "31"} +{"volume": 10108, "symbol": "TNDM", "ts": "2018-08-31 10:28:00", "month": "08", "high": 44.98, "low": 44.9685, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.98, "open": 44.9685, "day": "31"} +{"volume": 2275, "symbol": "TNDM", "ts": "2018-08-31 10:29:00", "month": "08", "high": 45.0, "low": 45.0, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.0, "open": 45.0, "day": "31"} +{"volume": 628, "symbol": "CDNA", "ts": "2018-08-31 10:02:00", "month": "08", "high": 23.65, "low": 23.65, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.65, "open": 23.65, "day": "31"} +{"volume": 2525, "symbol": "CDNA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 23.571, "low": 23.571, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.571, "open": 23.571, "day": "31"} +{"volume": 1568, "symbol": "CDNA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 23.6011, "low": 23.6011, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.6011, "open": 23.6011, "day": "31"} +{"volume": 251, "symbol": "CDNA", "ts": "2018-08-31 10:06:00", "month": "08", "high": 23.57, "low": 23.57, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.57, "open": 23.57, "day": "31"} +{"volume": 1410, "symbol": "CDNA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 23.62, "low": 23.62, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.62, "open": 23.62, "day": "31"} +{"volume": 3211, "symbol": "CDNA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 23.53, "low": 23.53, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.53, "open": 23.53, "day": "31"} +{"volume": 210, "symbol": "CDNA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 23.59, "low": 23.59, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.59, "open": 23.59, "day": "31"} +{"volume": 1703, "symbol": "CDNA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 23.51, "low": 23.51, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.51, "open": 23.51, "day": "31"} +{"volume": 2221, "symbol": "CDNA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 23.58, "low": 23.58, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.58, "open": 23.58, "day": "31"} +{"volume": 5683, "symbol": "CDNA", "ts": "2018-08-31 10:17:00", "month": "08", "high": 23.66, "low": 23.59, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.66, "open": 23.59, "day": "31"} +{"volume": 4050, "symbol": "CDNA", "ts": "2018-08-31 10:18:00", "month": "08", "high": 23.7218, "low": 23.7218, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.7218, "open": 23.7218, "day": "31"} +{"volume": 7411, "symbol": "CDNA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 23.76, "low": 23.76, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.76, "open": 23.76, "day": "31"} +{"volume": 2636, "symbol": "CDNA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 23.85, "low": 23.85, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.85, "open": 23.85, "day": "31"} +{"volume": 1765, "symbol": "CDNA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 23.77, "low": 23.77, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.77, "open": 23.77, "day": "31"} +{"volume": 2498, "symbol": "CDNA", "ts": "2018-08-31 10:23:00", "month": "08", "high": 23.7226, "low": 23.7226, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.7226, "open": 23.7226, "day": "31"} +{"volume": 408, "symbol": "CDNA", "ts": "2018-08-31 10:25:00", "month": "08", "high": 23.735, "low": 23.735, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.735, "open": 23.735, "day": "31"} +{"volume": 226, "symbol": "CDNA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 23.8, "low": 23.8, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.8, "open": 23.8, "day": "31"} +{"volume": 800, "symbol": "CDNA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 23.765, "low": 23.765, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.765, "open": 23.765, "day": "31"} +{"volume": 616, "symbol": "IIN", "ts": "2018-08-31 10:02:00", "month": "08", "high": 73.6633, "low": 73.6633, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 73.6633, "open": 73.6633, "day": "31"} +{"volume": 610, "symbol": "IIN", "ts": "2018-08-31 10:04:00", "month": "08", "high": 73.625, "low": 73.625, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 73.625, "open": 73.625, "day": "31"} +{"volume": 1874, "symbol": "IIN", "ts": "2018-08-31 10:06:00", "month": "08", "high": 73.9, "low": 73.9, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 73.9, "open": 73.9, "day": "31"} +{"volume": 264, "symbol": "IIN", "ts": "2018-08-31 10:07:00", "month": "08", "high": 73.8952, "low": 73.8952, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 73.8952, "open": 73.8952, "day": "31"} +{"volume": 723, "symbol": "IIN", "ts": "2018-08-31 10:12:00", "month": "08", "high": 74.1148, "low": 74.1148, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.1148, "open": 74.1148, "day": "31"} +{"volume": 1405, "symbol": "IIN", "ts": "2018-08-31 10:14:00", "month": "08", "high": 74.1, "low": 74.1, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.1, "open": 74.1, "day": "31"} +{"volume": 125, "symbol": "IIN", "ts": "2018-08-31 10:15:00", "month": "08", "high": 74.0304, "low": 74.0304, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.0304, "open": 74.0304, "day": "31"} +{"volume": 634, "symbol": "IIN", "ts": "2018-08-31 10:17:00", "month": "08", "high": 74.15, "low": 74.15, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.15, "open": 74.15, "day": "31"} +{"volume": 682, "symbol": "IIN", "ts": "2018-08-31 10:18:00", "month": "08", "high": 74.35, "low": 74.35, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.35, "open": 74.35, "day": "31"} +{"volume": 1310, "symbol": "IIN", "ts": "2018-08-31 10:21:00", "month": "08", "high": 74.275, "low": 74.275, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.275, "open": 74.275, "day": "31"} +{"volume": 894, "symbol": "IIN", "ts": "2018-08-31 10:22:00", "month": "08", "high": 74.15, "low": 74.15, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.15, "open": 74.15, "day": "31"} +{"volume": 899, "symbol": "IIN", "ts": "2018-08-31 10:23:00", "month": "08", "high": 74.145, "low": 74.145, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.145, "open": 74.145, "day": "31"} +{"volume": 289, "symbol": "IIN", "ts": "2018-08-31 10:24:00", "month": "08", "high": 74.0, "low": 74.0, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.0, "open": 74.0, "day": "31"} +{"volume": 988, "symbol": "IIN", "ts": "2018-08-31 10:26:00", "month": "08", "high": 74.075, "low": 74.075, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.075, "open": 74.075, "day": "31"} +{"volume": 1850, "symbol": "IIN", "ts": "2018-08-31 10:27:00", "month": "08", "high": 74.36, "low": 74.36, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.36, "open": 74.36, "day": "31"} +{"volume": 520, "symbol": "IIN", "ts": "2018-08-31 10:28:00", "month": "08", "high": 74.2, "low": 74.2, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.2, "open": 74.2, "day": "31"} +{"volume": 9001, "symbol": "TPNL", "ts": "2018-08-31 10:07:00", "month": "08", "high": 3.301, "low": 3.14, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.14, "open": 3.301, "day": "31"} +{"volume": 1492, "symbol": "TPNL", "ts": "2018-08-31 10:10:00", "month": "08", "high": 3.34, "low": 3.34, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.34, "open": 3.34, "day": "31"} +{"volume": 20454, "symbol": "TPNL", "ts": "2018-08-31 10:13:00", "month": "08", "high": 3.39, "low": 3.348, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.348, "open": 3.39, "day": "31"} +{"volume": 1537, "symbol": "TPNL", "ts": "2018-08-31 10:22:00", "month": "08", "high": 3.22, "low": 3.22, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.22, "open": 3.22, "day": "31"} +{"volume": 8200, "symbol": "TPNL", "ts": "2018-08-31 10:25:00", "month": "08", "high": 3.2659, "low": 3.2659, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.2659, "open": 3.2659, "day": "31"} +{"volume": 110, "symbol": "TPNL", "ts": "2018-08-31 10:28:00", "month": "08", "high": 3.36, "low": 3.36, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 132815, "symbol": "CGC", "ts": "2018-08-31 10:01:00", "month": "08", "high": 45.77, "low": 45.56, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.77, "open": 45.56, "day": "31"} +{"volume": 116436, "symbol": "CGC", "ts": "2018-08-31 10:02:00", "month": "08", "high": 45.8, "low": 45.67, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7111, "open": 45.75, "day": "31"} +{"volume": 85073, "symbol": "CGC", "ts": "2018-08-31 10:03:00", "month": "08", "high": 45.809, "low": 45.68, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7501, "open": 45.7523, "day": "31"} +{"volume": 88191, "symbol": "CGC", "ts": "2018-08-31 10:04:00", "month": "08", "high": 45.75, "low": 45.5281, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.5281, "open": 45.73, "day": "31"} +{"volume": 51349, "symbol": "CGC", "ts": "2018-08-31 10:05:00", "month": "08", "high": 45.81, "low": 45.6, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.81, "open": 45.6, "day": "31"} +{"volume": 136968, "symbol": "CGC", "ts": "2018-08-31 10:06:00", "month": "08", "high": 45.88, "low": 45.7989, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.873, "open": 45.8299, "day": "31"} +{"volume": 129150, "symbol": "CGC", "ts": "2018-08-31 10:07:00", "month": "08", "high": 45.9799, "low": 45.8722, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.8722, "open": 45.8927, "day": "31"} +{"volume": 81988, "symbol": "CGC", "ts": "2018-08-31 10:08:00", "month": "08", "high": 45.91, "low": 45.71, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.77, "open": 45.8299, "day": "31"} +{"volume": 103694, "symbol": "CGC", "ts": "2018-08-31 10:09:00", "month": "08", "high": 45.7399, "low": 45.5, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7315, "open": 45.7399, "day": "31"} +{"volume": 55039, "symbol": "CGC", "ts": "2018-08-31 10:10:00", "month": "08", "high": 45.93, "low": 45.7297, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.8182, "open": 45.7297, "day": "31"} +{"volume": 48564, "symbol": "CGC", "ts": "2018-08-31 10:11:00", "month": "08", "high": 45.838, "low": 45.725, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7721, "open": 45.838, "day": "31"} +{"volume": 42783, "symbol": "CGC", "ts": "2018-08-31 10:12:00", "month": "08", "high": 45.77, "low": 45.66, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7097, "open": 45.7, "day": "31"} +{"volume": 38151, "symbol": "CGC", "ts": "2018-08-31 10:13:00", "month": "08", "high": 45.78, "low": 45.69, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.78, "open": 45.75, "day": "31"} +{"volume": 71381, "symbol": "CGC", "ts": "2018-08-31 10:14:00", "month": "08", "high": 45.93, "low": 45.7537, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.85, "open": 45.7537, "day": "31"} +{"volume": 67614, "symbol": "CGC", "ts": "2018-08-31 10:15:00", "month": "08", "high": 45.9654, "low": 45.88, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.9654, "open": 45.919, "day": "31"} +{"volume": 53706, "symbol": "CGC", "ts": "2018-08-31 10:16:00", "month": "08", "high": 45.9028, "low": 45.75, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.8098, "open": 45.9028, "day": "31"} +{"volume": 18571, "symbol": "CGC", "ts": "2018-08-31 10:17:00", "month": "08", "high": 45.7552, "low": 45.71, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7399, "open": 45.7552, "day": "31"} +{"volume": 16316, "symbol": "CGC", "ts": "2018-08-31 10:18:00", "month": "08", "high": 45.8799, "low": 45.8, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.8386, "open": 45.82, "day": "31"} +{"volume": 32142, "symbol": "CGC", "ts": "2018-08-31 10:19:00", "month": "08", "high": 45.8464, "low": 45.71, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.8464, "open": 45.81, "day": "31"} +{"volume": 173333, "symbol": "CGC", "ts": "2018-08-31 10:20:00", "month": "08", "high": 46.0173, "low": 45.89, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.985, "open": 45.89, "day": "31"} +{"volume": 138121, "symbol": "CGC", "ts": "2018-08-31 10:21:00", "month": "08", "high": 46.12, "low": 46.01, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 46.03, "open": 46.01, "day": "31"} +{"volume": 59248, "symbol": "CGC", "ts": "2018-08-31 10:22:00", "month": "08", "high": 46.1, "low": 45.9631, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 46.05, "open": 46.04, "day": "31"} +{"volume": 75864, "symbol": "CGC", "ts": "2018-08-31 10:23:00", "month": "08", "high": 46.14, "low": 46.0, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 46.0699, "open": 46.1, "day": "31"} +{"volume": 60214, "symbol": "CGC", "ts": "2018-08-31 10:24:00", "month": "08", "high": 46.0827, "low": 45.93, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 46.0827, "open": 45.99, "day": "31"} +{"volume": 147310, "symbol": "CGC", "ts": "2018-08-31 10:25:00", "month": "08", "high": 46.03, "low": 45.74, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.74, "open": 46.03, "day": "31"} +{"volume": 96464, "symbol": "CGC", "ts": "2018-08-31 10:26:00", "month": "08", "high": 45.785, "low": 45.62, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.62, "open": 45.745, "day": "31"} +{"volume": 61298, "symbol": "CGC", "ts": "2018-08-31 10:27:00", "month": "08", "high": 45.72, "low": 45.54, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.555, "open": 45.6361, "day": "31"} +{"volume": 48254, "symbol": "CGC", "ts": "2018-08-31 10:28:00", "month": "08", "high": 45.69, "low": 45.5643, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.62, "open": 45.5643, "day": "31"} +{"volume": 43375, "symbol": "CGC", "ts": "2018-08-31 10:29:00", "month": "08", "high": 45.75, "low": 45.7, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.731, "open": 45.73, "day": "31"} +{"volume": 1208, "symbol": "FNKO", "ts": "2018-08-31 10:02:00", "month": "08", "high": 24.69, "low": 24.69, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.69, "open": 24.69, "day": "31"} +{"volume": 2085, "symbol": "FNKO", "ts": "2018-08-31 10:05:00", "month": "08", "high": 24.682, "low": 24.682, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.682, "open": 24.682, "day": "31"} +{"volume": 2850, "symbol": "FNKO", "ts": "2018-08-31 10:06:00", "month": "08", "high": 24.6311, "low": 24.6311, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.6311, "open": 24.6311, "day": "31"} +{"volume": 1844, "symbol": "FNKO", "ts": "2018-08-31 10:08:00", "month": "08", "high": 24.621, "low": 24.621, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.621, "open": 24.621, "day": "31"} +{"volume": 1558, "symbol": "FNKO", "ts": "2018-08-31 10:11:00", "month": "08", "high": 24.6388, "low": 24.6388, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.6388, "open": 24.6388, "day": "31"} +{"volume": 1991, "symbol": "FNKO", "ts": "2018-08-31 10:12:00", "month": "08", "high": 24.635, "low": 24.635, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.635, "open": 24.635, "day": "31"} +{"volume": 2532, "symbol": "FNKO", "ts": "2018-08-31 10:14:00", "month": "08", "high": 24.6469, "low": 24.6469, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.6469, "open": 24.6469, "day": "31"} +{"volume": 2760, "symbol": "FNKO", "ts": "2018-08-31 10:16:00", "month": "08", "high": 24.8036, "low": 24.8036, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8036, "open": 24.8036, "day": "31"} +{"volume": 1558, "symbol": "FNKO", "ts": "2018-08-31 10:17:00", "month": "08", "high": 24.8, "low": 24.8, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8, "open": 24.8, "day": "31"} +{"volume": 2346, "symbol": "FNKO", "ts": "2018-08-31 10:18:00", "month": "08", "high": 24.84, "low": 24.84, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.84, "open": 24.84, "day": "31"} +{"volume": 2114, "symbol": "FNKO", "ts": "2018-08-31 10:20:00", "month": "08", "high": 24.93, "low": 24.93, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.93, "open": 24.93, "day": "31"} +{"volume": 4157, "symbol": "FNKO", "ts": "2018-08-31 10:22:00", "month": "08", "high": 24.94, "low": 24.94, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.94, "open": 24.94, "day": "31"} +{"volume": 6987, "symbol": "FNKO", "ts": "2018-08-31 10:23:00", "month": "08", "high": 25.0, "low": 25.0, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0, "open": 25.0, "day": "31"} +{"volume": 1521, "symbol": "FNKO", "ts": "2018-08-31 10:24:00", "month": "08", "high": 25.02, "low": 25.02, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.02, "open": 25.02, "day": "31"} +{"volume": 6113, "symbol": "FNKO", "ts": "2018-08-31 10:25:00", "month": "08", "high": 25.1, "low": 25.0737, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0737, "open": 25.1, "day": "31"} +{"volume": 3240, "symbol": "FNKO", "ts": "2018-08-31 10:27:00", "month": "08", "high": 25.0877, "low": 25.0877, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0877, "open": 25.0877, "day": "31"} +{"volume": 877, "symbol": "FNKO", "ts": "2018-08-31 10:28:00", "month": "08", "high": 25.032, "low": 25.032, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.032, "open": 25.032, "day": "31"} +{"volume": 1585, "symbol": "XENE", "ts": "2018-08-31 10:01:00", "month": "08", "high": 13.35, "low": 13.35, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.35, "open": 13.35, "day": "31"} +{"volume": 901, "symbol": "XENE", "ts": "2018-08-31 10:02:00", "month": "08", "high": 13.4, "low": 13.4, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 208, "symbol": "XENE", "ts": "2018-08-31 10:04:00", "month": "08", "high": 13.45, "low": 13.45, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.45, "open": 13.45, "day": "31"} +{"volume": 651, "symbol": "XENE", "ts": "2018-08-31 10:07:00", "month": "08", "high": 13.4, "low": 13.4, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 718, "symbol": "XENE", "ts": "2018-08-31 10:08:00", "month": "08", "high": 13.349, "low": 13.349, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.349, "open": 13.349, "day": "31"} +{"volume": 883, "symbol": "XENE", "ts": "2018-08-31 10:12:00", "month": "08", "high": 13.295, "low": 13.295, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.295, "open": 13.295, "day": "31"} +{"volume": 1317, "symbol": "XENE", "ts": "2018-08-31 10:17:00", "month": "08", "high": 13.35, "low": 13.35, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.35, "open": 13.35, "day": "31"} +{"volume": 262, "symbol": "XENE", "ts": "2018-08-31 10:27:00", "month": "08", "high": 13.3, "low": 13.3, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.3, "open": 13.3, "day": "31"} +{"volume": 886, "symbol": "SSTI", "ts": "2018-08-31 10:02:00", "month": "08", "high": 55.3299, "low": 55.3299, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.3299, "open": 55.3299, "day": "31"} +{"volume": 4008, "symbol": "SSTI", "ts": "2018-08-31 10:03:00", "month": "08", "high": 55.8246, "low": 55.8246, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.8246, "open": 55.8246, "day": "31"} +{"volume": 3109, "symbol": "SSTI", "ts": "2018-08-31 10:05:00", "month": "08", "high": 55.7986, "low": 55.7986, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.7986, "open": 55.7986, "day": "31"} +{"volume": 720, "symbol": "SSTI", "ts": "2018-08-31 10:07:00", "month": "08", "high": 55.73, "low": 55.73, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.73, "open": 55.73, "day": "31"} +{"volume": 1341, "symbol": "SSTI", "ts": "2018-08-31 10:09:00", "month": "08", "high": 55.47, "low": 55.47, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.47, "open": 55.47, "day": "31"} +{"volume": 4048, "symbol": "SSTI", "ts": "2018-08-31 10:10:00", "month": "08", "high": 55.73, "low": 55.69, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.73, "open": 55.69, "day": "31"} +{"volume": 12746, "symbol": "SSTI", "ts": "2018-08-31 10:11:00", "month": "08", "high": 55.69, "low": 55.455, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.455, "open": 55.69, "day": "31"} +{"volume": 135, "symbol": "SSTI", "ts": "2018-08-31 10:14:00", "month": "08", "high": 55.34, "low": 55.34, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.34, "open": 55.34, "day": "31"} +{"volume": 327, "symbol": "SSTI", "ts": "2018-08-31 10:17:00", "month": "08", "high": 55.5464, "low": 55.5464, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.5464, "open": 55.5464, "day": "31"} +{"volume": 100, "symbol": "SSTI", "ts": "2018-08-31 10:19:00", "month": "08", "high": 55.67, "low": 55.67, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.67, "open": 55.67, "day": "31"} +{"volume": 4522, "symbol": "SSTI", "ts": "2018-08-31 10:20:00", "month": "08", "high": 55.83, "low": 55.83, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.83, "open": 55.83, "day": "31"} +{"volume": 304, "symbol": "SSTI", "ts": "2018-08-31 10:25:00", "month": "08", "high": 55.8084, "low": 55.8084, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.8084, "open": 55.8084, "day": "31"} +{"volume": 119, "symbol": "SSTI", "ts": "2018-08-31 10:26:00", "month": "08", "high": 55.86, "low": 55.86, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.86, "open": 55.86, "day": "31"} +{"volume": 660, "symbol": "SSTI", "ts": "2018-08-31 10:28:00", "month": "08", "high": 55.7407, "low": 55.7407, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.7407, "open": 55.7407, "day": "31"} +{"volume": 9154, "symbol": "VRS", "ts": "2018-08-31 10:01:00", "month": "08", "high": 31.19, "low": 31.12, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1365, "open": 31.12, "day": "31"} +{"volume": 4271, "symbol": "VRS", "ts": "2018-08-31 10:02:00", "month": "08", "high": 31.05, "low": 31.05, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.05, "open": 31.05, "day": "31"} +{"volume": 5120, "symbol": "VRS", "ts": "2018-08-31 10:03:00", "month": "08", "high": 31.2, "low": 31.2, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.2, "open": 31.2, "day": "31"} +{"volume": 25539, "symbol": "VRS", "ts": "2018-08-31 10:04:00", "month": "08", "high": 31.23, "low": 31.22, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.23, "open": 31.22, "day": "31"} +{"volume": 9876, "symbol": "VRS", "ts": "2018-08-31 10:05:00", "month": "08", "high": 31.16, "low": 31.1239, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.16, "open": 31.1239, "day": "31"} +{"volume": 3776, "symbol": "VRS", "ts": "2018-08-31 10:06:00", "month": "08", "high": 31.11, "low": 31.11, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.11, "open": 31.11, "day": "31"} +{"volume": 7432, "symbol": "VRS", "ts": "2018-08-31 10:07:00", "month": "08", "high": 31.215, "low": 31.2, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.215, "open": 31.2, "day": "31"} +{"volume": 9760, "symbol": "VRS", "ts": "2018-08-31 10:08:00", "month": "08", "high": 31.214, "low": 31.164, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.164, "open": 31.214, "day": "31"} +{"volume": 5862, "symbol": "VRS", "ts": "2018-08-31 10:09:00", "month": "08", "high": 31.24, "low": 31.1999, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.24, "open": 31.1999, "day": "31"} +{"volume": 3528, "symbol": "VRS", "ts": "2018-08-31 10:10:00", "month": "08", "high": 31.25, "low": 31.25, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.25, "open": 31.25, "day": "31"} +{"volume": 5060, "symbol": "VRS", "ts": "2018-08-31 10:11:00", "month": "08", "high": 31.19, "low": 31.19, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.19, "open": 31.19, "day": "31"} +{"volume": 6825, "symbol": "VRS", "ts": "2018-08-31 10:12:00", "month": "08", "high": 31.21, "low": 31.2, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.2, "open": 31.21, "day": "31"} +{"volume": 3278, "symbol": "VRS", "ts": "2018-08-31 10:13:00", "month": "08", "high": 31.2, "low": 31.2, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.2, "open": 31.2, "day": "31"} +{"volume": 3210, "symbol": "VRS", "ts": "2018-08-31 10:15:00", "month": "08", "high": 31.23, "low": 31.23, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.23, "open": 31.23, "day": "31"} +{"volume": 6585, "symbol": "VRS", "ts": "2018-08-31 10:16:00", "month": "08", "high": 31.18, "low": 31.175, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.175, "open": 31.18, "day": "31"} +{"volume": 2921, "symbol": "VRS", "ts": "2018-08-31 10:17:00", "month": "08", "high": 31.15, "low": 31.15, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.15, "open": 31.15, "day": "31"} +{"volume": 5974, "symbol": "VRS", "ts": "2018-08-31 10:18:00", "month": "08", "high": 31.12, "low": 31.12, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.12, "open": 31.12, "day": "31"} +{"volume": 1497, "symbol": "VRS", "ts": "2018-08-31 10:19:00", "month": "08", "high": 31.1525, "low": 31.1525, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1525, "open": 31.1525, "day": "31"} +{"volume": 2448, "symbol": "VRS", "ts": "2018-08-31 10:20:00", "month": "08", "high": 31.1421, "low": 31.1421, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1421, "open": 31.1421, "day": "31"} +{"volume": 1810, "symbol": "VRS", "ts": "2018-08-31 10:21:00", "month": "08", "high": 31.17, "low": 31.17, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.17, "open": 31.17, "day": "31"} +{"volume": 3615, "symbol": "VRS", "ts": "2018-08-31 10:22:00", "month": "08", "high": 31.17, "low": 31.17, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.17, "open": 31.17, "day": "31"} +{"volume": 2591, "symbol": "VRS", "ts": "2018-08-31 10:23:00", "month": "08", "high": 31.1499, "low": 31.1499, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1499, "open": 31.1499, "day": "31"} +{"volume": 1718, "symbol": "VRS", "ts": "2018-08-31 10:24:00", "month": "08", "high": 31.17, "low": 31.17, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.17, "open": 31.17, "day": "31"} +{"volume": 2382, "symbol": "VRS", "ts": "2018-08-31 10:25:00", "month": "08", "high": 31.15, "low": 31.15, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.15, "open": 31.15, "day": "31"} +{"volume": 1956, "symbol": "VRS", "ts": "2018-08-31 10:26:00", "month": "08", "high": 31.15, "low": 31.15, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.15, "open": 31.15, "day": "31"} +{"volume": 3361, "symbol": "VRS", "ts": "2018-08-31 10:27:00", "month": "08", "high": 31.1555, "low": 31.1555, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1555, "open": 31.1555, "day": "31"} +{"volume": 4285, "symbol": "VRS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 31.15, "low": 31.12, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.12, "open": 31.15, "day": "31"} +{"volume": 5045, "symbol": "VRS", "ts": "2018-08-31 10:29:00", "month": "08", "high": 31.155, "low": 31.15, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.155, "open": 31.15, "day": "31"} +{"volume": 1970, "symbol": "ARWR", "ts": "2018-08-31 10:01:00", "month": "08", "high": 14.53, "low": 14.53, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.53, "open": 14.53, "day": "31"} +{"volume": 1309, "symbol": "ARWR", "ts": "2018-08-31 10:02:00", "month": "08", "high": 14.535, "low": 14.535, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.535, "open": 14.535, "day": "31"} +{"volume": 1648, "symbol": "ARWR", "ts": "2018-08-31 10:03:00", "month": "08", "high": 14.54, "low": 14.54, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.54, "open": 14.54, "day": "31"} +{"volume": 401, "symbol": "ARWR", "ts": "2018-08-31 10:04:00", "month": "08", "high": 14.54, "low": 14.54, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.54, "open": 14.54, "day": "31"} +{"volume": 1690, "symbol": "ARWR", "ts": "2018-08-31 10:05:00", "month": "08", "high": 14.53, "low": 14.53, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.53, "open": 14.53, "day": "31"} +{"volume": 8312, "symbol": "ARWR", "ts": "2018-08-31 10:06:00", "month": "08", "high": 14.465, "low": 14.45, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.465, "open": 14.45, "day": "31"} +{"volume": 4111, "symbol": "ARWR", "ts": "2018-08-31 10:07:00", "month": "08", "high": 14.48, "low": 14.48, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.48, "open": 14.48, "day": "31"} +{"volume": 2410, "symbol": "ARWR", "ts": "2018-08-31 10:08:00", "month": "08", "high": 14.44, "low": 14.44, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.44, "open": 14.44, "day": "31"} +{"volume": 721, "symbol": "ARWR", "ts": "2018-08-31 10:10:00", "month": "08", "high": 14.48, "low": 14.48, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.48, "open": 14.48, "day": "31"} +{"volume": 14958, "symbol": "ARWR", "ts": "2018-08-31 10:11:00", "month": "08", "high": 14.47, "low": 14.41, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.43, "open": 14.47, "day": "31"} +{"volume": 977, "symbol": "ARWR", "ts": "2018-08-31 10:13:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 6404, "symbol": "ARWR", "ts": "2018-08-31 10:14:00", "month": "08", "high": 14.44, "low": 14.43, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.44, "open": 14.43, "day": "31"} +{"volume": 1725, "symbol": "ARWR", "ts": "2018-08-31 10:15:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 3364, "symbol": "ARWR", "ts": "2018-08-31 10:16:00", "month": "08", "high": 14.46, "low": 14.43, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.43, "open": 14.46, "day": "31"} +{"volume": 902, "symbol": "ARWR", "ts": "2018-08-31 10:18:00", "month": "08", "high": 14.467, "low": 14.467, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.467, "open": 14.467, "day": "31"} +{"volume": 2462, "symbol": "ARWR", "ts": "2018-08-31 10:20:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 222, "symbol": "ARWR", "ts": "2018-08-31 10:21:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 2938, "symbol": "ARWR", "ts": "2018-08-31 10:23:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 607, "symbol": "ARWR", "ts": "2018-08-31 10:24:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 2201, "symbol": "ARWR", "ts": "2018-08-31 10:26:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 4359, "symbol": "ARWR", "ts": "2018-08-31 10:27:00", "month": "08", "high": 14.41, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.41, "open": 14.4, "day": "31"} +{"volume": 4919, "symbol": "ARWR", "ts": "2018-08-31 10:28:00", "month": "08", "high": 14.42, "low": 14.42, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.42, "open": 14.42, "day": "31"} +{"volume": 3950, "symbol": "ARWR", "ts": "2018-08-31 10:29:00", "month": "08", "high": 14.44, "low": 14.42, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.44, "open": 14.42, "day": "31"} +{"volume": 420, "symbol": "MRTX", "ts": "2018-08-31 10:02:00", "month": "08", "high": 56.6, "low": 56.6, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.6, "open": 56.6, "day": "31"} +{"volume": 308, "symbol": "MRTX", "ts": "2018-08-31 10:03:00", "month": "08", "high": 56.6, "low": 56.6, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.6, "open": 56.6, "day": "31"} +{"volume": 504, "symbol": "MRTX", "ts": "2018-08-31 10:06:00", "month": "08", "high": 56.45, "low": 56.45, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.45, "open": 56.45, "day": "31"} +{"volume": 1333, "symbol": "MRTX", "ts": "2018-08-31 10:08:00", "month": "08", "high": 56.3, "low": 56.3, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.3, "open": 56.3, "day": "31"} +{"volume": 1163, "symbol": "MRTX", "ts": "2018-08-31 10:09:00", "month": "08", "high": 56.3, "low": 56.3, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.3, "open": 56.3, "day": "31"} +{"volume": 336, "symbol": "MRTX", "ts": "2018-08-31 10:10:00", "month": "08", "high": 56.25, "low": 56.25, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.25, "open": 56.25, "day": "31"} +{"volume": 626, "symbol": "MRTX", "ts": "2018-08-31 10:13:00", "month": "08", "high": 56.1, "low": 56.1, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.1, "open": 56.1, "day": "31"} +{"volume": 1000, "symbol": "MRTX", "ts": "2018-08-31 10:16:00", "month": "08", "high": 56.25, "low": 56.25, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.25, "open": 56.25, "day": "31"} +{"volume": 1469, "symbol": "MRTX", "ts": "2018-08-31 10:17:00", "month": "08", "high": 56.1, "low": 56.1, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.1, "open": 56.1, "day": "31"} +{"volume": 1498, "symbol": "MRTX", "ts": "2018-08-31 10:18:00", "month": "08", "high": 56.15, "low": 56.15, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.15, "open": 56.15, "day": "31"} +{"volume": 239, "symbol": "MRTX", "ts": "2018-08-31 10:20:00", "month": "08", "high": 56.2, "low": 56.2, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.2, "open": 56.2, "day": "31"} +{"volume": 1234, "symbol": "MRTX", "ts": "2018-08-31 10:22:00", "month": "08", "high": 56.3, "low": 56.3, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.3, "open": 56.3, "day": "31"} +{"volume": 606, "symbol": "MRTX", "ts": "2018-08-31 10:24:00", "month": "08", "high": 56.4, "low": 56.4, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.4, "open": 56.4, "day": "31"} +{"volume": 326, "symbol": "MRTX", "ts": "2018-08-31 10:25:00", "month": "08", "high": 56.3, "low": 56.3, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.3, "open": 56.3, "day": "31"} +{"volume": 319, "symbol": "MRTX", "ts": "2018-08-31 10:27:00", "month": "08", "high": 56.35, "low": 56.35, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.35, "open": 56.35, "day": "31"} +{"volume": 1844, "symbol": "MRTX", "ts": "2018-08-31 10:28:00", "month": "08", "high": 56.15, "low": 56.15, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.15, "open": 56.15, "day": "31"} +{"volume": 1226, "symbol": "WWE", "ts": "2018-08-31 10:01:00", "month": "08", "high": 86.42, "low": 86.42, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.42, "open": 86.42, "day": "31"} +{"volume": 1303, "symbol": "WWE", "ts": "2018-08-31 10:02:00", "month": "08", "high": 86.4458, "low": 86.4458, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.4458, "open": 86.4458, "day": "31"} +{"volume": 559, "symbol": "WWE", "ts": "2018-08-31 10:03:00", "month": "08", "high": 86.36, "low": 86.36, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.36, "open": 86.36, "day": "31"} +{"volume": 2154, "symbol": "WWE", "ts": "2018-08-31 10:05:00", "month": "08", "high": 86.37, "low": 86.37, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.37, "open": 86.37, "day": "31"} +{"volume": 1728, "symbol": "WWE", "ts": "2018-08-31 10:06:00", "month": "08", "high": 86.41, "low": 86.41, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.41, "open": 86.41, "day": "31"} +{"volume": 557, "symbol": "WWE", "ts": "2018-08-31 10:07:00", "month": "08", "high": 86.29, "low": 86.29, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.29, "open": 86.29, "day": "31"} +{"volume": 2368, "symbol": "WWE", "ts": "2018-08-31 10:09:00", "month": "08", "high": 86.25, "low": 86.25, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.25, "open": 86.25, "day": "31"} +{"volume": 531, "symbol": "WWE", "ts": "2018-08-31 10:10:00", "month": "08", "high": 86.2244, "low": 86.2244, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.2244, "open": 86.2244, "day": "31"} +{"volume": 1023, "symbol": "WWE", "ts": "2018-08-31 10:13:00", "month": "08", "high": 86.107, "low": 86.107, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.107, "open": 86.107, "day": "31"} +{"volume": 1790, "symbol": "WWE", "ts": "2018-08-31 10:14:00", "month": "08", "high": 86.36, "low": 86.36, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.36, "open": 86.36, "day": "31"} +{"volume": 2249, "symbol": "WWE", "ts": "2018-08-31 10:16:00", "month": "08", "high": 86.5125, "low": 86.44, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.5125, "open": 86.44, "day": "31"} +{"volume": 419, "symbol": "WWE", "ts": "2018-08-31 10:18:00", "month": "08", "high": 86.475, "low": 86.475, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.475, "open": 86.475, "day": "31"} +{"volume": 1137, "symbol": "WWE", "ts": "2018-08-31 10:21:00", "month": "08", "high": 86.42, "low": 86.42, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.42, "open": 86.42, "day": "31"} +{"volume": 322, "symbol": "WWE", "ts": "2018-08-31 10:22:00", "month": "08", "high": 86.485, "low": 86.485, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.485, "open": 86.485, "day": "31"} +{"volume": 718, "symbol": "WWE", "ts": "2018-08-31 10:24:00", "month": "08", "high": 86.495, "low": 86.495, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.495, "open": 86.495, "day": "31"} +{"volume": 772, "symbol": "WWE", "ts": "2018-08-31 10:26:00", "month": "08", "high": 86.44, "low": 86.44, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.44, "open": 86.44, "day": "31"} +{"volume": 1543, "symbol": "WWE", "ts": "2018-08-31 10:27:00", "month": "08", "high": 86.3265, "low": 86.3265, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.3265, "open": 86.3265, "day": "31"} +{"volume": 1356, "symbol": "WWE", "ts": "2018-08-31 10:28:00", "month": "08", "high": 86.4806, "low": 86.4806, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.4806, "open": 86.4806, "day": "31"} +{"volume": 119, "symbol": "RFIL", "ts": "2018-08-31 10:04:00", "month": "08", "high": 11.8, "low": 11.8, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.8, "open": 11.8, "day": "31"} +{"volume": 300, "symbol": "RFIL", "ts": "2018-08-31 10:06:00", "month": "08", "high": 11.793, "low": 11.7, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.7, "open": 11.793, "day": "31"} +{"volume": 2793, "symbol": "RFIL", "ts": "2018-08-31 10:14:00", "month": "08", "high": 11.85, "low": 11.7, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.7, "open": 11.85, "day": "31"} +{"volume": 117, "symbol": "RFIL", "ts": "2018-08-31 10:16:00", "month": "08", "high": 11.7, "low": 11.7, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.7, "open": 11.7, "day": "31"} +{"volume": 250, "symbol": "RFIL", "ts": "2018-08-31 10:18:00", "month": "08", "high": 11.7365, "low": 11.7365, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.7365, "open": 11.7365, "day": "31"} +{"volume": 1432, "symbol": "RFIL", "ts": "2018-08-31 10:19:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 120, "symbol": "RFIL", "ts": "2018-08-31 10:25:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 389, "symbol": "RFIL", "ts": "2018-08-31 10:26:00", "month": "08", "high": 11.65, "low": 11.65, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.65, "open": 11.65, "day": "31"} +{"volume": 185, "symbol": "MED", "ts": "2018-08-31 10:01:00", "month": "08", "high": 226.57, "low": 226.57, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 226.57, "open": 226.57, "day": "31"} +{"volume": 342, "symbol": "MED", "ts": "2018-08-31 10:03:00", "month": "08", "high": 226.5024, "low": 226.5024, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 226.5024, "open": 226.5024, "day": "31"} +{"volume": 791, "symbol": "MED", "ts": "2018-08-31 10:05:00", "month": "08", "high": 226.57, "low": 226.57, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 226.57, "open": 226.57, "day": "31"} +{"volume": 1164, "symbol": "MED", "ts": "2018-08-31 10:06:00", "month": "08", "high": 227.02, "low": 227.02, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.02, "open": 227.02, "day": "31"} +{"volume": 1021, "symbol": "MED", "ts": "2018-08-31 10:08:00", "month": "08", "high": 227.95, "low": 227.95, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.95, "open": 227.95, "day": "31"} +{"volume": 386, "symbol": "MED", "ts": "2018-08-31 10:11:00", "month": "08", "high": 227.54, "low": 227.54, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.54, "open": 227.54, "day": "31"} +{"volume": 633, "symbol": "MED", "ts": "2018-08-31 10:13:00", "month": "08", "high": 227.9128, "low": 227.9128, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.9128, "open": 227.9128, "day": "31"} +{"volume": 231, "symbol": "MED", "ts": "2018-08-31 10:16:00", "month": "08", "high": 227.658, "low": 227.658, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.658, "open": 227.658, "day": "31"} +{"volume": 515, "symbol": "MED", "ts": "2018-08-31 10:17:00", "month": "08", "high": 227.54, "low": 227.54, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.54, "open": 227.54, "day": "31"} +{"volume": 165, "symbol": "MED", "ts": "2018-08-31 10:22:00", "month": "08", "high": 227.44, "low": 227.44, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.44, "open": 227.44, "day": "31"} +{"volume": 210, "symbol": "MED", "ts": "2018-08-31 10:27:00", "month": "08", "high": 227.89, "low": 227.89, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.89, "open": 227.89, "day": "31"} +{"volume": 1489, "symbol": "STAA", "ts": "2018-08-31 10:02:00", "month": "08", "high": 47.9917, "low": 47.9917, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.9917, "open": 47.9917, "day": "31"} +{"volume": 751, "symbol": "STAA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 47.95, "low": 47.95, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.95, "open": 47.95, "day": "31"} +{"volume": 1136, "symbol": "STAA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 48.0478, "low": 48.0478, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.0478, "open": 48.0478, "day": "31"} +{"volume": 1010, "symbol": "STAA", "ts": "2018-08-31 10:07:00", "month": "08", "high": 47.99, "low": 47.99, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.99, "open": 47.99, "day": "31"} +{"volume": 2036, "symbol": "STAA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 47.8, "low": 47.8, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.8, "open": 47.8, "day": "31"} +{"volume": 4279, "symbol": "STAA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 47.655, "low": 47.655, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.655, "open": 47.655, "day": "31"} +{"volume": 3641, "symbol": "STAA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 47.65, "low": 47.6, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.65, "open": 47.6, "day": "31"} +{"volume": 720, "symbol": "STAA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 47.65, "low": 47.65, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.65, "open": 47.65, "day": "31"} +{"volume": 1374, "symbol": "STAA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 47.7, "low": 47.6623, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.6623, "open": 47.7, "day": "31"} +{"volume": 1018, "symbol": "STAA", "ts": "2018-08-31 10:23:00", "month": "08", "high": 47.6, "low": 47.6, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.6, "open": 47.6, "day": "31"} +{"volume": 901, "symbol": "STAA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 47.65, "low": 47.65, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.65, "open": 47.65, "day": "31"} +{"volume": 2597, "symbol": "STAA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 800, "symbol": "STAA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 111, "symbol": "TRHC", "ts": "2018-08-31 10:01:00", "month": "08", "high": 86.88, "low": 86.88, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.88, "open": 86.88, "day": "31"} +{"volume": 1230, "symbol": "TRHC", "ts": "2018-08-31 10:03:00", "month": "08", "high": 86.88, "low": 86.88, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.88, "open": 86.88, "day": "31"} +{"volume": 1805, "symbol": "TRHC", "ts": "2018-08-31 10:05:00", "month": "08", "high": 86.84, "low": 86.84, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.84, "open": 86.84, "day": "31"} +{"volume": 220, "symbol": "TRHC", "ts": "2018-08-31 10:07:00", "month": "08", "high": 86.62, "low": 86.62, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.62, "open": 86.62, "day": "31"} +{"volume": 152, "symbol": "TRHC", "ts": "2018-08-31 10:10:00", "month": "08", "high": 86.88, "low": 86.88, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.88, "open": 86.88, "day": "31"} +{"volume": 416, "symbol": "TRHC", "ts": "2018-08-31 10:12:00", "month": "08", "high": 86.85, "low": 86.85, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.85, "open": 86.85, "day": "31"} +{"volume": 763, "symbol": "TRHC", "ts": "2018-08-31 10:14:00", "month": "08", "high": 86.6, "low": 86.6, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.6, "open": 86.6, "day": "31"} +{"volume": 119, "symbol": "TRHC", "ts": "2018-08-31 10:17:00", "month": "08", "high": 86.68, "low": 86.68, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.68, "open": 86.68, "day": "31"} +{"volume": 300, "symbol": "TRHC", "ts": "2018-08-31 10:20:00", "month": "08", "high": 86.68, "low": 86.68, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.68, "open": 86.68, "day": "31"} +{"volume": 335, "symbol": "TRHC", "ts": "2018-08-31 10:21:00", "month": "08", "high": 86.68, "low": 86.68, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.68, "open": 86.68, "day": "31"} +{"volume": 113, "symbol": "TRHC", "ts": "2018-08-31 10:22:00", "month": "08", "high": 86.68, "low": 86.68, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.68, "open": 86.68, "day": "31"} +{"volume": 150, "symbol": "TRHC", "ts": "2018-08-31 10:23:00", "month": "08", "high": 86.68, "low": 86.68, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.68, "open": 86.68, "day": "31"} +{"volume": 100, "symbol": "TRHC", "ts": "2018-08-31 10:25:00", "month": "08", "high": 86.69, "low": 86.69, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.69, "open": 86.69, "day": "31"} +{"volume": 1046, "symbol": "TRHC", "ts": "2018-08-31 10:28:00", "month": "08", "high": 86.94, "low": 86.94, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.94, "open": 86.94, "day": "31"} +{"volume": 110, "symbol": "TRHC", "ts": "2018-08-31 10:29:00", "month": "08", "high": 86.95, "low": 86.95, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.95, "open": 86.95, "day": "31"} +{"volume": 9472, "symbol": "I", "ts": "2018-08-31 10:01:00", "month": "08", "high": 22.07, "low": 21.97, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 22.07, "open": 21.97, "day": "31"} +{"volume": 8199, "symbol": "I", "ts": "2018-08-31 10:02:00", "month": "08", "high": 21.95, "low": 21.95, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.95, "open": 21.95, "day": "31"} +{"volume": 2929, "symbol": "I", "ts": "2018-08-31 10:03:00", "month": "08", "high": 21.89, "low": 21.89, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.89, "open": 21.89, "day": "31"} +{"volume": 9984, "symbol": "I", "ts": "2018-08-31 10:04:00", "month": "08", "high": 21.895, "low": 21.895, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.895, "open": 21.895, "day": "31"} +{"volume": 771, "symbol": "I", "ts": "2018-08-31 10:05:00", "month": "08", "high": 21.92, "low": 21.92, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.92, "open": 21.92, "day": "31"} +{"volume": 1150, "symbol": "I", "ts": "2018-08-31 10:07:00", "month": "08", "high": 21.93, "low": 21.93, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.93, "open": 21.93, "day": "31"} +{"volume": 1488, "symbol": "I", "ts": "2018-08-31 10:09:00", "month": "08", "high": 21.9366, "low": 21.9366, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.9366, "open": 21.9366, "day": "31"} +{"volume": 4413, "symbol": "I", "ts": "2018-08-31 10:10:00", "month": "08", "high": 21.9536, "low": 21.9536, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.9536, "open": 21.9536, "day": "31"} +{"volume": 3197, "symbol": "I", "ts": "2018-08-31 10:11:00", "month": "08", "high": 21.935, "low": 21.935, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.935, "open": 21.935, "day": "31"} +{"volume": 682, "symbol": "I", "ts": "2018-08-31 10:14:00", "month": "08", "high": 21.9334, "low": 21.9334, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.9334, "open": 21.9334, "day": "31"} +{"volume": 1300, "symbol": "I", "ts": "2018-08-31 10:15:00", "month": "08", "high": 21.965, "low": 21.965, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.965, "open": 21.965, "day": "31"} +{"volume": 674, "symbol": "I", "ts": "2018-08-31 10:16:00", "month": "08", "high": 21.96, "low": 21.96, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.96, "open": 21.96, "day": "31"} +{"volume": 5325, "symbol": "I", "ts": "2018-08-31 10:18:00", "month": "08", "high": 21.975, "low": 21.975, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.975, "open": 21.975, "day": "31"} +{"volume": 3556, "symbol": "I", "ts": "2018-08-31 10:19:00", "month": "08", "high": 21.93, "low": 21.93, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.93, "open": 21.93, "day": "31"} +{"volume": 1684, "symbol": "I", "ts": "2018-08-31 10:21:00", "month": "08", "high": 21.91, "low": 21.91, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.91, "open": 21.91, "day": "31"} +{"volume": 2026, "symbol": "I", "ts": "2018-08-31 10:22:00", "month": "08", "high": 21.92, "low": 21.92, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.92, "open": 21.92, "day": "31"} +{"volume": 1258, "symbol": "I", "ts": "2018-08-31 10:23:00", "month": "08", "high": 21.94, "low": 21.94, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.94, "open": 21.94, "day": "31"} +{"volume": 2550, "symbol": "I", "ts": "2018-08-31 10:25:00", "month": "08", "high": 21.92, "low": 21.92, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.92, "open": 21.92, "day": "31"} +{"volume": 1150, "symbol": "I", "ts": "2018-08-31 10:26:00", "month": "08", "high": 21.89, "low": 21.89, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.89, "open": 21.89, "day": "31"} +{"volume": 1642, "symbol": "I", "ts": "2018-08-31 10:27:00", "month": "08", "high": 21.86, "low": 21.86, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.86, "open": 21.86, "day": "31"} +{"volume": 2024, "symbol": "I", "ts": "2018-08-31 10:29:00", "month": "08", "high": 21.82, "low": 21.82, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.82, "open": 21.82, "day": "31"} +{"volume": 10500, "symbol": "ARQL", "ts": "2018-08-31 10:01:00", "month": "08", "high": 6.62, "low": 6.62, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.62, "open": 6.62, "day": "31"} +{"volume": 4419, "symbol": "ARQL", "ts": "2018-08-31 10:02:00", "month": "08", "high": 6.62, "low": 6.62, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.62, "open": 6.62, "day": "31"} +{"volume": 4794, "symbol": "ARQL", "ts": "2018-08-31 10:04:00", "month": "08", "high": 6.63, "low": 6.6249, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.63, "open": 6.6249, "day": "31"} +{"volume": 1226, "symbol": "ARQL", "ts": "2018-08-31 10:07:00", "month": "08", "high": 6.6336, "low": 6.6336, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6336, "open": 6.6336, "day": "31"} +{"volume": 8885, "symbol": "ARQL", "ts": "2018-08-31 10:08:00", "month": "08", "high": 6.62, "low": 6.6042, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6042, "open": 6.62, "day": "31"} +{"volume": 3141, "symbol": "ARQL", "ts": "2018-08-31 10:10:00", "month": "08", "high": 6.6, "low": 6.6, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6, "open": 6.6, "day": "31"} +{"volume": 930, "symbol": "ARQL", "ts": "2018-08-31 10:11:00", "month": "08", "high": 6.59, "low": 6.59, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.59, "open": 6.59, "day": "31"} +{"volume": 4034, "symbol": "ARQL", "ts": "2018-08-31 10:13:00", "month": "08", "high": 6.58, "low": 6.58, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.58, "open": 6.58, "day": "31"} +{"volume": 2957, "symbol": "ARQL", "ts": "2018-08-31 10:14:00", "month": "08", "high": 6.58, "low": 6.58, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.58, "open": 6.58, "day": "31"} +{"volume": 3610, "symbol": "ARQL", "ts": "2018-08-31 10:15:00", "month": "08", "high": 6.6144, "low": 6.6144, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6144, "open": 6.6144, "day": "31"} +{"volume": 8960, "symbol": "ARQL", "ts": "2018-08-31 10:17:00", "month": "08", "high": 6.6498, "low": 6.61, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6498, "open": 6.61, "day": "31"} +{"volume": 7330, "symbol": "ARQL", "ts": "2018-08-31 10:18:00", "month": "08", "high": 6.67, "low": 6.67, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.67, "open": 6.67, "day": "31"} +{"volume": 2179, "symbol": "ARQL", "ts": "2018-08-31 10:19:00", "month": "08", "high": 6.68, "low": 6.68, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.68, "open": 6.68, "day": "31"} +{"volume": 5214, "symbol": "ARQL", "ts": "2018-08-31 10:21:00", "month": "08", "high": 6.68, "low": 6.66, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.66, "open": 6.68, "day": "31"} +{"volume": 1960, "symbol": "ARQL", "ts": "2018-08-31 10:22:00", "month": "08", "high": 6.67, "low": 6.67, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.67, "open": 6.67, "day": "31"} +{"volume": 702, "symbol": "ARQL", "ts": "2018-08-31 10:23:00", "month": "08", "high": 6.67, "low": 6.67, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.67, "open": 6.67, "day": "31"} +{"volume": 111, "symbol": "ARQL", "ts": "2018-08-31 10:25:00", "month": "08", "high": 6.67, "low": 6.67, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.67, "open": 6.67, "day": "31"} +{"volume": 4118, "symbol": "ARQL", "ts": "2018-08-31 10:27:00", "month": "08", "high": 6.68, "low": 6.6799, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6799, "open": 6.68, "day": "31"} +{"volume": 83668, "symbol": "DNR", "ts": "2018-08-31 10:01:00", "month": "08", "high": 5.4542, "low": 5.45, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.45, "open": 5.45, "day": "31"} +{"volume": 33221, "symbol": "DNR", "ts": "2018-08-31 10:02:00", "month": "08", "high": 5.44, "low": 5.4362, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.4362, "open": 5.44, "day": "31"} +{"volume": 10950, "symbol": "DNR", "ts": "2018-08-31 10:03:00", "month": "08", "high": 5.4352, "low": 5.4352, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.4352, "open": 5.4352, "day": "31"} +{"volume": 14016, "symbol": "DNR", "ts": "2018-08-31 10:04:00", "month": "08", "high": 5.4354, "low": 5.43, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.43, "open": 5.4354, "day": "31"} +{"volume": 104924, "symbol": "DNR", "ts": "2018-08-31 10:05:00", "month": "08", "high": 5.44, "low": 5.415, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.415, "open": 5.44, "day": "31"} +{"volume": 188679, "symbol": "DNR", "ts": "2018-08-31 10:06:00", "month": "08", "high": 5.45, "low": 5.405, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.45, "open": 5.405, "day": "31"} +{"volume": 22769, "symbol": "DNR", "ts": "2018-08-31 10:07:00", "month": "08", "high": 5.445, "low": 5.44, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.44, "open": 5.445, "day": "31"} +{"volume": 34808, "symbol": "DNR", "ts": "2018-08-31 10:08:00", "month": "08", "high": 5.4441, "low": 5.435, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.435, "open": 5.4441, "day": "31"} +{"volume": 42461, "symbol": "DNR", "ts": "2018-08-31 10:09:00", "month": "08", "high": 5.44, "low": 5.43, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.44, "open": 5.43, "day": "31"} +{"volume": 65095, "symbol": "DNR", "ts": "2018-08-31 10:10:00", "month": "08", "high": 5.435, "low": 5.43, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.43, "open": 5.435, "day": "31"} +{"volume": 50789, "symbol": "DNR", "ts": "2018-08-31 10:11:00", "month": "08", "high": 5.43, "low": 5.425, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.43, "open": 5.425, "day": "31"} +{"volume": 35805, "symbol": "DNR", "ts": "2018-08-31 10:12:00", "month": "08", "high": 5.425, "low": 5.425, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.425, "open": 5.425, "day": "31"} +{"volume": 65719, "symbol": "DNR", "ts": "2018-08-31 10:13:00", "month": "08", "high": 5.435, "low": 5.42, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.435, "open": 5.42, "day": "31"} +{"volume": 11179, "symbol": "DNR", "ts": "2018-08-31 10:14:00", "month": "08", "high": 5.435, "low": 5.435, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.435, "open": 5.435, "day": "31"} +{"volume": 10794, "symbol": "DNR", "ts": "2018-08-31 10:15:00", "month": "08", "high": 5.44, "low": 5.4335, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.44, "open": 5.4335, "day": "31"} +{"volume": 19418, "symbol": "DNR", "ts": "2018-08-31 10:16:00", "month": "08", "high": 5.4468, "low": 5.4468, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.4468, "open": 5.4468, "day": "31"} +{"volume": 16244, "symbol": "DNR", "ts": "2018-08-31 10:17:00", "month": "08", "high": 5.445, "low": 5.445, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.445, "open": 5.445, "day": "31"} +{"volume": 31508, "symbol": "DNR", "ts": "2018-08-31 10:18:00", "month": "08", "high": 5.45, "low": 5.45, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.45, "open": 5.45, "day": "31"} +{"volume": 46167, "symbol": "DNR", "ts": "2018-08-31 10:19:00", "month": "08", "high": 5.475, "low": 5.44, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.475, "open": 5.44, "day": "31"} +{"volume": 13968, "symbol": "DNR", "ts": "2018-08-31 10:20:00", "month": "08", "high": 5.475, "low": 5.475, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.475, "open": 5.475, "day": "31"} +{"volume": 6939, "symbol": "DNR", "ts": "2018-08-31 10:21:00", "month": "08", "high": 5.475, "low": 5.475, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.475, "open": 5.475, "day": "31"} +{"volume": 23238, "symbol": "DNR", "ts": "2018-08-31 10:22:00", "month": "08", "high": 5.49, "low": 5.475, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.475, "day": "31"} +{"volume": 13814, "symbol": "DNR", "ts": "2018-08-31 10:23:00", "month": "08", "high": 5.49, "low": 5.49, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.49, "day": "31"} +{"volume": 112754, "symbol": "DNR", "ts": "2018-08-31 10:24:00", "month": "08", "high": 5.495, "low": 5.48, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.48, "open": 5.49, "day": "31"} +{"volume": 200017, "symbol": "DNR", "ts": "2018-08-31 10:25:00", "month": "08", "high": 5.5, "low": 5.49, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.5, "open": 5.495, "day": "31"} +{"volume": 6291, "symbol": "DNR", "ts": "2018-08-31 10:26:00", "month": "08", "high": 5.49, "low": 5.49, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.49, "day": "31"} +{"volume": 22208, "symbol": "DNR", "ts": "2018-08-31 10:27:00", "month": "08", "high": 5.5, "low": 5.495, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.5, "open": 5.495, "day": "31"} +{"volume": 28267, "symbol": "DNR", "ts": "2018-08-31 10:28:00", "month": "08", "high": 5.4925, "low": 5.49, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.4925, "day": "31"} +{"volume": 4944, "symbol": "DNR", "ts": "2018-08-31 10:29:00", "month": "08", "high": 5.485, "low": 5.485, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.485, "open": 5.485, "day": "31"} +{"volume": 2200, "symbol": "CVNA", "ts": "2018-08-31 10:02:00", "month": "08", "high": 62.032, "low": 62.032, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.032, "open": 62.032, "day": "31"} +{"volume": 1073, "symbol": "CVNA", "ts": "2018-08-31 10:04:00", "month": "08", "high": 62.118, "low": 62.118, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.118, "open": 62.118, "day": "31"} +{"volume": 1670, "symbol": "CVNA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 61.98, "low": 61.98, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.98, "open": 61.98, "day": "31"} +{"volume": 857, "symbol": "CVNA", "ts": "2018-08-31 10:07:00", "month": "08", "high": 61.93, "low": 61.93, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.93, "open": 61.93, "day": "31"} +{"volume": 2343, "symbol": "CVNA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 61.85, "low": 61.71, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.85, "open": 61.71, "day": "31"} +{"volume": 272, "symbol": "CVNA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 61.83, "low": 61.83, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.83, "open": 61.83, "day": "31"} +{"volume": 1959, "symbol": "CVNA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 62.03, "low": 62.03, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.03, "open": 62.03, "day": "31"} +{"volume": 1713, "symbol": "CVNA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 62.1, "low": 62.1, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.1, "open": 62.1, "day": "31"} +{"volume": 1025, "symbol": "CVNA", "ts": "2018-08-31 10:18:00", "month": "08", "high": 62.01, "low": 62.01, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.01, "open": 62.01, "day": "31"} +{"volume": 705, "symbol": "CVNA", "ts": "2018-08-31 10:19:00", "month": "08", "high": 61.92, "low": 61.92, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.92, "open": 61.92, "day": "31"} +{"volume": 1814, "symbol": "CVNA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 62.04, "low": 62.04, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.04, "open": 62.04, "day": "31"} +{"volume": 876, "symbol": "CVNA", "ts": "2018-08-31 10:23:00", "month": "08", "high": 62.065, "low": 62.065, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.065, "open": 62.065, "day": "31"} +{"volume": 3338, "symbol": "CVNA", "ts": "2018-08-31 10:25:00", "month": "08", "high": 62.065, "low": 61.82, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.82, "open": 62.065, "day": "31"} +{"volume": 516, "symbol": "CVNA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 61.82, "low": 61.82, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.82, "open": 61.82, "day": "31"} +{"volume": 2554, "symbol": "CVNA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 62.01, "low": 62.01, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.01, "open": 62.01, "day": "31"} +{"volume": 6255, "symbol": "CVNA", "ts": "2018-08-31 10:29:00", "month": "08", "high": 62.2, "low": 62.07, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.2, "open": 62.07, "day": "31"} +{"volume": 2686, "symbol": "BOOT", "ts": "2018-08-31 10:01:00", "month": "08", "high": 29.37, "low": 29.37, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.37, "open": 29.37, "day": "31"} +{"volume": 600, "symbol": "BOOT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 29.3999, "low": 29.3999, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.3999, "open": 29.3999, "day": "31"} +{"volume": 1140, "symbol": "BOOT", "ts": "2018-08-31 10:04:00", "month": "08", "high": 29.38, "low": 29.38, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.38, "open": 29.38, "day": "31"} +{"volume": 2462, "symbol": "BOOT", "ts": "2018-08-31 10:06:00", "month": "08", "high": 29.4, "low": 29.4, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.4, "open": 29.4, "day": "31"} +{"volume": 2201, "symbol": "BOOT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 29.27, "low": 29.27, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.27, "open": 29.27, "day": "31"} +{"volume": 1759, "symbol": "BOOT", "ts": "2018-08-31 10:13:00", "month": "08", "high": 29.225, "low": 29.225, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.225, "open": 29.225, "day": "31"} +{"volume": 755, "symbol": "BOOT", "ts": "2018-08-31 10:14:00", "month": "08", "high": 29.225, "low": 29.225, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.225, "open": 29.225, "day": "31"} +{"volume": 1490, "symbol": "BOOT", "ts": "2018-08-31 10:16:00", "month": "08", "high": 29.23, "low": 29.23, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.23, "open": 29.23, "day": "31"} +{"volume": 2077, "symbol": "BOOT", "ts": "2018-08-31 10:17:00", "month": "08", "high": 29.25, "low": 29.25, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.25, "open": 29.25, "day": "31"} +{"volume": 2001, "symbol": "BOOT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 29.36, "low": 29.36, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.36, "open": 29.36, "day": "31"} +{"volume": 500, "symbol": "BOOT", "ts": "2018-08-31 10:19:00", "month": "08", "high": 29.38, "low": 29.38, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.38, "open": 29.38, "day": "31"} +{"volume": 505, "symbol": "BOOT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 29.3897, "low": 29.3897, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.3897, "open": 29.3897, "day": "31"} +{"volume": 1832, "symbol": "BOOT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 29.36, "low": 29.36, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.36, "open": 29.36, "day": "31"} +{"volume": 3210, "symbol": "BOOT", "ts": "2018-08-31 10:25:00", "month": "08", "high": 29.44, "low": 29.44, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.44, "open": 29.44, "day": "31"} +{"volume": 1450, "symbol": "BOOT", "ts": "2018-08-31 10:27:00", "month": "08", "high": 29.43, "low": 29.43, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.43, "open": 29.43, "day": "31"} +{"volume": 1137, "symbol": "BOOT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 29.47, "low": 29.47, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.47, "open": 29.47, "day": "31"} +{"volume": 42091, "symbol": "SQ", "ts": "2018-08-31 10:01:00", "month": "08", "high": 88.0273, "low": 87.88, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.88, "open": 87.96, "day": "31"} +{"volume": 34689, "symbol": "SQ", "ts": "2018-08-31 10:02:00", "month": "08", "high": 88.01, "low": 87.92, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.9901, "open": 87.92, "day": "31"} +{"volume": 65445, "symbol": "SQ", "ts": "2018-08-31 10:03:00", "month": "08", "high": 88.03, "low": 87.96, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.98, "open": 88.01, "day": "31"} +{"volume": 65178, "symbol": "SQ", "ts": "2018-08-31 10:04:00", "month": "08", "high": 87.98, "low": 87.82, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.88, "open": 87.98, "day": "31"} +{"volume": 59787, "symbol": "SQ", "ts": "2018-08-31 10:05:00", "month": "08", "high": 87.98, "low": 87.89, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.9799, "open": 87.89, "day": "31"} +{"volume": 61175, "symbol": "SQ", "ts": "2018-08-31 10:06:00", "month": "08", "high": 87.96, "low": 87.83, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.92, "open": 87.93, "day": "31"} +{"volume": 49174, "symbol": "SQ", "ts": "2018-08-31 10:07:00", "month": "08", "high": 87.93, "low": 87.78, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.78, "open": 87.93, "day": "31"} +{"volume": 95318, "symbol": "SQ", "ts": "2018-08-31 10:08:00", "month": "08", "high": 87.75, "low": 87.5432, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.5432, "open": 87.75, "day": "31"} +{"volume": 132449, "symbol": "SQ", "ts": "2018-08-31 10:09:00", "month": "08", "high": 87.56, "low": 87.39, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.47, "open": 87.5388, "day": "31"} +{"volume": 49616, "symbol": "SQ", "ts": "2018-08-31 10:10:00", "month": "08", "high": 87.825, "low": 87.441, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.8149, "open": 87.441, "day": "31"} +{"volume": 76574, "symbol": "SQ", "ts": "2018-08-31 10:11:00", "month": "08", "high": 87.79, "low": 87.42, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.42, "open": 87.73, "day": "31"} +{"volume": 47272, "symbol": "SQ", "ts": "2018-08-31 10:12:00", "month": "08", "high": 87.76, "low": 87.5, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.76, "open": 87.5, "day": "31"} +{"volume": 22073, "symbol": "SQ", "ts": "2018-08-31 10:13:00", "month": "08", "high": 87.73, "low": 87.65, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.69, "open": 87.73, "day": "31"} +{"volume": 94219, "symbol": "SQ", "ts": "2018-08-31 10:14:00", "month": "08", "high": 88.01, "low": 87.757, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.0, "open": 87.76, "day": "31"} +{"volume": 66918, "symbol": "SQ", "ts": "2018-08-31 10:15:00", "month": "08", "high": 88.17, "low": 87.98, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.1699, "open": 87.99, "day": "31"} +{"volume": 45479, "symbol": "SQ", "ts": "2018-08-31 10:16:00", "month": "08", "high": 88.29, "low": 88.1873, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.29, "open": 88.1873, "day": "31"} +{"volume": 62892, "symbol": "SQ", "ts": "2018-08-31 10:17:00", "month": "08", "high": 88.3, "low": 88.1201, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.1201, "open": 88.2, "day": "31"} +{"volume": 30576, "symbol": "SQ", "ts": "2018-08-31 10:18:00", "month": "08", "high": 88.185, "low": 88.05, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.05, "open": 88.169, "day": "31"} +{"volume": 27028, "symbol": "SQ", "ts": "2018-08-31 10:19:00", "month": "08", "high": 88.13, "low": 88.0008, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.11, "open": 88.0008, "day": "31"} +{"volume": 30915, "symbol": "SQ", "ts": "2018-08-31 10:20:00", "month": "08", "high": 88.19, "low": 88.0345, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.12, "open": 88.06, "day": "31"} +{"volume": 42058, "symbol": "SQ", "ts": "2018-08-31 10:21:00", "month": "08", "high": 88.145, "low": 88.03, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.03, "open": 88.08, "day": "31"} +{"volume": 35025, "symbol": "SQ", "ts": "2018-08-31 10:22:00", "month": "08", "high": 88.14, "low": 88.0702, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.14, "open": 88.0702, "day": "31"} +{"volume": 31898, "symbol": "SQ", "ts": "2018-08-31 10:23:00", "month": "08", "high": 88.22, "low": 88.14, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.22, "open": 88.14, "day": "31"} +{"volume": 44883, "symbol": "SQ", "ts": "2018-08-31 10:24:00", "month": "08", "high": 88.35, "low": 88.21, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.31, "open": 88.21, "day": "31"} +{"volume": 30245, "symbol": "SQ", "ts": "2018-08-31 10:25:00", "month": "08", "high": 88.37, "low": 88.27, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.37, "open": 88.3292, "day": "31"} +{"volume": 30944, "symbol": "SQ", "ts": "2018-08-31 10:26:00", "month": "08", "high": 88.3504, "low": 88.14, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.14, "open": 88.3504, "day": "31"} +{"volume": 22963, "symbol": "SQ", "ts": "2018-08-31 10:27:00", "month": "08", "high": 88.24, "low": 88.11, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.24, "open": 88.11, "day": "31"} +{"volume": 26853, "symbol": "SQ", "ts": "2018-08-31 10:28:00", "month": "08", "high": 88.26, "low": 88.079, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.09, "open": 88.26, "day": "31"} +{"volume": 36902, "symbol": "SQ", "ts": "2018-08-31 10:29:00", "month": "08", "high": 88.1, "low": 87.96, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.0371, "open": 88.1, "day": "31"} +{"volume": 7521, "symbol": "EGAN", "ts": "2018-08-31 10:01:00", "month": "08", "high": 14.4193, "low": 14.4193, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4193, "open": 14.4193, "day": "31"} +{"volume": 902, "symbol": "EGAN", "ts": "2018-08-31 10:02:00", "month": "08", "high": 14.45, "low": 14.45, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.45, "open": 14.45, "day": "31"} +{"volume": 277, "symbol": "EGAN", "ts": "2018-08-31 10:06:00", "month": "08", "high": 14.54, "low": 14.54, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.54, "open": 14.54, "day": "31"} +{"volume": 987, "symbol": "EGAN", "ts": "2018-08-31 10:07:00", "month": "08", "high": 14.5248, "low": 14.5248, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.5248, "open": 14.5248, "day": "31"} +{"volume": 410, "symbol": "EGAN", "ts": "2018-08-31 10:09:00", "month": "08", "high": 14.525, "low": 14.525, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.525, "open": 14.525, "day": "31"} +{"volume": 130, "symbol": "EGAN", "ts": "2018-08-31 10:10:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 590, "symbol": "EGAN", "ts": "2018-08-31 10:11:00", "month": "08", "high": 14.549, "low": 14.549, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.549, "open": 14.549, "day": "31"} +{"volume": 13127, "symbol": "EGAN", "ts": "2018-08-31 10:14:00", "month": "08", "high": 14.6, "low": 14.6, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.6, "open": 14.6, "day": "31"} +{"volume": 2714, "symbol": "EGAN", "ts": "2018-08-31 10:15:00", "month": "08", "high": 14.6, "low": 14.6, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.6, "open": 14.6, "day": "31"} +{"volume": 800, "symbol": "EGAN", "ts": "2018-08-31 10:19:00", "month": "08", "high": 14.75, "low": 14.75, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.75, "open": 14.75, "day": "31"} +{"volume": 100, "symbol": "EGAN", "ts": "2018-08-31 10:20:00", "month": "08", "high": 14.7, "low": 14.7, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.7, "open": 14.7, "day": "31"} +{"volume": 2852, "symbol": "EGAN", "ts": "2018-08-31 10:21:00", "month": "08", "high": 14.75, "low": 14.75, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.75, "open": 14.75, "day": "31"} +{"volume": 4990, "symbol": "EGAN", "ts": "2018-08-31 10:22:00", "month": "08", "high": 14.8, "low": 14.7, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.7, "open": 14.8, "day": "31"} +{"volume": 1716, "symbol": "EGAN", "ts": "2018-08-31 10:25:00", "month": "08", "high": 14.7, "low": 14.7, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.7, "open": 14.7, "day": "31"} +{"volume": 203, "symbol": "EGAN", "ts": "2018-08-31 10:27:00", "month": "08", "high": 14.75, "low": 14.75, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.75, "open": 14.75, "day": "31"} +{"volume": 100, "symbol": "EGAN", "ts": "2018-08-31 10:28:00", "month": "08", "high": 14.705, "low": 14.705, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.705, "open": 14.705, "day": "31"} +{"volume": 5775, "symbol": "RCKT", "ts": "2018-08-31 10:01:00", "month": "08", "high": 23.665, "low": 23.52, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.52, "open": 23.665, "day": "31"} +{"volume": 423, "symbol": "RCKT", "ts": "2018-08-31 10:08:00", "month": "08", "high": 23.71, "low": 23.71, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.71, "open": 23.71, "day": "31"} +{"volume": 607, "symbol": "RCKT", "ts": "2018-08-31 10:20:00", "month": "08", "high": 23.73, "low": 23.55, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.55, "open": 23.73, "day": "31"} +{"volume": 200, "symbol": "RCKT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 23.56, "low": 23.56, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.56, "open": 23.56, "day": "31"} +{"volume": 4866, "symbol": "RCKT", "ts": "2018-08-31 10:25:00", "month": "08", "high": 23.67, "low": 23.43, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.43, "open": 23.63, "day": "31"} +{"volume": 2262, "symbol": "RCKT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 23.32, "low": 23.32, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.32, "open": 23.32, "day": "31"} +{"volume": 1909, "symbol": "NGVC", "ts": "2018-08-31 10:02:00", "month": "08", "high": 19.0, "low": 18.94, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.94, "open": 19.0, "day": "31"} +{"volume": 564, "symbol": "NGVC", "ts": "2018-08-31 10:04:00", "month": "08", "high": 18.97, "low": 18.97, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.97, "open": 18.97, "day": "31"} +{"volume": 580, "symbol": "NGVC", "ts": "2018-08-31 10:08:00", "month": "08", "high": 19.01, "low": 19.01, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.01, "open": 19.01, "day": "31"} +{"volume": 676, "symbol": "NGVC", "ts": "2018-08-31 10:10:00", "month": "08", "high": 18.98, "low": 18.98, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.98, "open": 18.98, "day": "31"} +{"volume": 406, "symbol": "NGVC", "ts": "2018-08-31 10:13:00", "month": "08", "high": 18.96, "low": 18.96, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.96, "open": 18.96, "day": "31"} +{"volume": 890, "symbol": "NGVC", "ts": "2018-08-31 10:14:00", "month": "08", "high": 18.98, "low": 18.98, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.98, "open": 18.98, "day": "31"} +{"volume": 1050, "symbol": "NGVC", "ts": "2018-08-31 10:16:00", "month": "08", "high": 18.94, "low": 18.94, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.94, "open": 18.94, "day": "31"} +{"volume": 437, "symbol": "NGVC", "ts": "2018-08-31 10:29:00", "month": "08", "high": 18.96, "low": 18.96, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.96, "open": 18.96, "day": "31"} +{"volume": 191, "symbol": "SHSP", "ts": "2018-08-31 10:04:00", "month": "08", "high": 14.0637, "low": 14.0637, "key": "SHSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0637, "open": 14.0637, "day": "31"} +{"volume": 2300, "symbol": "VNCE", "ts": "2018-08-31 10:14:00", "month": "08", "high": 20.97, "low": 20.9314, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.97, "open": 20.9314, "day": "31"} +{"volume": 501, "symbol": "VNCE", "ts": "2018-08-31 10:21:00", "month": "08", "high": 20.9116, "low": 20.9116, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.9116, "open": 20.9116, "day": "31"} +{"volume": 2706, "symbol": "VNCE", "ts": "2018-08-31 10:29:00", "month": "08", "high": 20.86, "low": 20.86, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.86, "open": 20.86, "day": "31"} +{"volume": 2245, "symbol": "CRC", "ts": "2018-08-31 10:01:00", "month": "08", "high": 39.91, "low": 39.91, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.91, "open": 39.91, "day": "31"} +{"volume": 1542, "symbol": "CRC", "ts": "2018-08-31 10:02:00", "month": "08", "high": 39.88, "low": 39.88, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.88, "open": 39.88, "day": "31"} +{"volume": 4439, "symbol": "CRC", "ts": "2018-08-31 10:03:00", "month": "08", "high": 39.96, "low": 39.96, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.96, "open": 39.96, "day": "31"} +{"volume": 2151, "symbol": "CRC", "ts": "2018-08-31 10:04:00", "month": "08", "high": 40.005, "low": 40.005, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.005, "open": 40.005, "day": "31"} +{"volume": 2263, "symbol": "CRC", "ts": "2018-08-31 10:05:00", "month": "08", "high": 40.0305, "low": 40.0305, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.0305, "open": 40.0305, "day": "31"} +{"volume": 4900, "symbol": "CRC", "ts": "2018-08-31 10:07:00", "month": "08", "high": 40.172, "low": 40.172, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.172, "open": 40.172, "day": "31"} +{"volume": 2583, "symbol": "CRC", "ts": "2018-08-31 10:08:00", "month": "08", "high": 40.12, "low": 40.12, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.12, "open": 40.12, "day": "31"} +{"volume": 1475, "symbol": "CRC", "ts": "2018-08-31 10:10:00", "month": "08", "high": 40.12, "low": 40.12, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.12, "open": 40.12, "day": "31"} +{"volume": 757, "symbol": "CRC", "ts": "2018-08-31 10:11:00", "month": "08", "high": 40.134, "low": 40.134, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.134, "open": 40.134, "day": "31"} +{"volume": 5284, "symbol": "CRC", "ts": "2018-08-31 10:13:00", "month": "08", "high": 40.25, "low": 40.2, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.25, "open": 40.2, "day": "31"} +{"volume": 401, "symbol": "CRC", "ts": "2018-08-31 10:15:00", "month": "08", "high": 40.3119, "low": 40.3119, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3119, "open": 40.3119, "day": "31"} +{"volume": 6290, "symbol": "CRC", "ts": "2018-08-31 10:17:00", "month": "08", "high": 40.28, "low": 40.11, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.11, "open": 40.28, "day": "31"} +{"volume": 6633, "symbol": "CRC", "ts": "2018-08-31 10:18:00", "month": "08", "high": 40.03, "low": 39.98, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.98, "open": 40.03, "day": "31"} +{"volume": 4244, "symbol": "CRC", "ts": "2018-08-31 10:19:00", "month": "08", "high": 40.15, "low": 40.15, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.15, "open": 40.15, "day": "31"} +{"volume": 2122, "symbol": "CRC", "ts": "2018-08-31 10:20:00", "month": "08", "high": 40.29, "low": 40.29, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.29, "open": 40.29, "day": "31"} +{"volume": 801, "symbol": "CRC", "ts": "2018-08-31 10:21:00", "month": "08", "high": 40.29, "low": 40.29, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.29, "open": 40.29, "day": "31"} +{"volume": 821, "symbol": "CRC", "ts": "2018-08-31 10:23:00", "month": "08", "high": 40.305, "low": 40.305, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.305, "open": 40.305, "day": "31"} +{"volume": 1656, "symbol": "CRC", "ts": "2018-08-31 10:24:00", "month": "08", "high": 40.315, "low": 40.315, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.315, "open": 40.315, "day": "31"} +{"volume": 1588, "symbol": "CRC", "ts": "2018-08-31 10:25:00", "month": "08", "high": 40.23, "low": 40.23, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.23, "open": 40.23, "day": "31"} +{"volume": 3408, "symbol": "CRC", "ts": "2018-08-31 10:26:00", "month": "08", "high": 40.2315, "low": 40.2315, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.2315, "open": 40.2315, "day": "31"} +{"volume": 2300, "symbol": "CRC", "ts": "2018-08-31 10:27:00", "month": "08", "high": 40.17, "low": 40.17, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.17, "open": 40.17, "day": "31"} +{"volume": 10953, "symbol": "CRC", "ts": "2018-08-31 10:28:00", "month": "08", "high": 40.15, "low": 40.11, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.15, "open": 40.11, "day": "31"} +{"volume": 7400, "symbol": "CRC", "ts": "2018-08-31 10:29:00", "month": "08", "high": 40.16, "low": 40.125, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.125, "open": 40.16, "day": "31"} +{"volume": 5331, "symbol": "VKTX", "ts": "2018-08-31 10:01:00", "month": "08", "high": 12.66, "low": 12.66, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.66, "open": 12.66, "day": "31"} +{"volume": 18730, "symbol": "VKTX", "ts": "2018-08-31 10:02:00", "month": "08", "high": 12.67, "low": 12.62, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.67, "open": 12.655, "day": "31"} +{"volume": 18149, "symbol": "VKTX", "ts": "2018-08-31 10:03:00", "month": "08", "high": 12.67, "low": 12.6522, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6522, "open": 12.67, "day": "31"} +{"volume": 21411, "symbol": "VKTX", "ts": "2018-08-31 10:04:00", "month": "08", "high": 12.71, "low": 12.6934, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6934, "open": 12.71, "day": "31"} +{"volume": 8999, "symbol": "VKTX", "ts": "2018-08-31 10:05:00", "month": "08", "high": 12.7107, "low": 12.7107, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.7107, "open": 12.7107, "day": "31"} +{"volume": 17351, "symbol": "VKTX", "ts": "2018-08-31 10:06:00", "month": "08", "high": 12.72, "low": 12.69, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.69, "open": 12.72, "day": "31"} +{"volume": 9695, "symbol": "VKTX", "ts": "2018-08-31 10:07:00", "month": "08", "high": 12.78, "low": 12.73, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.78, "open": 12.73, "day": "31"} +{"volume": 12423, "symbol": "VKTX", "ts": "2018-08-31 10:08:00", "month": "08", "high": 12.7354, "low": 12.7354, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.7354, "open": 12.7354, "day": "31"} +{"volume": 13044, "symbol": "VKTX", "ts": "2018-08-31 10:09:00", "month": "08", "high": 12.7884, "low": 12.78, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.7884, "open": 12.78, "day": "31"} +{"volume": 11080, "symbol": "VKTX", "ts": "2018-08-31 10:10:00", "month": "08", "high": 12.81, "low": 12.81, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.81, "open": 12.81, "day": "31"} +{"volume": 1229, "symbol": "VKTX", "ts": "2018-08-31 10:11:00", "month": "08", "high": 12.77, "low": 12.77, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.77, "open": 12.77, "day": "31"} +{"volume": 5853, "symbol": "VKTX", "ts": "2018-08-31 10:12:00", "month": "08", "high": 12.829, "low": 12.829, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.829, "open": 12.829, "day": "31"} +{"volume": 9883, "symbol": "VKTX", "ts": "2018-08-31 10:13:00", "month": "08", "high": 12.78, "low": 12.78, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.78, "open": 12.78, "day": "31"} +{"volume": 45480, "symbol": "VKTX", "ts": "2018-08-31 10:14:00", "month": "08", "high": 12.75, "low": 12.75, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.75, "open": 12.75, "day": "31"} +{"volume": 7259, "symbol": "VKTX", "ts": "2018-08-31 10:15:00", "month": "08", "high": 12.71, "low": 12.71, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.71, "open": 12.71, "day": "31"} +{"volume": 22507, "symbol": "VKTX", "ts": "2018-08-31 10:16:00", "month": "08", "high": 12.7, "low": 12.66, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.66, "open": 12.7, "day": "31"} +{"volume": 11083, "symbol": "VKTX", "ts": "2018-08-31 10:17:00", "month": "08", "high": 12.665, "low": 12.665, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.665, "open": 12.665, "day": "31"} +{"volume": 4947, "symbol": "VKTX", "ts": "2018-08-31 10:18:00", "month": "08", "high": 12.7, "low": 12.7, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.7, "open": 12.7, "day": "31"} +{"volume": 3540, "symbol": "VKTX", "ts": "2018-08-31 10:19:00", "month": "08", "high": 12.69, "low": 12.69, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.69, "open": 12.69, "day": "31"} +{"volume": 2797, "symbol": "VKTX", "ts": "2018-08-31 10:20:00", "month": "08", "high": 12.63, "low": 12.63, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.63, "open": 12.63, "day": "31"} +{"volume": 8753, "symbol": "VKTX", "ts": "2018-08-31 10:21:00", "month": "08", "high": 12.675, "low": 12.64, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.675, "open": 12.64, "day": "31"} +{"volume": 3410, "symbol": "VKTX", "ts": "2018-08-31 10:23:00", "month": "08", "high": 12.6832, "low": 12.6832, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6832, "open": 12.6832, "day": "31"} +{"volume": 2673, "symbol": "VKTX", "ts": "2018-08-31 10:24:00", "month": "08", "high": 12.6301, "low": 12.6301, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6301, "open": 12.6301, "day": "31"} +{"volume": 9689, "symbol": "VKTX", "ts": "2018-08-31 10:25:00", "month": "08", "high": 12.6599, "low": 12.61, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.61, "open": 12.6599, "day": "31"} +{"volume": 11322, "symbol": "VKTX", "ts": "2018-08-31 10:26:00", "month": "08", "high": 12.64, "low": 12.64, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.64, "open": 12.64, "day": "31"} +{"volume": 6222, "symbol": "VKTX", "ts": "2018-08-31 10:27:00", "month": "08", "high": 12.62, "low": 12.62, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.62, "open": 12.62, "day": "31"} +{"volume": 4603, "symbol": "VKTX", "ts": "2018-08-31 10:28:00", "month": "08", "high": 12.625, "low": 12.625, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.625, "open": 12.625, "day": "31"} +{"volume": 24252, "symbol": "VKTX", "ts": "2018-08-31 10:29:00", "month": "08", "high": 12.6, "low": 12.6, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6, "open": 12.6, "day": "31"} +{"volume": 426, "symbol": "CDXS", "ts": "2018-08-31 10:05:00", "month": "08", "high": 17.101, "low": 17.101, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.101, "open": 17.101, "day": "31"} +{"volume": 474, "symbol": "CDXS", "ts": "2018-08-31 10:08:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 251, "symbol": "CDXS", "ts": "2018-08-31 10:10:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 1296, "symbol": "CDXS", "ts": "2018-08-31 10:13:00", "month": "08", "high": 17.15, "low": 17.15, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.15, "open": 17.15, "day": "31"} +{"volume": 202, "symbol": "CDXS", "ts": "2018-08-31 10:16:00", "month": "08", "high": 17.15, "low": 17.15, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.15, "open": 17.15, "day": "31"} +{"volume": 1073, "symbol": "CDXS", "ts": "2018-08-31 10:17:00", "month": "08", "high": 17.145, "low": 17.145, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.145, "open": 17.145, "day": "31"} +{"volume": 400, "symbol": "CDXS", "ts": "2018-08-31 10:20:00", "month": "08", "high": 17.15, "low": 17.15, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.15, "open": 17.15, "day": "31"} +{"volume": 730, "symbol": "CDXS", "ts": "2018-08-31 10:21:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 100, "symbol": "CDXS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 700, "symbol": "CDXS", "ts": "2018-08-31 10:29:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 117, "symbol": "LFVN", "ts": "2018-08-31 10:03:00", "month": "08", "high": 12.0059, "low": 12.0059, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.0059, "open": 12.0059, "day": "31"} +{"volume": 808, "symbol": "LFVN", "ts": "2018-08-31 10:05:00", "month": "08", "high": 12.07, "low": 12.07, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.07, "open": 12.07, "day": "31"} +{"volume": 694, "symbol": "LFVN", "ts": "2018-08-31 10:09:00", "month": "08", "high": 12.17, "low": 12.14, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.17, "open": 12.14, "day": "31"} +{"volume": 1396, "symbol": "LFVN", "ts": "2018-08-31 10:13:00", "month": "08", "high": 12.25, "low": 12.25, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.25, "open": 12.25, "day": "31"} +{"volume": 2067, "symbol": "LFVN", "ts": "2018-08-31 10:17:00", "month": "08", "high": 12.2564, "low": 12.2564, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.2564, "open": 12.2564, "day": "31"} +{"volume": 1400, "symbol": "LFVN", "ts": "2018-08-31 10:18:00", "month": "08", "high": 12.265, "low": 12.265, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.265, "open": 12.265, "day": "31"} +{"volume": 927, "symbol": "LFVN", "ts": "2018-08-31 10:19:00", "month": "08", "high": 12.3, "low": 12.3, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.3, "open": 12.3, "day": "31"} +{"volume": 2860, "symbol": "LFVN", "ts": "2018-08-31 10:22:00", "month": "08", "high": 12.21, "low": 12.21, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.21, "open": 12.21, "day": "31"} +{"volume": 1775, "symbol": "LFVN", "ts": "2018-08-31 10:26:00", "month": "08", "high": 12.255, "low": 12.255, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.255, "open": 12.255, "day": "31"} +{"volume": 198, "symbol": "SRDX", "ts": "2018-08-31 10:07:00", "month": "08", "high": 76.275, "low": 76.275, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 76.275, "open": 76.275, "day": "31"} +{"volume": 1453, "symbol": "SRDX", "ts": "2018-08-31 10:22:00", "month": "08", "high": 76.7, "low": 76.7, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 76.7, "open": 76.7, "day": "31"} +{"volume": 252, "symbol": "SRDX", "ts": "2018-08-31 10:23:00", "month": "08", "high": 76.725, "low": 76.725, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 76.725, "open": 76.725, "day": "31"} +{"volume": 100, "symbol": "SRDX", "ts": "2018-08-31 10:27:00", "month": "08", "high": 76.95, "low": 76.95, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 76.95, "open": 76.95, "day": "31"} +{"volume": 1216, "symbol": "SRDX", "ts": "2018-08-31 10:29:00", "month": "08", "high": 77.15, "low": 77.15, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.15, "open": 77.15, "day": "31"} +{"volume": 0, "symbol": "LGCYP", "ts": "2018-08-31 10:09:00", "month": "08", "high": 15.64, "low": 15.63, "key": "LGCYP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.64, "open": 15.63, "day": "31"} +{"volume": 2103, "symbol": "SDPI", "ts": "2018-08-31 10:02:00", "month": "08", "high": 1.891, "low": 1.891, "key": "SDPI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1.891, "open": 1.891, "day": "31"} +{"volume": 1005, "symbol": "SDPI", "ts": "2018-08-31 10:27:00", "month": "08", "high": 1.9098, "low": 1.9098, "key": "SDPI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1.9098, "open": 1.9098, "day": "31"} +{"volume": 907, "symbol": "MDB", "ts": "2018-08-31 10:01:00", "month": "08", "high": 71.3268, "low": 71.3268, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.3268, "open": 71.3268, "day": "31"} +{"volume": 1494, "symbol": "MDB", "ts": "2018-08-31 10:03:00", "month": "08", "high": 71.415, "low": 71.415, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.415, "open": 71.415, "day": "31"} +{"volume": 142, "symbol": "MDB", "ts": "2018-08-31 10:05:00", "month": "08", "high": 71.54, "low": 71.54, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.54, "open": 71.54, "day": "31"} +{"volume": 7846, "symbol": "MDB", "ts": "2018-08-31 10:07:00", "month": "08", "high": 71.585, "low": 71.585, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.585, "open": 71.585, "day": "31"} +{"volume": 1085, "symbol": "MDB", "ts": "2018-08-31 10:10:00", "month": "08", "high": 71.67, "low": 71.67, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.67, "open": 71.67, "day": "31"} +{"volume": 2670, "symbol": "MDB", "ts": "2018-08-31 10:11:00", "month": "08", "high": 71.77, "low": 71.77, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.77, "open": 71.77, "day": "31"} +{"volume": 2148, "symbol": "MDB", "ts": "2018-08-31 10:12:00", "month": "08", "high": 71.9479, "low": 71.9479, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.9479, "open": 71.9479, "day": "31"} +{"volume": 867, "symbol": "MDB", "ts": "2018-08-31 10:13:00", "month": "08", "high": 71.83, "low": 71.83, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.83, "open": 71.83, "day": "31"} +{"volume": 5013, "symbol": "MDB", "ts": "2018-08-31 10:15:00", "month": "08", "high": 72.0735, "low": 72.0, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 72.0735, "open": 72.0, "day": "31"} +{"volume": 1613, "symbol": "MDB", "ts": "2018-08-31 10:17:00", "month": "08", "high": 71.975, "low": 71.975, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.975, "open": 71.975, "day": "31"} +{"volume": 1473, "symbol": "MDB", "ts": "2018-08-31 10:18:00", "month": "08", "high": 71.9579, "low": 71.9579, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.9579, "open": 71.9579, "day": "31"} +{"volume": 1602, "symbol": "MDB", "ts": "2018-08-31 10:20:00", "month": "08", "high": 71.78, "low": 71.78, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.78, "open": 71.78, "day": "31"} +{"volume": 1248, "symbol": "MDB", "ts": "2018-08-31 10:21:00", "month": "08", "high": 71.72, "low": 71.72, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.72, "open": 71.72, "day": "31"} +{"volume": 1097, "symbol": "MDB", "ts": "2018-08-31 10:23:00", "month": "08", "high": 71.7, "low": 71.7, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.7, "open": 71.7, "day": "31"} +{"volume": 1531, "symbol": "MDB", "ts": "2018-08-31 10:24:00", "month": "08", "high": 71.805, "low": 71.805, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.805, "open": 71.805, "day": "31"} +{"volume": 404, "symbol": "MDB", "ts": "2018-08-31 10:25:00", "month": "08", "high": 71.75, "low": 71.75, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.75, "open": 71.75, "day": "31"} +{"volume": 286, "symbol": "MDB", "ts": "2018-08-31 10:26:00", "month": "08", "high": 71.64, "low": 71.64, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.64, "open": 71.64, "day": "31"} +{"volume": 3159, "symbol": "LGCY", "ts": "2018-08-31 10:06:00", "month": "08", "high": 5.38, "low": 5.38, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.38, "open": 5.38, "day": "31"} +{"volume": 100, "symbol": "LGCY", "ts": "2018-08-31 10:09:00", "month": "08", "high": 5.39, "low": 5.39, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.39, "open": 5.39, "day": "31"} +{"volume": 6300, "symbol": "LGCY", "ts": "2018-08-31 10:12:00", "month": "08", "high": 5.3622, "low": 5.3622, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.3622, "open": 5.3622, "day": "31"} +{"volume": 1873, "symbol": "LGCY", "ts": "2018-08-31 10:15:00", "month": "08", "high": 5.345, "low": 5.34, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.34, "open": 5.345, "day": "31"} +{"volume": 340, "symbol": "LGCY", "ts": "2018-08-31 10:19:00", "month": "08", "high": 5.35, "low": 5.35, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.35, "open": 5.35, "day": "31"} +{"volume": 640, "symbol": "LGCY", "ts": "2018-08-31 10:26:00", "month": "08", "high": 5.3523, "low": 5.3523, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.3523, "open": 5.3523, "day": "31"} +{"volume": 3592, "symbol": "ATTU", "ts": "2018-08-31 10:01:00", "month": "08", "high": 20.63, "low": 20.63, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.63, "open": 20.63, "day": "31"} +{"volume": 725, "symbol": "ATTU", "ts": "2018-08-31 10:04:00", "month": "08", "high": 20.64, "low": 20.62, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.62, "open": 20.64, "day": "31"} +{"volume": 2717, "symbol": "ATTU", "ts": "2018-08-31 10:07:00", "month": "08", "high": 20.67, "low": 20.635, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.67, "open": 20.635, "day": "31"} +{"volume": 200, "symbol": "ATTU", "ts": "2018-08-31 10:11:00", "month": "08", "high": 20.64, "low": 20.64, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.64, "open": 20.64, "day": "31"} +{"volume": 702, "symbol": "ATTU", "ts": "2018-08-31 10:12:00", "month": "08", "high": 20.669, "low": 20.669, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.669, "open": 20.669, "day": "31"} +{"volume": 750, "symbol": "ATTU", "ts": "2018-08-31 10:15:00", "month": "08", "high": 20.64, "low": 20.64, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.64, "open": 20.64, "day": "31"} +{"volume": 3617, "symbol": "ATTU", "ts": "2018-08-31 10:17:00", "month": "08", "high": 20.67, "low": 20.64, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.67, "open": 20.64, "day": "31"} +{"volume": 4244, "symbol": "ATTU", "ts": "2018-08-31 10:20:00", "month": "08", "high": 20.675, "low": 20.615, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.615, "open": 20.675, "day": "31"} +{"volume": 1215, "symbol": "ATTU", "ts": "2018-08-31 10:22:00", "month": "08", "high": 20.6594, "low": 20.6594, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.6594, "open": 20.6594, "day": "31"} +{"volume": 2800, "symbol": "ATTU", "ts": "2018-08-31 10:25:00", "month": "08", "high": 20.59, "low": 20.59, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.59, "open": 20.59, "day": "31"} +{"volume": 4100, "symbol": "ATTU", "ts": "2018-08-31 10:26:00", "month": "08", "high": 20.6, "low": 20.5864, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.6, "open": 20.5864, "day": "31"} +{"volume": 1214, "symbol": "AYX", "ts": "2018-08-31 10:01:00", "month": "08", "high": 56.51, "low": 56.51, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.51, "open": 56.51, "day": "31"} +{"volume": 2010, "symbol": "AYX", "ts": "2018-08-31 10:02:00", "month": "08", "high": 56.62, "low": 56.62, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.62, "open": 56.62, "day": "31"} +{"volume": 1751, "symbol": "AYX", "ts": "2018-08-31 10:04:00", "month": "08", "high": 56.87, "low": 56.87, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.87, "open": 56.87, "day": "31"} +{"volume": 1741, "symbol": "AYX", "ts": "2018-08-31 10:05:00", "month": "08", "high": 56.71, "low": 56.71, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.71, "open": 56.71, "day": "31"} +{"volume": 1600, "symbol": "AYX", "ts": "2018-08-31 10:07:00", "month": "08", "high": 56.83, "low": 56.83, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.83, "open": 56.83, "day": "31"} +{"volume": 1600, "symbol": "AYX", "ts": "2018-08-31 10:08:00", "month": "08", "high": 56.76, "low": 56.76, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.76, "open": 56.76, "day": "31"} +{"volume": 2319, "symbol": "AYX", "ts": "2018-08-31 10:10:00", "month": "08", "high": 56.99, "low": 56.99, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.99, "open": 56.99, "day": "31"} +{"volume": 2873, "symbol": "AYX", "ts": "2018-08-31 10:11:00", "month": "08", "high": 57.06, "low": 57.06, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.06, "open": 57.06, "day": "31"} +{"volume": 1047, "symbol": "AYX", "ts": "2018-08-31 10:13:00", "month": "08", "high": 56.96, "low": 56.96, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.96, "open": 56.96, "day": "31"} +{"volume": 2590, "symbol": "AYX", "ts": "2018-08-31 10:14:00", "month": "08", "high": 57.22, "low": 57.22, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.22, "open": 57.22, "day": "31"} +{"volume": 1898, "symbol": "AYX", "ts": "2018-08-31 10:15:00", "month": "08", "high": 57.24, "low": 57.24, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.24, "open": 57.24, "day": "31"} +{"volume": 1683, "symbol": "AYX", "ts": "2018-08-31 10:17:00", "month": "08", "high": 57.21, "low": 57.21, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.21, "open": 57.21, "day": "31"} +{"volume": 3888, "symbol": "AYX", "ts": "2018-08-31 10:19:00", "month": "08", "high": 57.2799, "low": 57.145, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.145, "open": 57.2799, "day": "31"} +{"volume": 3852, "symbol": "AYX", "ts": "2018-08-31 10:20:00", "month": "08", "high": 57.08, "low": 57.08, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.08, "open": 57.08, "day": "31"} +{"volume": 400, "symbol": "AYX", "ts": "2018-08-31 10:21:00", "month": "08", "high": 57.0861, "low": 57.0861, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.0861, "open": 57.0861, "day": "31"} +{"volume": 746, "symbol": "AYX", "ts": "2018-08-31 10:24:00", "month": "08", "high": 57.045, "low": 57.045, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.045, "open": 57.045, "day": "31"} +{"volume": 1056, "symbol": "AYX", "ts": "2018-08-31 10:27:00", "month": "08", "high": 57.065, "low": 57.065, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.065, "open": 57.065, "day": "31"} +{"volume": 2575, "symbol": "AYX", "ts": "2018-08-31 10:28:00", "month": "08", "high": 56.8355, "low": 56.8355, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.8355, "open": 56.8355, "day": "31"} +{"volume": 754, "symbol": "AYX", "ts": "2018-08-31 10:29:00", "month": "08", "high": 56.793, "low": 56.793, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.793, "open": 56.793, "day": "31"} +{"volume": 690, "symbol": "IRMD", "ts": "2018-08-31 10:03:00", "month": "08", "high": 26.65, "low": 26.65, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.65, "open": 26.65, "day": "31"} +{"volume": 698, "symbol": "IRMD", "ts": "2018-08-31 10:14:00", "month": "08", "high": 26.65, "low": 26.65, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.65, "open": 26.65, "day": "31"} +{"volume": 100, "symbol": "IRMD", "ts": "2018-08-31 10:20:00", "month": "08", "high": 26.725, "low": 26.725, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.725, "open": 26.725, "day": "31"} +{"volume": 465, "symbol": "IRMD", "ts": "2018-08-31 10:21:00", "month": "08", "high": 26.875, "low": 26.875, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.875, "open": 26.875, "day": "31"} +{"volume": 705, "symbol": "IRMD", "ts": "2018-08-31 10:24:00", "month": "08", "high": 26.75, "low": 26.75, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.75, "open": 26.75, "day": "31"} +{"volume": 1134, "symbol": "IRMD", "ts": "2018-08-31 10:28:00", "month": "08", "high": 26.675, "low": 26.675, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.675, "open": 26.675, "day": "31"} +{"volume": 3470, "symbol": "SEAS", "ts": "2018-08-31 10:01:00", "month": "08", "high": 28.985, "low": 28.985, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.985, "open": 28.985, "day": "31"} +{"volume": 934, "symbol": "SEAS", "ts": "2018-08-31 10:02:00", "month": "08", "high": 28.985, "low": 28.985, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.985, "open": 28.985, "day": "31"} +{"volume": 50872, "symbol": "SEAS", "ts": "2018-08-31 10:03:00", "month": "08", "high": 28.95, "low": 28.95, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.95, "open": 28.95, "day": "31"} +{"volume": 33288, "symbol": "SEAS", "ts": "2018-08-31 10:04:00", "month": "08", "high": 28.93, "low": 28.92, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.92, "open": 28.93, "day": "31"} +{"volume": 6748, "symbol": "SEAS", "ts": "2018-08-31 10:05:00", "month": "08", "high": 28.92, "low": 28.915, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.92, "open": 28.915, "day": "31"} +{"volume": 3999, "symbol": "SEAS", "ts": "2018-08-31 10:06:00", "month": "08", "high": 28.95, "low": 28.95, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.95, "open": 28.95, "day": "31"} +{"volume": 2403, "symbol": "SEAS", "ts": "2018-08-31 10:07:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 5722, "symbol": "SEAS", "ts": "2018-08-31 10:09:00", "month": "08", "high": 29.0, "low": 28.99, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.0, "open": 28.99, "day": "31"} +{"volume": 3800, "symbol": "SEAS", "ts": "2018-08-31 10:10:00", "month": "08", "high": 29.0, "low": 29.0, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.0, "open": 29.0, "day": "31"} +{"volume": 6484, "symbol": "SEAS", "ts": "2018-08-31 10:11:00", "month": "08", "high": 29.02, "low": 29.02, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.02, "open": 29.02, "day": "31"} +{"volume": 2954, "symbol": "SEAS", "ts": "2018-08-31 10:12:00", "month": "08", "high": 29.08, "low": 29.08, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.08, "open": 29.08, "day": "31"} +{"volume": 2486, "symbol": "SEAS", "ts": "2018-08-31 10:13:00", "month": "08", "high": 29.09, "low": 29.09, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.09, "open": 29.09, "day": "31"} +{"volume": 2843, "symbol": "SEAS", "ts": "2018-08-31 10:14:00", "month": "08", "high": 29.08, "low": 29.08, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.08, "open": 29.08, "day": "31"} +{"volume": 2425, "symbol": "SEAS", "ts": "2018-08-31 10:15:00", "month": "08", "high": 29.06, "low": 29.06, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.06, "open": 29.06, "day": "31"} +{"volume": 1995, "symbol": "SEAS", "ts": "2018-08-31 10:17:00", "month": "08", "high": 29.04, "low": 29.04, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.04, "open": 29.04, "day": "31"} +{"volume": 805, "symbol": "SEAS", "ts": "2018-08-31 10:18:00", "month": "08", "high": 29.04, "low": 29.04, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.04, "open": 29.04, "day": "31"} +{"volume": 2744, "symbol": "SEAS", "ts": "2018-08-31 10:19:00", "month": "08", "high": 29.03, "low": 29.03, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.03, "open": 29.03, "day": "31"} +{"volume": 1007, "symbol": "SEAS", "ts": "2018-08-31 10:20:00", "month": "08", "high": 29.03, "low": 29.03, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.03, "open": 29.03, "day": "31"} +{"volume": 5151, "symbol": "SEAS", "ts": "2018-08-31 10:22:00", "month": "08", "high": 29.05, "low": 29.05, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.05, "open": 29.05, "day": "31"} +{"volume": 4296, "symbol": "SEAS", "ts": "2018-08-31 10:23:00", "month": "08", "high": 29.1, "low": 29.1, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.1, "open": 29.1, "day": "31"} +{"volume": 5857, "symbol": "SEAS", "ts": "2018-08-31 10:24:00", "month": "08", "high": 29.165, "low": 29.165, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.165, "open": 29.165, "day": "31"} +{"volume": 5146, "symbol": "SEAS", "ts": "2018-08-31 10:25:00", "month": "08", "high": 29.18, "low": 29.18, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.18, "open": 29.18, "day": "31"} +{"volume": 24197, "symbol": "SEAS", "ts": "2018-08-31 10:26:00", "month": "08", "high": 29.18, "low": 29.18, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.18, "open": 29.18, "day": "31"} +{"volume": 3750, "symbol": "SEAS", "ts": "2018-08-31 10:27:00", "month": "08", "high": 29.12, "low": 29.12, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.12, "open": 29.12, "day": "31"} +{"volume": 1885, "symbol": "SEAS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 29.125, "low": 29.125, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.125, "open": 29.125, "day": "31"} +{"volume": 4789, "symbol": "SEAS", "ts": "2018-08-31 10:29:00", "month": "08", "high": 29.13, "low": 29.13, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.13, "open": 29.13, "day": "31"} +{"volume": 4531, "symbol": "RIBT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 3.24, "low": 3.24, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.24, "open": 3.24, "day": "31"} +{"volume": 1947, "symbol": "RIBT", "ts": "2018-08-31 10:03:00", "month": "08", "high": 3.25, "low": 3.24, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.24, "open": 3.25, "day": "31"} +{"volume": 1919, "symbol": "RIBT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 3288, "symbol": "RIBT", "ts": "2018-08-31 10:15:00", "month": "08", "high": 3.22, "low": 3.22, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.22, "open": 3.22, "day": "31"} +{"volume": 3693, "symbol": "RIBT", "ts": "2018-08-31 10:17:00", "month": "08", "high": 3.2025, "low": 3.2025, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.2025, "open": 3.2025, "day": "31"} +{"volume": 1346, "symbol": "RIBT", "ts": "2018-08-31 10:19:00", "month": "08", "high": 3.2362, "low": 3.2362, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.2362, "open": 3.2362, "day": "31"} +{"volume": 15205, "symbol": "RIBT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 3.249, "low": 3.249, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.249, "open": 3.249, "day": "31"} +{"volume": 23367, "symbol": "RIBT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 4815, "symbol": "RIBT", "ts": "2018-08-31 10:23:00", "month": "08", "high": 3.25, "low": 3.25, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.25, "day": "31"} +{"volume": 6100, "symbol": "RIBT", "ts": "2018-08-31 10:29:00", "month": "08", "high": 3.27, "low": 3.27, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.27, "open": 3.27, "day": "31"} +{"volume": 836, "symbol": "NSP", "ts": "2018-08-31 10:01:00", "month": "08", "high": 118.45, "low": 118.45, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.45, "open": 118.45, "day": "31"} +{"volume": 416, "symbol": "NSP", "ts": "2018-08-31 10:03:00", "month": "08", "high": 118.2, "low": 118.2, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.2, "open": 118.2, "day": "31"} +{"volume": 790, "symbol": "NSP", "ts": "2018-08-31 10:07:00", "month": "08", "high": 118.3, "low": 118.3, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.3, "open": 118.3, "day": "31"} +{"volume": 329, "symbol": "NSP", "ts": "2018-08-31 10:09:00", "month": "08", "high": 118.2, "low": 118.2, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.2, "open": 118.2, "day": "31"} +{"volume": 1108, "symbol": "NSP", "ts": "2018-08-31 10:10:00", "month": "08", "high": 117.95, "low": 117.95, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 117.95, "open": 117.95, "day": "31"} +{"volume": 435, "symbol": "NSP", "ts": "2018-08-31 10:14:00", "month": "08", "high": 118.05, "low": 118.05, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.05, "open": 118.05, "day": "31"} +{"volume": 1502, "symbol": "NSP", "ts": "2018-08-31 10:16:00", "month": "08", "high": 118.15, "low": 118.15, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.15, "open": 118.15, "day": "31"} +{"volume": 531, "symbol": "NSP", "ts": "2018-08-31 10:19:00", "month": "08", "high": 118.2, "low": 118.2, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.2, "open": 118.2, "day": "31"} +{"volume": 582, "symbol": "NSP", "ts": "2018-08-31 10:21:00", "month": "08", "high": 118.225, "low": 118.05, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.05, "open": 118.225, "day": "31"} +{"volume": 138, "symbol": "NSP", "ts": "2018-08-31 10:24:00", "month": "08", "high": 118.15, "low": 118.15, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.15, "open": 118.15, "day": "31"} +{"volume": 809, "symbol": "NSP", "ts": "2018-08-31 10:25:00", "month": "08", "high": 118.0, "low": 118.0, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.0, "open": 118.0, "day": "31"} +{"volume": 531, "symbol": "NSP", "ts": "2018-08-31 10:27:00", "month": "08", "high": 117.95, "low": 117.95, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 117.95, "open": 117.95, "day": "31"} +{"volume": 282, "symbol": "GDS", "ts": "2018-08-31 10:01:00", "month": "08", "high": 37.69, "low": 37.69, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.69, "open": 37.69, "day": "31"} +{"volume": 3282, "symbol": "GDS", "ts": "2018-08-31 10:02:00", "month": "08", "high": 37.695, "low": 37.69, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.69, "open": 37.695, "day": "31"} +{"volume": 803, "symbol": "GDS", "ts": "2018-08-31 10:03:00", "month": "08", "high": 37.7, "low": 37.7, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.7, "open": 37.7, "day": "31"} +{"volume": 3330, "symbol": "GDS", "ts": "2018-08-31 10:05:00", "month": "08", "high": 37.629, "low": 37.629, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.629, "open": 37.629, "day": "31"} +{"volume": 3477, "symbol": "GDS", "ts": "2018-08-31 10:07:00", "month": "08", "high": 37.41, "low": 37.4, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.41, "open": 37.4, "day": "31"} +{"volume": 1843, "symbol": "GDS", "ts": "2018-08-31 10:09:00", "month": "08", "high": 37.415, "low": 37.415, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.415, "open": 37.415, "day": "31"} +{"volume": 396, "symbol": "GDS", "ts": "2018-08-31 10:12:00", "month": "08", "high": 37.37, "low": 37.37, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.37, "open": 37.37, "day": "31"} +{"volume": 1180, "symbol": "GDS", "ts": "2018-08-31 10:13:00", "month": "08", "high": 37.46, "low": 37.46, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.46, "open": 37.46, "day": "31"} +{"volume": 1545, "symbol": "GDS", "ts": "2018-08-31 10:14:00", "month": "08", "high": 37.515, "low": 37.515, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.515, "open": 37.515, "day": "31"} +{"volume": 1889, "symbol": "GDS", "ts": "2018-08-31 10:16:00", "month": "08", "high": 37.515, "low": 37.515, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.515, "open": 37.515, "day": "31"} +{"volume": 446, "symbol": "GDS", "ts": "2018-08-31 10:18:00", "month": "08", "high": 37.569, "low": 37.569, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.569, "open": 37.569, "day": "31"} +{"volume": 761, "symbol": "GDS", "ts": "2018-08-31 10:19:00", "month": "08", "high": 37.7, "low": 37.7, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.7, "open": 37.7, "day": "31"} +{"volume": 1839, "symbol": "GDS", "ts": "2018-08-31 10:21:00", "month": "08", "high": 37.68, "low": 37.68, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.68, "open": 37.68, "day": "31"} +{"volume": 3035, "symbol": "GDS", "ts": "2018-08-31 10:22:00", "month": "08", "high": 37.72, "low": 37.65, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.72, "open": 37.65, "day": "31"} +{"volume": 8804, "symbol": "GDS", "ts": "2018-08-31 10:24:00", "month": "08", "high": 37.93, "low": 37.89, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.93, "open": 37.89, "day": "31"} +{"volume": 7960, "symbol": "GDS", "ts": "2018-08-31 10:26:00", "month": "08", "high": 37.97, "low": 37.96, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.96, "open": 37.97, "day": "31"} +{"volume": 9643, "symbol": "GDS", "ts": "2018-08-31 10:27:00", "month": "08", "high": 38.13, "low": 37.99, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.13, "open": 37.99, "day": "31"} +{"volume": 3487, "symbol": "GDS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 38.16, "low": 38.16, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.16, "open": 38.16, "day": "31"} +{"volume": 7575, "symbol": "GDS", "ts": "2018-08-31 10:29:00", "month": "08", "high": 38.15, "low": 38.12, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.12, "open": 38.12, "day": "31"} +{"volume": 937, "symbol": "VICR", "ts": "2018-08-31 10:02:00", "month": "08", "high": 62.05, "low": 62.0, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.0, "open": 62.05, "day": "31"} +{"volume": 1230, "symbol": "VICR", "ts": "2018-08-31 10:07:00", "month": "08", "high": 62.25, "low": 62.25, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.25, "open": 62.25, "day": "31"} +{"volume": 410, "symbol": "VICR", "ts": "2018-08-31 10:10:00", "month": "08", "high": 62.15, "low": 62.15, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.15, "open": 62.15, "day": "31"} +{"volume": 7902, "symbol": "VICR", "ts": "2018-08-31 10:11:00", "month": "08", "high": 62.25, "low": 62.0, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.0, "open": 62.25, "day": "31"} +{"volume": 700, "symbol": "VICR", "ts": "2018-08-31 10:13:00", "month": "08", "high": 61.9, "low": 61.9, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.9, "open": 61.9, "day": "31"} +{"volume": 700, "symbol": "VICR", "ts": "2018-08-31 10:17:00", "month": "08", "high": 62.05, "low": 62.05, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.05, "open": 62.05, "day": "31"} +{"volume": 2620, "symbol": "VICR", "ts": "2018-08-31 10:18:00", "month": "08", "high": 61.95, "low": 61.95, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.95, "open": 61.95, "day": "31"} +{"volume": 1185, "symbol": "VICR", "ts": "2018-08-31 10:20:00", "month": "08", "high": 61.6, "low": 61.6, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.6, "open": 61.6, "day": "31"} +{"volume": 440, "symbol": "VICR", "ts": "2018-08-31 10:21:00", "month": "08", "high": 61.65, "low": 61.65, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.65, "open": 61.65, "day": "31"} +{"volume": 114, "symbol": "VICR", "ts": "2018-08-31 10:23:00", "month": "08", "high": 61.55, "low": 61.55, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 122, "symbol": "VICR", "ts": "2018-08-31 10:26:00", "month": "08", "high": 61.6, "low": 61.6, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.6, "open": 61.6, "day": "31"} +{"volume": 343, "symbol": "VICR", "ts": "2018-08-31 10:28:00", "month": "08", "high": 61.575, "low": 61.575, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.575, "open": 61.575, "day": "31"} +{"volume": 468544, "symbol": "CRON", "ts": "2018-08-31 10:01:00", "month": "08", "high": 10.25, "low": 10.06, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.25, "open": 10.0801, "day": "31"} +{"volume": 617968, "symbol": "CRON", "ts": "2018-08-31 10:02:00", "month": "08", "high": 10.2801, "low": 10.051, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.1188, "open": 10.2481, "day": "31"} +{"volume": 271181, "symbol": "CRON", "ts": "2018-08-31 10:03:00", "month": "08", "high": 10.25, "low": 10.1099, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.25, "open": 10.139, "day": "31"} +{"volume": 431896, "symbol": "CRON", "ts": "2018-08-31 10:04:00", "month": "08", "high": 10.2708, "low": 10.119, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.2, "open": 10.25, "day": "31"} +{"volume": 349857, "symbol": "CRON", "ts": "2018-08-31 10:05:00", "month": "08", "high": 10.22, "low": 10.0689, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.18, "open": 10.22, "day": "31"} +{"volume": 168257, "symbol": "CRON", "ts": "2018-08-31 10:06:00", "month": "08", "high": 10.1956, "low": 10.1101, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.1101, "open": 10.1956, "day": "31"} +{"volume": 470405, "symbol": "CRON", "ts": "2018-08-31 10:07:00", "month": "08", "high": 10.1137, "low": 9.9201, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.995, "open": 10.09, "day": "31"} +{"volume": 351333, "symbol": "CRON", "ts": "2018-08-31 10:08:00", "month": "08", "high": 9.9987, "low": 9.86, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.98, "day": "31"} +{"volume": 171990, "symbol": "CRON", "ts": "2018-08-31 10:09:00", "month": "08", "high": 9.99, "low": 9.9301, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9895, "open": 9.94, "day": "31"} +{"volume": 287830, "symbol": "CRON", "ts": "2018-08-31 10:10:00", "month": "08", "high": 10.065, "low": 9.975, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0618, "open": 9.98, "day": "31"} +{"volume": 258183, "symbol": "CRON", "ts": "2018-08-31 10:11:00", "month": "08", "high": 10.05, "low": 9.93, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9401, "open": 10.05, "day": "31"} +{"volume": 202107, "symbol": "CRON", "ts": "2018-08-31 10:12:00", "month": "08", "high": 9.94, "low": 9.885, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9162, "open": 9.94, "day": "31"} +{"volume": 114771, "symbol": "CRON", "ts": "2018-08-31 10:13:00", "month": "08", "high": 9.94, "low": 9.9, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9201, "open": 9.94, "day": "31"} +{"volume": 311131, "symbol": "CRON", "ts": "2018-08-31 10:14:00", "month": "08", "high": 10.035, "low": 9.92, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9901, "open": 9.9299, "day": "31"} +{"volume": 138710, "symbol": "CRON", "ts": "2018-08-31 10:15:00", "month": "08", "high": 10.0199, "low": 9.9701, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0, "open": 9.9701, "day": "31"} +{"volume": 153111, "symbol": "CRON", "ts": "2018-08-31 10:16:00", "month": "08", "high": 10.0092, "low": 9.901, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.916, "open": 10.0092, "day": "31"} +{"volume": 181648, "symbol": "CRON", "ts": "2018-08-31 10:17:00", "month": "08", "high": 9.99, "low": 9.92, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.99, "open": 9.929, "day": "31"} +{"volume": 72515, "symbol": "CRON", "ts": "2018-08-31 10:18:00", "month": "08", "high": 9.99, "low": 9.95, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.99, "open": 9.95, "day": "31"} +{"volume": 258748, "symbol": "CRON", "ts": "2018-08-31 10:19:00", "month": "08", "high": 10.07, "low": 9.95, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0432, "open": 9.97, "day": "31"} +{"volume": 266651, "symbol": "CRON", "ts": "2018-08-31 10:20:00", "month": "08", "high": 10.1, "low": 10.0201, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.09, "open": 10.06, "day": "31"} +{"volume": 343155, "symbol": "CRON", "ts": "2018-08-31 10:21:00", "month": "08", "high": 10.14, "low": 9.9701, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9701, "open": 10.1, "day": "31"} +{"volume": 264335, "symbol": "CRON", "ts": "2018-08-31 10:22:00", "month": "08", "high": 10.08, "low": 9.97, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0299, "open": 9.98, "day": "31"} +{"volume": 130079, "symbol": "CRON", "ts": "2018-08-31 10:23:00", "month": "08", "high": 10.08, "low": 10.02, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0501, "open": 10.02, "day": "31"} +{"volume": 118722, "symbol": "CRON", "ts": "2018-08-31 10:24:00", "month": "08", "high": 10.05, "low": 10.0101, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0398, "open": 10.05, "day": "31"} +{"volume": 168621, "symbol": "CRON", "ts": "2018-08-31 10:25:00", "month": "08", "high": 10.0201, "low": 9.94, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 10.0201, "day": "31"} +{"volume": 542578, "symbol": "CRON", "ts": "2018-08-31 10:26:00", "month": "08", "high": 9.93, "low": 9.705, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.7899, "open": 9.92, "day": "31"} +{"volume": 350188, "symbol": "CRON", "ts": "2018-08-31 10:27:00", "month": "08", "high": 9.8947, "low": 9.72, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.87, "open": 9.775, "day": "31"} +{"volume": 121307, "symbol": "CRON", "ts": "2018-08-31 10:28:00", "month": "08", "high": 9.88, "low": 9.8452, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.88, "open": 9.88, "day": "31"} +{"volume": 195779, "symbol": "CRON", "ts": "2018-08-31 10:29:00", "month": "08", "high": 9.959, "low": 9.88, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.8933, "open": 9.88, "day": "31"} +{"volume": 733, "symbol": "INS", "ts": "2018-08-31 10:12:00", "month": "08", "high": 13.0772, "low": 13.0772, "key": "INS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.0772, "open": 13.0772, "day": "31"} +{"volume": 7271, "symbol": "SRPT", "ts": "2018-08-31 10:01:00", "month": "08", "high": 139.0, "low": 138.865, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.865, "open": 138.95, "day": "31"} +{"volume": 1256, "symbol": "SRPT", "ts": "2018-08-31 10:03:00", "month": "08", "high": 138.821, "low": 138.821, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.821, "open": 138.821, "day": "31"} +{"volume": 1394, "symbol": "SRPT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 138.5877, "low": 138.5877, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.5877, "open": 138.5877, "day": "31"} +{"volume": 427, "symbol": "SRPT", "ts": "2018-08-31 10:06:00", "month": "08", "high": 138.4705, "low": 138.4705, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.4705, "open": 138.4705, "day": "31"} +{"volume": 8342, "symbol": "SRPT", "ts": "2018-08-31 10:08:00", "month": "08", "high": 138.32, "low": 137.92, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.92, "open": 138.32, "day": "31"} +{"volume": 1957, "symbol": "SRPT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 137.6, "low": 137.6, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.6, "open": 137.6, "day": "31"} +{"volume": 1672, "symbol": "SRPT", "ts": "2018-08-31 10:10:00", "month": "08", "high": 137.56, "low": 137.56, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.56, "open": 137.56, "day": "31"} +{"volume": 4705, "symbol": "SRPT", "ts": "2018-08-31 10:11:00", "month": "08", "high": 137.75, "low": 137.75, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.75, "open": 137.75, "day": "31"} +{"volume": 2109, "symbol": "SRPT", "ts": "2018-08-31 10:12:00", "month": "08", "high": 137.6924, "low": 137.6924, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.6924, "open": 137.6924, "day": "31"} +{"volume": 2839, "symbol": "SRPT", "ts": "2018-08-31 10:14:00", "month": "08", "high": 137.98, "low": 137.87, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.98, "open": 137.87, "day": "31"} +{"volume": 1401, "symbol": "SRPT", "ts": "2018-08-31 10:16:00", "month": "08", "high": 138.1, "low": 138.1, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.1, "open": 138.1, "day": "31"} +{"volume": 202, "symbol": "SRPT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 138.2156, "low": 138.2156, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.2156, "open": 138.2156, "day": "31"} +{"volume": 2028, "symbol": "SRPT", "ts": "2018-08-31 10:20:00", "month": "08", "high": 137.735, "low": 137.735, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.735, "open": 137.735, "day": "31"} +{"volume": 2148, "symbol": "SRPT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 137.89, "low": 137.89, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.89, "open": 137.89, "day": "31"} +{"volume": 5660, "symbol": "SRPT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 138.23, "low": 138.21, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.23, "open": 138.21, "day": "31"} +{"volume": 551, "symbol": "SRPT", "ts": "2018-08-31 10:24:00", "month": "08", "high": 138.18, "low": 138.18, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.18, "open": 138.18, "day": "31"} +{"volume": 1480, "symbol": "SRPT", "ts": "2018-08-31 10:26:00", "month": "08", "high": 137.891, "low": 137.891, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.891, "open": 137.891, "day": "31"} +{"volume": 2985, "symbol": "SRPT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 138.0, "low": 137.74, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.74, "open": 138.0, "day": "31"} +{"volume": 2113, "symbol": "HEAR", "ts": "2018-08-31 10:01:00", "month": "08", "high": 23.67, "low": 23.67, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.67, "open": 23.67, "day": "31"} +{"volume": 5421, "symbol": "HEAR", "ts": "2018-08-31 10:02:00", "month": "08", "high": 23.669, "low": 23.669, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.669, "open": 23.669, "day": "31"} +{"volume": 2180, "symbol": "HEAR", "ts": "2018-08-31 10:03:00", "month": "08", "high": 23.6447, "low": 23.6447, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.6447, "open": 23.6447, "day": "31"} +{"volume": 7798, "symbol": "HEAR", "ts": "2018-08-31 10:05:00", "month": "08", "high": 23.6018, "low": 23.57, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.57, "open": 23.6018, "day": "31"} +{"volume": 2368, "symbol": "HEAR", "ts": "2018-08-31 10:06:00", "month": "08", "high": 23.5, "low": 23.5, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.5, "open": 23.5, "day": "31"} +{"volume": 5275, "symbol": "HEAR", "ts": "2018-08-31 10:07:00", "month": "08", "high": 23.42, "low": 23.4089, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.42, "open": 23.4089, "day": "31"} +{"volume": 3997, "symbol": "HEAR", "ts": "2018-08-31 10:08:00", "month": "08", "high": 23.6, "low": 23.6, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.6, "open": 23.6, "day": "31"} +{"volume": 5150, "symbol": "HEAR", "ts": "2018-08-31 10:10:00", "month": "08", "high": 23.4428, "low": 23.43, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.43, "open": 23.4428, "day": "31"} +{"volume": 2906, "symbol": "HEAR", "ts": "2018-08-31 10:11:00", "month": "08", "high": 23.3977, "low": 23.3977, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.3977, "open": 23.3977, "day": "31"} +{"volume": 1826, "symbol": "HEAR", "ts": "2018-08-31 10:13:00", "month": "08", "high": 23.24, "low": 23.24, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.24, "open": 23.24, "day": "31"} +{"volume": 4526, "symbol": "HEAR", "ts": "2018-08-31 10:15:00", "month": "08", "high": 23.32, "low": 23.288, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.32, "open": 23.288, "day": "31"} +{"volume": 2267, "symbol": "HEAR", "ts": "2018-08-31 10:17:00", "month": "08", "high": 23.39, "low": 23.39, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.39, "open": 23.39, "day": "31"} +{"volume": 2336, "symbol": "HEAR", "ts": "2018-08-31 10:18:00", "month": "08", "high": 23.47, "low": 23.47, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.47, "open": 23.47, "day": "31"} +{"volume": 4492, "symbol": "HEAR", "ts": "2018-08-31 10:19:00", "month": "08", "high": 23.429, "low": 23.429, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.429, "open": 23.429, "day": "31"} +{"volume": 2590, "symbol": "HEAR", "ts": "2018-08-31 10:21:00", "month": "08", "high": 23.39, "low": 23.39, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.39, "open": 23.39, "day": "31"} +{"volume": 4730, "symbol": "HEAR", "ts": "2018-08-31 10:22:00", "month": "08", "high": 23.3965, "low": 23.3965, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.3965, "open": 23.3965, "day": "31"} +{"volume": 13077, "symbol": "HEAR", "ts": "2018-08-31 10:24:00", "month": "08", "high": 23.3, "low": 23.1857, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.1857, "open": 23.3, "day": "31"} +{"volume": 3266, "symbol": "HEAR", "ts": "2018-08-31 10:26:00", "month": "08", "high": 23.12, "low": 23.12, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.12, "open": 23.12, "day": "31"} +{"volume": 7617, "symbol": "HEAR", "ts": "2018-08-31 10:27:00", "month": "08", "high": 23.184, "low": 23.184, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.184, "open": 23.184, "day": "31"} +{"volume": 2174, "symbol": "HEAR", "ts": "2018-08-31 10:28:00", "month": "08", "high": 23.239, "low": 23.239, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.239, "open": 23.239, "day": "31"} +{"volume": 3572, "symbol": "HEAR", "ts": "2018-08-31 10:29:00", "month": "08", "high": 23.153, "low": 23.153, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.153, "open": 23.153, "day": "31"} +{"volume": 1359, "symbol": "BLFS", "ts": "2018-08-31 10:03:00", "month": "08", "high": 24.259, "low": 24.2, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.2, "open": 24.259, "day": "31"} +{"volume": 365, "symbol": "BLFS", "ts": "2018-08-31 10:05:00", "month": "08", "high": 24.199, "low": 24.199, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.199, "open": 24.199, "day": "31"} +{"volume": 1540, "symbol": "BLFS", "ts": "2018-08-31 10:09:00", "month": "08", "high": 24.29, "low": 24.29, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.29, "open": 24.29, "day": "31"} +{"volume": 3932, "symbol": "BLFS", "ts": "2018-08-31 10:12:00", "month": "08", "high": 24.3015, "low": 24.2263, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.3015, "open": 24.2263, "day": "31"} +{"volume": 4105, "symbol": "BLFS", "ts": "2018-08-31 10:16:00", "month": "08", "high": 24.1859, "low": 24.1859, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.1859, "open": 24.1859, "day": "31"} +{"volume": 604, "symbol": "BLFS", "ts": "2018-08-31 10:20:00", "month": "08", "high": 24.0778, "low": 24.0778, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.0778, "open": 24.0778, "day": "31"} +{"volume": 310, "symbol": "BLFS", "ts": "2018-08-31 10:24:00", "month": "08", "high": 24.021, "low": 24.021, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.021, "open": 24.021, "day": "31"} +{"volume": 329, "symbol": "BLFS", "ts": "2018-08-31 10:27:00", "month": "08", "high": 24.159, "low": 24.159, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.159, "open": 24.159, "day": "31"} +{"volume": 241, "symbol": "BLFS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 24.19, "low": 24.19, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.19, "open": 24.19, "day": "31"} +{"volume": 400, "symbol": "LGCYO", "ts": "2018-08-31 10:03:00", "month": "08", "high": 15.41, "low": 15.41, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.41, "open": 15.41, "day": "31"} +{"volume": 1050, "symbol": "LGCYO", "ts": "2018-08-31 10:09:00", "month": "08", "high": 15.51, "low": 15.51, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.51, "open": 15.51, "day": "31"} +{"volume": 6650, "symbol": "LGCYO", "ts": "2018-08-31 10:10:00", "month": "08", "high": 15.5, "low": 15.5, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.5, "open": 15.5, "day": "31"} +{"volume": 700, "symbol": "LGCYO", "ts": "2018-08-31 10:11:00", "month": "08", "high": 15.5, "low": 15.5, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.5, "open": 15.5, "day": "31"} +{"volume": 5000, "symbol": "LGCYO", "ts": "2018-08-31 10:20:00", "month": "08", "high": 15.5, "low": 15.5, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.5, "open": 15.5, "day": "31"} +{"volume": 100, "symbol": "NIHD", "ts": "2018-08-31 10:02:00", "month": "08", "high": 5.68, "low": 5.68, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.68, "open": 5.68, "day": "31"} +{"volume": 4847, "symbol": "NIHD", "ts": "2018-08-31 10:03:00", "month": "08", "high": 5.7, "low": 5.7, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.7, "open": 5.7, "day": "31"} +{"volume": 2071, "symbol": "NIHD", "ts": "2018-08-31 10:04:00", "month": "08", "high": 5.705, "low": 5.705, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.705, "open": 5.705, "day": "31"} +{"volume": 1120, "symbol": "NIHD", "ts": "2018-08-31 10:05:00", "month": "08", "high": 5.706, "low": 5.706, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.706, "open": 5.706, "day": "31"} +{"volume": 2057, "symbol": "NIHD", "ts": "2018-08-31 10:07:00", "month": "08", "high": 5.71, "low": 5.71, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.71, "open": 5.71, "day": "31"} +{"volume": 1739, "symbol": "NIHD", "ts": "2018-08-31 10:08:00", "month": "08", "high": 5.72, "low": 5.72, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.72, "open": 5.72, "day": "31"} +{"volume": 1389, "symbol": "NIHD", "ts": "2018-08-31 10:12:00", "month": "08", "high": 5.72, "low": 5.72, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.72, "open": 5.72, "day": "31"} +{"volume": 2269, "symbol": "NIHD", "ts": "2018-08-31 10:14:00", "month": "08", "high": 5.71, "low": 5.71, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.71, "open": 5.71, "day": "31"} +{"volume": 2412, "symbol": "NIHD", "ts": "2018-08-31 10:15:00", "month": "08", "high": 5.715, "low": 5.715, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.715, "open": 5.715, "day": "31"} +{"volume": 100, "symbol": "NIHD", "ts": "2018-08-31 10:19:00", "month": "08", "high": 5.715, "low": 5.715, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.715, "open": 5.715, "day": "31"} +{"volume": 6607, "symbol": "NIHD", "ts": "2018-08-31 10:20:00", "month": "08", "high": 5.7, "low": 5.7, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.7, "open": 5.7, "day": "31"} +{"volume": 526, "symbol": "NIHD", "ts": "2018-08-31 10:21:00", "month": "08", "high": 5.695, "low": 5.695, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.695, "open": 5.695, "day": "31"} +{"volume": 309, "symbol": "NIHD", "ts": "2018-08-31 10:24:00", "month": "08", "high": 5.695, "low": 5.695, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.695, "open": 5.695, "day": "31"} +{"volume": 3867, "symbol": "NIHD", "ts": "2018-08-31 10:26:00", "month": "08", "high": 5.685, "low": 5.685, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.685, "open": 5.685, "day": "31"} +{"volume": 1156, "symbol": "NIHD", "ts": "2018-08-31 10:27:00", "month": "08", "high": 5.68, "low": 5.68, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.68, "open": 5.68, "day": "31"} +{"volume": 2125, "symbol": "NIHD", "ts": "2018-08-31 10:29:00", "month": "08", "high": 5.67, "low": 5.67, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.67, "open": 5.67, "day": "31"} +{"volume": 1797, "symbol": "NEPT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 3.9927, "low": 3.9927, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.9927, "open": 3.9927, "day": "31"} +{"volume": 3582, "symbol": "NEPT", "ts": "2018-08-31 10:03:00", "month": "08", "high": 4.03, "low": 4.03, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.03, "open": 4.03, "day": "31"} +{"volume": 3554, "symbol": "NEPT", "ts": "2018-08-31 10:04:00", "month": "08", "high": 4.025, "low": 4.025, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.025, "open": 4.025, "day": "31"} +{"volume": 3319, "symbol": "NEPT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 4.001, "low": 4.001, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.001, "open": 4.001, "day": "31"} +{"volume": 9564, "symbol": "NEPT", "ts": "2018-08-31 10:06:00", "month": "08", "high": 4.0184, "low": 4.0184, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0184, "open": 4.0184, "day": "31"} +{"volume": 6333, "symbol": "NEPT", "ts": "2018-08-31 10:07:00", "month": "08", "high": 3.9952, "low": 3.9952, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.9952, "open": 3.9952, "day": "31"} +{"volume": 6259, "symbol": "NEPT", "ts": "2018-08-31 10:08:00", "month": "08", "high": 3.99, "low": 3.99, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.99, "open": 3.99, "day": "31"} +{"volume": 2954, "symbol": "NEPT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 3.9889, "low": 3.9889, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.9889, "open": 3.9889, "day": "31"} +{"volume": 6954, "symbol": "NEPT", "ts": "2018-08-31 10:10:00", "month": "08", "high": 3.97, "low": 3.97, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.97, "open": 3.97, "day": "31"} +{"volume": 9789, "symbol": "NEPT", "ts": "2018-08-31 10:11:00", "month": "08", "high": 3.97, "low": 3.97, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.97, "open": 3.97, "day": "31"} +{"volume": 2619, "symbol": "NEPT", "ts": "2018-08-31 10:12:00", "month": "08", "high": 3.9792, "low": 3.9792, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.9792, "open": 3.9792, "day": "31"} +{"volume": 9477, "symbol": "NEPT", "ts": "2018-08-31 10:14:00", "month": "08", "high": 3.98, "low": 3.98, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.98, "open": 3.98, "day": "31"} +{"volume": 12386, "symbol": "NEPT", "ts": "2018-08-31 10:16:00", "month": "08", "high": 3.97, "low": 3.97, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.97, "open": 3.97, "day": "31"} +{"volume": 9760, "symbol": "NEPT", "ts": "2018-08-31 10:17:00", "month": "08", "high": 3.9616, "low": 3.9616, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.9616, "open": 3.9616, "day": "31"} +{"volume": 1466, "symbol": "NEPT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 3.98, "low": 3.98, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.98, "open": 3.98, "day": "31"} +{"volume": 1328, "symbol": "NEPT", "ts": "2018-08-31 10:20:00", "month": "08", "high": 3.979, "low": 3.979, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.979, "open": 3.979, "day": "31"} +{"volume": 11458, "symbol": "NEPT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 3.99, "low": 3.99, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.99, "open": 3.99, "day": "31"} +{"volume": 55112, "symbol": "NEPT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 3.9893, "low": 3.9893, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.9893, "open": 3.9893, "day": "31"} +{"volume": 914, "symbol": "NEPT", "ts": "2018-08-31 10:24:00", "month": "08", "high": 3.994, "low": 3.994, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.994, "open": 3.994, "day": "31"} +{"volume": 203, "symbol": "NEPT", "ts": "2018-08-31 10:25:00", "month": "08", "high": 4.0, "low": 4.0, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0, "open": 4.0, "day": "31"} +{"volume": 561, "symbol": "NEPT", "ts": "2018-08-31 10:26:00", "month": "08", "high": 3.99, "low": 3.99, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.99, "open": 3.99, "day": "31"} +{"volume": 1784, "symbol": "NEPT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 3.98, "low": 3.98, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.98, "open": 3.98, "day": "31"} +{"volume": 995, "symbol": "INGN", "ts": "2018-08-31 10:04:00", "month": "08", "high": 260.91, "low": 260.91, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 260.91, "open": 260.91, "day": "31"} +{"volume": 256, "symbol": "INGN", "ts": "2018-08-31 10:06:00", "month": "08", "high": 260.97, "low": 260.97, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 260.97, "open": 260.97, "day": "31"} +{"volume": 202, "symbol": "INGN", "ts": "2018-08-31 10:09:00", "month": "08", "high": 261.21, "low": 261.21, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 261.21, "open": 261.21, "day": "31"} +{"volume": 460, "symbol": "INGN", "ts": "2018-08-31 10:10:00", "month": "08", "high": 261.0642, "low": 261.0642, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 261.0642, "open": 261.0642, "day": "31"} +{"volume": 1113, "symbol": "INGN", "ts": "2018-08-31 10:12:00", "month": "08", "high": 261.505, "low": 261.505, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 261.505, "open": 261.505, "day": "31"} +{"volume": 1873, "symbol": "INGN", "ts": "2018-08-31 10:14:00", "month": "08", "high": 262.02, "low": 262.02, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 262.02, "open": 262.02, "day": "31"} +{"volume": 559, "symbol": "INGN", "ts": "2018-08-31 10:16:00", "month": "08", "high": 263.09, "low": 263.09, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.09, "open": 263.09, "day": "31"} +{"volume": 870, "symbol": "INGN", "ts": "2018-08-31 10:19:00", "month": "08", "high": 263.3912, "low": 263.115, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.115, "open": 263.3912, "day": "31"} +{"volume": 579, "symbol": "INGN", "ts": "2018-08-31 10:27:00", "month": "08", "high": 263.131, "low": 263.131, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.131, "open": 263.131, "day": "31"} +{"volume": 716, "symbol": "INGN", "ts": "2018-08-31 10:28:00", "month": "08", "high": 263.57, "low": 263.57, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.57, "open": 263.57, "day": "31"} +{"volume": 291, "symbol": "INGN", "ts": "2018-08-31 10:29:00", "month": "08", "high": 263.355, "low": 263.355, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.355, "open": 263.355, "day": "31"} +{"volume": 125, "symbol": "GFN", "ts": "2018-08-31 10:01:00", "month": "08", "high": 13.45, "low": 13.45, "key": "GFN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.45, "open": 13.45, "day": "31"} +{"volume": 121, "symbol": "GFN", "ts": "2018-08-31 10:06:00", "month": "08", "high": 13.4, "low": 13.4, "key": "GFN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 200, "symbol": "GFN", "ts": "2018-08-31 10:21:00", "month": "08", "high": 13.45, "low": 13.45, "key": "GFN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.45, "open": 13.45, "day": "31"} +{"volume": 2700, "symbol": "ECYT", "ts": "2018-08-31 10:01:00", "month": "08", "high": 19.1, "low": 19.1, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.1, "open": 19.1, "day": "31"} +{"volume": 929, "symbol": "ECYT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 19.1, "low": 19.1, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.1, "open": 19.1, "day": "31"} +{"volume": 1470, "symbol": "ECYT", "ts": "2018-08-31 10:04:00", "month": "08", "high": 19.1, "low": 19.1, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.1, "open": 19.1, "day": "31"} +{"volume": 3000, "symbol": "ECYT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 19.26, "low": 19.26, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.26, "open": 19.26, "day": "31"} +{"volume": 1900, "symbol": "ECYT", "ts": "2018-08-31 10:06:00", "month": "08", "high": 19.34, "low": 19.34, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.34, "open": 19.34, "day": "31"} +{"volume": 2311, "symbol": "ECYT", "ts": "2018-08-31 10:07:00", "month": "08", "high": 19.3315, "low": 19.3315, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.3315, "open": 19.3315, "day": "31"} +{"volume": 3538, "symbol": "ECYT", "ts": "2018-08-31 10:08:00", "month": "08", "high": 19.28, "low": 19.28, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.28, "open": 19.28, "day": "31"} +{"volume": 2849, "symbol": "ECYT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 19.3, "low": 19.3, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.3, "open": 19.3, "day": "31"} +{"volume": 4415, "symbol": "ECYT", "ts": "2018-08-31 10:10:00", "month": "08", "high": 19.3, "low": 19.3, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.3, "open": 19.3, "day": "31"} +{"volume": 10697, "symbol": "ECYT", "ts": "2018-08-31 10:11:00", "month": "08", "high": 19.28, "low": 19.28, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.28, "open": 19.28, "day": "31"} +{"volume": 2985, "symbol": "ECYT", "ts": "2018-08-31 10:12:00", "month": "08", "high": 19.28, "low": 19.28, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.28, "open": 19.28, "day": "31"} +{"volume": 4031, "symbol": "ECYT", "ts": "2018-08-31 10:13:00", "month": "08", "high": 19.275, "low": 19.275, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.275, "open": 19.275, "day": "31"} +{"volume": 1523, "symbol": "ECYT", "ts": "2018-08-31 10:15:00", "month": "08", "high": 19.29, "low": 19.29, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.29, "open": 19.29, "day": "31"} +{"volume": 5101, "symbol": "ECYT", "ts": "2018-08-31 10:16:00", "month": "08", "high": 19.31, "low": 19.31, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.31, "open": 19.31, "day": "31"} +{"volume": 2782, "symbol": "ECYT", "ts": "2018-08-31 10:17:00", "month": "08", "high": 19.27, "low": 19.27, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.27, "open": 19.27, "day": "31"} +{"volume": 8533, "symbol": "ECYT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 19.215, "low": 19.2, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.2, "open": 19.215, "day": "31"} +{"volume": 4123, "symbol": "ECYT", "ts": "2018-08-31 10:19:00", "month": "08", "high": 19.22, "low": 19.22, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.22, "open": 19.22, "day": "31"} +{"volume": 1809, "symbol": "ECYT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 19.24, "low": 19.24, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.24, "open": 19.24, "day": "31"} +{"volume": 5770, "symbol": "ECYT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 19.29, "low": 19.2354, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.2354, "open": 19.29, "day": "31"} +{"volume": 3255, "symbol": "ECYT", "ts": "2018-08-31 10:23:00", "month": "08", "high": 19.26, "low": 19.26, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.26, "open": 19.26, "day": "31"} +{"volume": 6570, "symbol": "ECYT", "ts": "2018-08-31 10:24:00", "month": "08", "high": 19.24, "low": 19.24, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.24, "open": 19.24, "day": "31"} +{"volume": 1920, "symbol": "ECYT", "ts": "2018-08-31 10:26:00", "month": "08", "high": 19.25, "low": 19.25, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.25, "open": 19.25, "day": "31"} +{"volume": 2200, "symbol": "ECYT", "ts": "2018-08-31 10:27:00", "month": "08", "high": 19.29, "low": 19.29, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.29, "open": 19.29, "day": "31"} +{"volume": 3924, "symbol": "ECYT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 19.25, "low": 19.25, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.25, "open": 19.25, "day": "31"} +{"volume": 5293, "symbol": "ECYT", "ts": "2018-08-31 10:29:00", "month": "08", "high": 19.28, "low": 19.25, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.25, "open": 19.28, "day": "31"} +{"volume": 274136, "symbol": "AMD", "ts": "2018-08-31 10:01:00", "month": "08", "high": 24.8133, "low": 24.76, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8114, "open": 24.8, "day": "31"} +{"volume": 206113, "symbol": "AMD", "ts": "2018-08-31 10:02:00", "month": "08", "high": 24.811, "low": 24.765, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.789, "open": 24.8, "day": "31"} +{"volume": 348144, "symbol": "AMD", "ts": "2018-08-31 10:03:00", "month": "08", "high": 24.87, "low": 24.8, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.87, "open": 24.8, "day": "31"} +{"volume": 152650, "symbol": "AMD", "ts": "2018-08-31 10:04:00", "month": "08", "high": 24.86, "low": 24.84, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.845, "open": 24.86, "day": "31"} +{"volume": 232447, "symbol": "AMD", "ts": "2018-08-31 10:05:00", "month": "08", "high": 24.85, "low": 24.785, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.7927, "open": 24.85, "day": "31"} +{"volume": 216113, "symbol": "AMD", "ts": "2018-08-31 10:06:00", "month": "08", "high": 24.8562, "low": 24.8, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8562, "open": 24.8, "day": "31"} +{"volume": 445185, "symbol": "AMD", "ts": "2018-08-31 10:07:00", "month": "08", "high": 24.92, "low": 24.86, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9, "open": 24.86, "day": "31"} +{"volume": 295943, "symbol": "AMD", "ts": "2018-08-31 10:08:00", "month": "08", "high": 24.9399, "low": 24.91, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.935, "open": 24.91, "day": "31"} +{"volume": 407065, "symbol": "AMD", "ts": "2018-08-31 10:09:00", "month": "08", "high": 24.93, "low": 24.86, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8631, "open": 24.93, "day": "31"} +{"volume": 204296, "symbol": "AMD", "ts": "2018-08-31 10:10:00", "month": "08", "high": 24.925, "low": 24.87, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.895, "open": 24.87, "day": "31"} +{"volume": 139810, "symbol": "AMD", "ts": "2018-08-31 10:11:00", "month": "08", "high": 24.9, "low": 24.88, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8901, "open": 24.88, "day": "31"} +{"volume": 242036, "symbol": "AMD", "ts": "2018-08-31 10:12:00", "month": "08", "high": 24.88, "low": 24.84, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.85, "open": 24.88, "day": "31"} +{"volume": 179383, "symbol": "AMD", "ts": "2018-08-31 10:13:00", "month": "08", "high": 24.86, "low": 24.84, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.845, "open": 24.85, "day": "31"} +{"volume": 225074, "symbol": "AMD", "ts": "2018-08-31 10:14:00", "month": "08", "high": 24.85, "low": 24.801, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.84, "open": 24.84, "day": "31"} +{"volume": 169268, "symbol": "AMD", "ts": "2018-08-31 10:15:00", "month": "08", "high": 24.85, "low": 24.81, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.81, "open": 24.85, "day": "31"} +{"volume": 446516, "symbol": "AMD", "ts": "2018-08-31 10:16:00", "month": "08", "high": 24.805, "low": 24.74, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.78, "open": 24.805, "day": "31"} +{"volume": 143161, "symbol": "AMD", "ts": "2018-08-31 10:17:00", "month": "08", "high": 24.8, "low": 24.771, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.8, "open": 24.79, "day": "31"} +{"volume": 270616, "symbol": "AMD", "ts": "2018-08-31 10:18:00", "month": "08", "high": 24.81, "low": 24.7501, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.76, "open": 24.8, "day": "31"} +{"volume": 261280, "symbol": "AMD", "ts": "2018-08-31 10:19:00", "month": "08", "high": 24.815, "low": 24.7489, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.81, "open": 24.75, "day": "31"} +{"volume": 181319, "symbol": "AMD", "ts": "2018-08-31 10:20:00", "month": "08", "high": 24.84, "low": 24.8, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.82, "open": 24.8018, "day": "31"} +{"volume": 290936, "symbol": "AMD", "ts": "2018-08-31 10:21:00", "month": "08", "high": 24.88, "low": 24.8281, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.86, "open": 24.8281, "day": "31"} +{"volume": 274071, "symbol": "AMD", "ts": "2018-08-31 10:22:00", "month": "08", "high": 24.91, "low": 24.855, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9, "open": 24.86, "day": "31"} +{"volume": 284047, "symbol": "AMD", "ts": "2018-08-31 10:23:00", "month": "08", "high": 24.94, "low": 24.89, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.94, "open": 24.9, "day": "31"} +{"volume": 542794, "symbol": "AMD", "ts": "2018-08-31 10:24:00", "month": "08", "high": 24.98, "low": 24.92, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.95, "open": 24.94, "day": "31"} +{"volume": 232881, "symbol": "AMD", "ts": "2018-08-31 10:25:00", "month": "08", "high": 24.9485, "low": 24.93, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.939, "open": 24.9485, "day": "31"} +{"volume": 240261, "symbol": "AMD", "ts": "2018-08-31 10:26:00", "month": "08", "high": 24.96, "low": 24.905, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.91, "open": 24.935, "day": "31"} +{"volume": 274747, "symbol": "AMD", "ts": "2018-08-31 10:27:00", "month": "08", "high": 24.911, "low": 24.86, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.86, "open": 24.911, "day": "31"} +{"volume": 424294, "symbol": "AMD", "ts": "2018-08-31 10:28:00", "month": "08", "high": 24.865, "low": 24.8235, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.835, "open": 24.865, "day": "31"} +{"volume": 123381, "symbol": "AMD", "ts": "2018-08-31 10:29:00", "month": "08", "high": 24.8875, "low": 24.8357, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.87, "open": 24.8357, "day": "31"} +{"volume": 2102, "symbol": "CDMO", "ts": "2018-08-31 10:01:00", "month": "08", "high": 7.06, "low": 7.06, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.06, "open": 7.06, "day": "31"} +{"volume": 110, "symbol": "CDMO", "ts": "2018-08-31 10:02:00", "month": "08", "high": 7.02, "low": 7.02, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.02, "open": 7.02, "day": "31"} +{"volume": 307, "symbol": "CDMO", "ts": "2018-08-31 10:03:00", "month": "08", "high": 7.02, "low": 7.02, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.02, "open": 7.02, "day": "31"} +{"volume": 999, "symbol": "CDMO", "ts": "2018-08-31 10:05:00", "month": "08", "high": 7.0341, "low": 7.0341, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0341, "open": 7.0341, "day": "31"} +{"volume": 100, "symbol": "CDMO", "ts": "2018-08-31 10:08:00", "month": "08", "high": 7.02, "low": 7.02, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.02, "open": 7.02, "day": "31"} +{"volume": 2672, "symbol": "CDMO", "ts": "2018-08-31 10:09:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 2511, "symbol": "CDMO", "ts": "2018-08-31 10:11:00", "month": "08", "high": 7.01, "low": 7.01, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.01, "open": 7.01, "day": "31"} +{"volume": 1638, "symbol": "CDMO", "ts": "2018-08-31 10:14:00", "month": "08", "high": 7.0135, "low": 7.0135, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0135, "open": 7.0135, "day": "31"} +{"volume": 16663, "symbol": "CDMO", "ts": "2018-08-31 10:18:00", "month": "08", "high": 7.01, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.01, "day": "31"} +{"volume": 10594, "symbol": "CDMO", "ts": "2018-08-31 10:20:00", "month": "08", "high": 7.005, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.005, "day": "31"} +{"volume": 100, "symbol": "CDMO", "ts": "2018-08-31 10:23:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 2535, "symbol": "CDMO", "ts": "2018-08-31 10:24:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 9888, "symbol": "CDMO", "ts": "2018-08-31 10:25:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 4714, "symbol": "CDMO", "ts": "2018-08-31 10:27:00", "month": "08", "high": 6.9966, "low": 6.9966, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.9966, "open": 6.9966, "day": "31"} +{"volume": 1052, "symbol": "CDMO", "ts": "2018-08-31 10:29:00", "month": "08", "high": 6.9801, "low": 6.9801, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.9801, "open": 6.9801, "day": "31"} +{"volume": 8952, "symbol": "SFIX", "ts": "2018-08-31 10:01:00", "month": "08", "high": 39.821, "low": 39.675, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.821, "open": 39.675, "day": "31"} +{"volume": 1049, "symbol": "SFIX", "ts": "2018-08-31 10:02:00", "month": "08", "high": 39.81, "low": 39.81, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.81, "open": 39.81, "day": "31"} +{"volume": 2785, "symbol": "SFIX", "ts": "2018-08-31 10:03:00", "month": "08", "high": 39.73, "low": 39.73, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.73, "open": 39.73, "day": "31"} +{"volume": 3696, "symbol": "SFIX", "ts": "2018-08-31 10:04:00", "month": "08", "high": 39.77, "low": 39.77, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.77, "open": 39.77, "day": "31"} +{"volume": 3032, "symbol": "SFIX", "ts": "2018-08-31 10:05:00", "month": "08", "high": 39.7641, "low": 39.7641, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.7641, "open": 39.7641, "day": "31"} +{"volume": 2369, "symbol": "SFIX", "ts": "2018-08-31 10:06:00", "month": "08", "high": 39.8, "low": 39.8, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.8, "open": 39.8, "day": "31"} +{"volume": 1552, "symbol": "SFIX", "ts": "2018-08-31 10:07:00", "month": "08", "high": 39.775, "low": 39.775, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.775, "open": 39.775, "day": "31"} +{"volume": 32903, "symbol": "SFIX", "ts": "2018-08-31 10:08:00", "month": "08", "high": 40.0, "low": 39.8703, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.0, "open": 39.8703, "day": "31"} +{"volume": 12683, "symbol": "SFIX", "ts": "2018-08-31 10:09:00", "month": "08", "high": 40.04, "low": 39.99, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.99, "open": 40.04, "day": "31"} +{"volume": 9721, "symbol": "SFIX", "ts": "2018-08-31 10:10:00", "month": "08", "high": 40.01, "low": 40.0, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.0, "open": 40.01, "day": "31"} +{"volume": 11593, "symbol": "SFIX", "ts": "2018-08-31 10:11:00", "month": "08", "high": 39.939, "low": 39.915, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.939, "open": 39.915, "day": "31"} +{"volume": 2334, "symbol": "SFIX", "ts": "2018-08-31 10:12:00", "month": "08", "high": 39.89, "low": 39.89, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.89, "open": 39.89, "day": "31"} +{"volume": 1852, "symbol": "SFIX", "ts": "2018-08-31 10:13:00", "month": "08", "high": 40.0, "low": 40.0, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.0, "open": 40.0, "day": "31"} +{"volume": 9215, "symbol": "SFIX", "ts": "2018-08-31 10:14:00", "month": "08", "high": 40.12, "low": 39.99, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.12, "open": 39.99, "day": "31"} +{"volume": 6314, "symbol": "SFIX", "ts": "2018-08-31 10:15:00", "month": "08", "high": 40.0823, "low": 40.0823, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.0823, "open": 40.0823, "day": "31"} +{"volume": 4955, "symbol": "SFIX", "ts": "2018-08-31 10:16:00", "month": "08", "high": 40.0893, "low": 40.05, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.05, "open": 40.0893, "day": "31"} +{"volume": 11630, "symbol": "SFIX", "ts": "2018-08-31 10:17:00", "month": "08", "high": 40.13, "low": 40.07, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.07, "open": 40.13, "day": "31"} +{"volume": 1071, "symbol": "SFIX", "ts": "2018-08-31 10:18:00", "month": "08", "high": 40.125, "low": 40.125, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.125, "open": 40.125, "day": "31"} +{"volume": 11553, "symbol": "SFIX", "ts": "2018-08-31 10:19:00", "month": "08", "high": 40.025, "low": 40.0001, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.0001, "open": 40.025, "day": "31"} +{"volume": 6216, "symbol": "SFIX", "ts": "2018-08-31 10:20:00", "month": "08", "high": 39.9874, "low": 39.9874, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.9874, "open": 39.9874, "day": "31"} +{"volume": 5705, "symbol": "SFIX", "ts": "2018-08-31 10:21:00", "month": "08", "high": 39.954, "low": 39.91, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.954, "open": 39.91, "day": "31"} +{"volume": 2813, "symbol": "SFIX", "ts": "2018-08-31 10:22:00", "month": "08", "high": 39.9475, "low": 39.9475, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.9475, "open": 39.9475, "day": "31"} +{"volume": 7158, "symbol": "SFIX", "ts": "2018-08-31 10:23:00", "month": "08", "high": 39.965, "low": 39.9432, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.965, "open": 39.9432, "day": "31"} +{"volume": 5790, "symbol": "SFIX", "ts": "2018-08-31 10:24:00", "month": "08", "high": 40.04, "low": 39.9692, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.9692, "open": 40.04, "day": "31"} +{"volume": 5074, "symbol": "SFIX", "ts": "2018-08-31 10:26:00", "month": "08", "high": 40.03, "low": 40.01, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.03, "open": 40.01, "day": "31"} +{"volume": 7057, "symbol": "SFIX", "ts": "2018-08-31 10:27:00", "month": "08", "high": 39.96, "low": 39.96, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.96, "open": 39.96, "day": "31"} +{"volume": 1516, "symbol": "SFIX", "ts": "2018-08-31 10:28:00", "month": "08", "high": 39.9678, "low": 39.9678, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.9678, "open": 39.9678, "day": "31"} +{"volume": 4010, "symbol": "SFIX", "ts": "2018-08-31 10:29:00", "month": "08", "high": 39.97, "low": 39.97, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 39.97, "open": 39.97, "day": "31"} +{"volume": 530, "symbol": "DRNA", "ts": "2018-08-31 10:01:00", "month": "08", "high": 15.89, "low": 15.89, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.89, "open": 15.89, "day": "31"} +{"volume": 1584, "symbol": "DRNA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 15.93, "low": 15.93, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.93, "open": 15.93, "day": "31"} +{"volume": 1211, "symbol": "DRNA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 15.9673, "low": 15.9673, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.9673, "open": 15.9673, "day": "31"} +{"volume": 100, "symbol": "DRNA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 15.98, "low": 15.98, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.98, "open": 15.98, "day": "31"} +{"volume": 1707, "symbol": "DRNA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 15.93, "low": 15.83, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.83, "open": 15.93, "day": "31"} +{"volume": 764, "symbol": "DRNA", "ts": "2018-08-31 10:12:00", "month": "08", "high": 15.87, "low": 15.87, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.87, "open": 15.87, "day": "31"} +{"volume": 2260, "symbol": "DRNA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 15.875, "low": 15.875, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.875, "open": 15.875, "day": "31"} +{"volume": 1086, "symbol": "DRNA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 15.86, "low": 15.86, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.86, "open": 15.86, "day": "31"} +{"volume": 514, "symbol": "DRNA", "ts": "2018-08-31 10:19:00", "month": "08", "high": 15.84, "low": 15.84, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.84, "open": 15.84, "day": "31"} +{"volume": 475, "symbol": "DRNA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 15.87, "low": 15.87, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.87, "open": 15.87, "day": "31"} +{"volume": 1199, "symbol": "DRNA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 15.9, "low": 15.9, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.9, "open": 15.9, "day": "31"} +{"volume": 511, "symbol": "DRNA", "ts": "2018-08-31 10:26:00", "month": "08", "high": 15.84, "low": 15.84, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.84, "open": 15.84, "day": "31"} +{"volume": 389, "symbol": "RFP", "ts": "2018-08-31 10:01:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 700, "symbol": "RFP", "ts": "2018-08-31 10:10:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 628, "symbol": "RFP", "ts": "2018-08-31 10:12:00", "month": "08", "high": 13.95, "low": 13.925, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.925, "open": 13.95, "day": "31"} +{"volume": 2900, "symbol": "RFP", "ts": "2018-08-31 10:15:00", "month": "08", "high": 13.95, "low": 13.9375, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.9375, "day": "31"} +{"volume": 213, "symbol": "RFP", "ts": "2018-08-31 10:20:00", "month": "08", "high": 14.0, "low": 14.0, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0, "open": 14.0, "day": "31"} +{"volume": 700, "symbol": "RFP", "ts": "2018-08-31 10:25:00", "month": "08", "high": 14.0, "low": 14.0, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0, "open": 14.0, "day": "31"} +{"volume": 13509, "symbol": "RFP", "ts": "2018-08-31 10:27:00", "month": "08", "high": 14.0, "low": 14.0, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0, "open": 14.0, "day": "31"} +{"volume": 1500, "symbol": "RFP", "ts": "2018-08-31 10:28:00", "month": "08", "high": 14.0, "low": 14.0, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0, "open": 14.0, "day": "31"} +{"volume": 2908, "symbol": "RFP", "ts": "2018-08-31 10:29:00", "month": "08", "high": 14.0, "low": 14.0, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0, "open": 14.0, "day": "31"} +{"volume": 701, "symbol": "BJRI", "ts": "2018-08-31 10:01:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 137, "symbol": "BJRI", "ts": "2018-08-31 10:04:00", "month": "08", "high": 74.575, "low": 74.575, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.575, "open": 74.575, "day": "31"} +{"volume": 729, "symbol": "BJRI", "ts": "2018-08-31 10:07:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 604, "symbol": "BJRI", "ts": "2018-08-31 10:08:00", "month": "08", "high": 74.45, "low": 74.45, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.45, "open": 74.45, "day": "31"} +{"volume": 317, "symbol": "BJRI", "ts": "2018-08-31 10:10:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 1214, "symbol": "BJRI", "ts": "2018-08-31 10:12:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 337, "symbol": "BJRI", "ts": "2018-08-31 10:14:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 159, "symbol": "BJRI", "ts": "2018-08-31 10:15:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 1679, "symbol": "BJRI", "ts": "2018-08-31 10:18:00", "month": "08", "high": 74.525, "low": 74.5, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.525, "day": "31"} +{"volume": 755, "symbol": "BJRI", "ts": "2018-08-31 10:21:00", "month": "08", "high": 74.525, "low": 74.525, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.525, "open": 74.525, "day": "31"} +{"volume": 573, "symbol": "BJRI", "ts": "2018-08-31 10:22:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 303, "symbol": "BJRI", "ts": "2018-08-31 10:26:00", "month": "08", "high": 74.54, "low": 74.54, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.54, "open": 74.54, "day": "31"} +{"volume": 1001, "symbol": "BJRI", "ts": "2018-08-31 10:27:00", "month": "08", "high": 74.525, "low": 74.525, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.525, "open": 74.525, "day": "31"} +{"volume": 886, "symbol": "BJRI", "ts": "2018-08-31 10:29:00", "month": "08", "high": 74.525, "low": 74.525, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.525, "open": 74.525, "day": "31"} +{"volume": 1687, "symbol": "NVCR", "ts": "2018-08-31 10:01:00", "month": "08", "high": 44.2, "low": 44.2, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.2, "open": 44.2, "day": "31"} +{"volume": 2413, "symbol": "NVCR", "ts": "2018-08-31 10:02:00", "month": "08", "high": 44.175, "low": 44.175, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.175, "open": 44.175, "day": "31"} +{"volume": 1267, "symbol": "NVCR", "ts": "2018-08-31 10:03:00", "month": "08", "high": 44.2, "low": 44.2, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.2, "open": 44.2, "day": "31"} +{"volume": 2047, "symbol": "NVCR", "ts": "2018-08-31 10:04:00", "month": "08", "high": 44.25, "low": 44.25, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.25, "open": 44.25, "day": "31"} +{"volume": 9730, "symbol": "NVCR", "ts": "2018-08-31 10:05:00", "month": "08", "high": 44.275, "low": 44.275, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.275, "open": 44.275, "day": "31"} +{"volume": 26079, "symbol": "NVCR", "ts": "2018-08-31 10:06:00", "month": "08", "high": 44.4, "low": 44.4, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.4, "open": 44.4, "day": "31"} +{"volume": 22851, "symbol": "NVCR", "ts": "2018-08-31 10:09:00", "month": "08", "high": 44.445, "low": 44.35, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.35, "open": 44.445, "day": "31"} +{"volume": 1795, "symbol": "NVCR", "ts": "2018-08-31 10:10:00", "month": "08", "high": 44.4, "low": 44.4, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.4, "open": 44.4, "day": "31"} +{"volume": 2192, "symbol": "NVCR", "ts": "2018-08-31 10:12:00", "month": "08", "high": 44.175, "low": 44.175, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.175, "open": 44.175, "day": "31"} +{"volume": 1681, "symbol": "NVCR", "ts": "2018-08-31 10:13:00", "month": "08", "high": 44.265, "low": 44.265, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.265, "open": 44.265, "day": "31"} +{"volume": 1543, "symbol": "NVCR", "ts": "2018-08-31 10:16:00", "month": "08", "high": 44.45, "low": 44.45, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.45, "open": 44.45, "day": "31"} +{"volume": 1531, "symbol": "NVCR", "ts": "2018-08-31 10:17:00", "month": "08", "high": 44.5, "low": 44.5, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.5, "open": 44.5, "day": "31"} +{"volume": 1496, "symbol": "NVCR", "ts": "2018-08-31 10:18:00", "month": "08", "high": 44.483, "low": 44.483, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.483, "open": 44.483, "day": "31"} +{"volume": 2075, "symbol": "NVCR", "ts": "2018-08-31 10:20:00", "month": "08", "high": 44.6, "low": 44.6, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.6, "open": 44.6, "day": "31"} +{"volume": 750, "symbol": "NVCR", "ts": "2018-08-31 10:21:00", "month": "08", "high": 44.6, "low": 44.6, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.6, "open": 44.6, "day": "31"} +{"volume": 2282, "symbol": "NVCR", "ts": "2018-08-31 10:23:00", "month": "08", "high": 44.65, "low": 44.65, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.65, "open": 44.65, "day": "31"} +{"volume": 1446, "symbol": "NVCR", "ts": "2018-08-31 10:24:00", "month": "08", "high": 44.75, "low": 44.75, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.75, "open": 44.75, "day": "31"} +{"volume": 572, "symbol": "NVCR", "ts": "2018-08-31 10:25:00", "month": "08", "high": 44.75, "low": 44.75, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.75, "open": 44.75, "day": "31"} +{"volume": 3223, "symbol": "NVCR", "ts": "2018-08-31 10:27:00", "month": "08", "high": 44.85, "low": 44.85, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.85, "open": 44.85, "day": "31"} +{"volume": 6010, "symbol": "NVCR", "ts": "2018-08-31 10:28:00", "month": "08", "high": 44.85, "low": 44.825, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.85, "open": 44.825, "day": "31"} +{"volume": 5157, "symbol": "NVCR", "ts": "2018-08-31 10:29:00", "month": "08", "high": 44.9, "low": 44.9, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.9, "open": 44.9, "day": "31"} +{"volume": 30518, "symbol": "NOG", "ts": "2018-08-31 10:01:00", "month": "08", "high": 3.365, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 2751, "symbol": "NOG", "ts": "2018-08-31 10:02:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 307, "symbol": "NOG", "ts": "2018-08-31 10:04:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 13093, "symbol": "NOG", "ts": "2018-08-31 10:05:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 11955, "symbol": "NOG", "ts": "2018-08-31 10:06:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 3000, "symbol": "NOG", "ts": "2018-08-31 10:07:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 218, "symbol": "NOG", "ts": "2018-08-31 10:08:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 5401, "symbol": "NOG", "ts": "2018-08-31 10:09:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 4035, "symbol": "NOG", "ts": "2018-08-31 10:10:00", "month": "08", "high": 3.3699, "low": 3.3699, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.3699, "open": 3.3699, "day": "31"} +{"volume": 75947, "symbol": "NOG", "ts": "2018-08-31 10:12:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 5404, "symbol": "NOG", "ts": "2018-08-31 10:13:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 3901, "symbol": "NOG", "ts": "2018-08-31 10:14:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 1215, "symbol": "NOG", "ts": "2018-08-31 10:15:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 2346, "symbol": "NOG", "ts": "2018-08-31 10:16:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 800, "symbol": "NOG", "ts": "2018-08-31 10:17:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 10188, "symbol": "NOG", "ts": "2018-08-31 10:18:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 4751, "symbol": "NOG", "ts": "2018-08-31 10:19:00", "month": "08", "high": 3.3526, "low": 3.3526, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.3526, "open": 3.3526, "day": "31"} +{"volume": 1526, "symbol": "NOG", "ts": "2018-08-31 10:21:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 3892, "symbol": "NOG", "ts": "2018-08-31 10:22:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 7147, "symbol": "NOG", "ts": "2018-08-31 10:24:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 15668, "symbol": "NOG", "ts": "2018-08-31 10:25:00", "month": "08", "high": 3.375, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.375, "day": "31"} +{"volume": 4521, "symbol": "NOG", "ts": "2018-08-31 10:27:00", "month": "08", "high": 3.3668, "low": 3.3603, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.3603, "open": 3.3668, "day": "31"} +{"volume": 2205, "symbol": "NOG", "ts": "2018-08-31 10:29:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 427, "symbol": "PTSI", "ts": "2018-08-31 10:03:00", "month": "08", "high": 59.205, "low": 59.205, "key": "PTSI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.205, "open": 59.205, "day": "31"} +{"volume": 200, "symbol": "BAND", "ts": "2018-08-31 10:06:00", "month": "08", "high": 44.91, "low": 44.91, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.91, "open": 44.91, "day": "31"} +{"volume": 1099, "symbol": "BAND", "ts": "2018-08-31 10:07:00", "month": "08", "high": 44.9, "low": 44.9, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.9, "open": 44.9, "day": "31"} +{"volume": 378, "symbol": "BAND", "ts": "2018-08-31 10:18:00", "month": "08", "high": 44.99, "low": 44.99, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.99, "open": 44.99, "day": "31"} +{"volume": 438, "symbol": "BAND", "ts": "2018-08-31 10:22:00", "month": "08", "high": 44.95, "low": 44.95, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.95, "open": 44.95, "day": "31"} +{"volume": 4626, "symbol": "AMRS", "ts": "2018-08-31 10:01:00", "month": "08", "high": 8.49, "low": 8.49, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.49, "open": 8.49, "day": "31"} +{"volume": 4041, "symbol": "AMRS", "ts": "2018-08-31 10:02:00", "month": "08", "high": 8.48, "low": 8.48, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.48, "open": 8.48, "day": "31"} +{"volume": 4553, "symbol": "AMRS", "ts": "2018-08-31 10:03:00", "month": "08", "high": 8.5136, "low": 8.5101, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.5136, "open": 8.5101, "day": "31"} +{"volume": 1570, "symbol": "AMRS", "ts": "2018-08-31 10:06:00", "month": "08", "high": 8.56, "low": 8.56, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.56, "open": 8.56, "day": "31"} +{"volume": 4496, "symbol": "AMRS", "ts": "2018-08-31 10:07:00", "month": "08", "high": 8.599, "low": 8.599, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.599, "open": 8.599, "day": "31"} +{"volume": 12901, "symbol": "AMRS", "ts": "2018-08-31 10:08:00", "month": "08", "high": 8.62, "low": 8.59, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.59, "open": 8.62, "day": "31"} +{"volume": 10823, "symbol": "AMRS", "ts": "2018-08-31 10:09:00", "month": "08", "high": 8.6368, "low": 8.61, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6368, "open": 8.61, "day": "31"} +{"volume": 6511, "symbol": "AMRS", "ts": "2018-08-31 10:10:00", "month": "08", "high": 8.5768, "low": 8.5768, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.5768, "open": 8.5768, "day": "31"} +{"volume": 7219, "symbol": "AMRS", "ts": "2018-08-31 10:11:00", "month": "08", "high": 8.58, "low": 8.58, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.58, "open": 8.58, "day": "31"} +{"volume": 6701, "symbol": "AMRS", "ts": "2018-08-31 10:13:00", "month": "08", "high": 8.58, "low": 8.58, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.58, "open": 8.58, "day": "31"} +{"volume": 5885, "symbol": "AMRS", "ts": "2018-08-31 10:16:00", "month": "08", "high": 8.6314, "low": 8.6314, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6314, "open": 8.6314, "day": "31"} +{"volume": 2905, "symbol": "AMRS", "ts": "2018-08-31 10:17:00", "month": "08", "high": 8.63, "low": 8.63, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.63, "open": 8.63, "day": "31"} +{"volume": 5140, "symbol": "AMRS", "ts": "2018-08-31 10:18:00", "month": "08", "high": 8.605, "low": 8.605, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.605, "open": 8.605, "day": "31"} +{"volume": 1615, "symbol": "AMRS", "ts": "2018-08-31 10:19:00", "month": "08", "high": 8.6285, "low": 8.6285, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6285, "open": 8.6285, "day": "31"} +{"volume": 2563, "symbol": "AMRS", "ts": "2018-08-31 10:21:00", "month": "08", "high": 8.6101, "low": 8.6101, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6101, "open": 8.6101, "day": "31"} +{"volume": 7680, "symbol": "AMRS", "ts": "2018-08-31 10:22:00", "month": "08", "high": 8.6, "low": 8.6, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6, "open": 8.6, "day": "31"} +{"volume": 325, "symbol": "AMRS", "ts": "2018-08-31 10:24:00", "month": "08", "high": 8.5801, "low": 8.5801, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.5801, "open": 8.5801, "day": "31"} +{"volume": 10065, "symbol": "AMRS", "ts": "2018-08-31 10:25:00", "month": "08", "high": 8.6599, "low": 8.63, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6599, "open": 8.63, "day": "31"} +{"volume": 4682, "symbol": "AMRS", "ts": "2018-08-31 10:26:00", "month": "08", "high": 8.6773, "low": 8.6773, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6773, "open": 8.6773, "day": "31"} +{"volume": 1682, "symbol": "AMRS", "ts": "2018-08-31 10:27:00", "month": "08", "high": 8.68, "low": 8.68, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.68, "open": 8.68, "day": "31"} +{"volume": 13650, "symbol": "AMRS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 8.72, "low": 8.69, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.72, "open": 8.69, "day": "31"} +{"volume": 4678, "symbol": "AMRS", "ts": "2018-08-31 10:29:00", "month": "08", "high": 8.7347, "low": 8.7347, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.7347, "open": 8.7347, "day": "31"} +{"volume": 1534, "symbol": "COUP", "ts": "2018-08-31 10:01:00", "month": "08", "high": 70.67, "low": 70.67, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.67, "open": 70.67, "day": "31"} +{"volume": 1038, "symbol": "COUP", "ts": "2018-08-31 10:03:00", "month": "08", "high": 70.7, "low": 70.7, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.7, "open": 70.7, "day": "31"} +{"volume": 968, "symbol": "COUP", "ts": "2018-08-31 10:04:00", "month": "08", "high": 70.86, "low": 70.86, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.86, "open": 70.86, "day": "31"} +{"volume": 723, "symbol": "COUP", "ts": "2018-08-31 10:07:00", "month": "08", "high": 71.07, "low": 71.07, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.07, "open": 71.07, "day": "31"} +{"volume": 2032, "symbol": "COUP", "ts": "2018-08-31 10:08:00", "month": "08", "high": 70.98, "low": 70.96, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.96, "open": 70.98, "day": "31"} +{"volume": 1532, "symbol": "COUP", "ts": "2018-08-31 10:10:00", "month": "08", "high": 71.03, "low": 71.03, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.03, "open": 71.03, "day": "31"} +{"volume": 1232, "symbol": "COUP", "ts": "2018-08-31 10:12:00", "month": "08", "high": 71.0, "low": 71.0, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.0, "open": 71.0, "day": "31"} +{"volume": 1157, "symbol": "COUP", "ts": "2018-08-31 10:13:00", "month": "08", "high": 71.0507, "low": 71.0507, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.0507, "open": 71.0507, "day": "31"} +{"volume": 3314, "symbol": "COUP", "ts": "2018-08-31 10:15:00", "month": "08", "high": 71.18, "low": 71.14, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.18, "open": 71.14, "day": "31"} +{"volume": 3048, "symbol": "COUP", "ts": "2018-08-31 10:17:00", "month": "08", "high": 71.19, "low": 71.19, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.19, "open": 71.19, "day": "31"} +{"volume": 1908, "symbol": "COUP", "ts": "2018-08-31 10:18:00", "month": "08", "high": 71.13, "low": 71.13, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.13, "open": 71.13, "day": "31"} +{"volume": 1768, "symbol": "COUP", "ts": "2018-08-31 10:20:00", "month": "08", "high": 71.02, "low": 71.01, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.01, "open": 71.02, "day": "31"} +{"volume": 404, "symbol": "COUP", "ts": "2018-08-31 10:23:00", "month": "08", "high": 71.02, "low": 71.02, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.02, "open": 71.02, "day": "31"} +{"volume": 1705, "symbol": "COUP", "ts": "2018-08-31 10:24:00", "month": "08", "high": 71.04, "low": 71.04, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.04, "open": 71.04, "day": "31"} +{"volume": 859, "symbol": "COUP", "ts": "2018-08-31 10:27:00", "month": "08", "high": 71.05, "low": 71.05, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.05, "open": 71.05, "day": "31"} +{"volume": 2395, "symbol": "COUP", "ts": "2018-08-31 10:29:00", "month": "08", "high": 70.815, "low": 70.815, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.815, "open": 70.815, "day": "31"} +{"volume": 1052, "symbol": "AAXN", "ts": "2018-08-31 10:02:00", "month": "08", "high": 67.03, "low": 67.03, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 67.03, "open": 67.03, "day": "31"} +{"volume": 344, "symbol": "AAXN", "ts": "2018-08-31 10:05:00", "month": "08", "high": 67.04, "low": 67.04, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 67.04, "open": 67.04, "day": "31"} +{"volume": 837, "symbol": "AAXN", "ts": "2018-08-31 10:06:00", "month": "08", "high": 66.9543, "low": 66.9543, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.9543, "open": 66.9543, "day": "31"} +{"volume": 2034, "symbol": "AAXN", "ts": "2018-08-31 10:12:00", "month": "08", "high": 66.93, "low": 66.84, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.84, "open": 66.93, "day": "31"} +{"volume": 674, "symbol": "AAXN", "ts": "2018-08-31 10:16:00", "month": "08", "high": 66.9395, "low": 66.9395, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.9395, "open": 66.9395, "day": "31"} +{"volume": 152, "symbol": "AAXN", "ts": "2018-08-31 10:18:00", "month": "08", "high": 67.02, "low": 67.02, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 67.02, "open": 67.02, "day": "31"} +{"volume": 1388, "symbol": "AAXN", "ts": "2018-08-31 10:21:00", "month": "08", "high": 66.9599, "low": 66.9599, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.9599, "open": 66.9599, "day": "31"} +{"volume": 1196, "symbol": "AAXN", "ts": "2018-08-31 10:22:00", "month": "08", "high": 66.9, "low": 66.9, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.9, "open": 66.9, "day": "31"} +{"volume": 3440, "symbol": "AAXN", "ts": "2018-08-31 10:25:00", "month": "08", "high": 66.87, "low": 66.87, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.87, "open": 66.87, "day": "31"} +{"volume": 500, "symbol": "AAXN", "ts": "2018-08-31 10:26:00", "month": "08", "high": 66.92, "low": 66.92, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.92, "open": 66.92, "day": "31"} +{"volume": 1199, "symbol": "AAXN", "ts": "2018-08-31 10:28:00", "month": "08", "high": 66.841, "low": 66.841, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.841, "open": 66.841, "day": "31"} +{"volume": 885, "symbol": "OKTA", "ts": "2018-08-31 10:01:00", "month": "08", "high": 61.07, "low": 61.07, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.07, "open": 61.07, "day": "31"} +{"volume": 8901, "symbol": "OKTA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 61.2, "low": 60.971, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 60.971, "open": 61.2, "day": "31"} +{"volume": 18376, "symbol": "OKTA", "ts": "2018-08-31 10:04:00", "month": "08", "high": 61.195, "low": 61.12, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.18, "open": 61.12, "day": "31"} +{"volume": 4849, "symbol": "OKTA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 61.205, "low": 61.205, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.205, "open": 61.205, "day": "31"} +{"volume": 2423, "symbol": "OKTA", "ts": "2018-08-31 10:06:00", "month": "08", "high": 61.255, "low": 61.255, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.255, "open": 61.255, "day": "31"} +{"volume": 1536, "symbol": "OKTA", "ts": "2018-08-31 10:07:00", "month": "08", "high": 61.2844, "low": 61.2844, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.2844, "open": 61.2844, "day": "31"} +{"volume": 1400, "symbol": "OKTA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 61.16, "low": 61.16, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.16, "open": 61.16, "day": "31"} +{"volume": 1030, "symbol": "OKTA", "ts": "2018-08-31 10:09:00", "month": "08", "high": 61.19, "low": 61.19, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.19, "open": 61.19, "day": "31"} +{"volume": 2203, "symbol": "OKTA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 61.24, "low": 61.24, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.24, "open": 61.24, "day": "31"} +{"volume": 2151, "symbol": "OKTA", "ts": "2018-08-31 10:12:00", "month": "08", "high": 61.18, "low": 61.18, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.18, "open": 61.18, "day": "31"} +{"volume": 1765, "symbol": "OKTA", "ts": "2018-08-31 10:13:00", "month": "08", "high": 61.25, "low": 61.25, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.25, "open": 61.25, "day": "31"} +{"volume": 4483, "symbol": "OKTA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 61.33, "low": 61.33, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.33, "open": 61.33, "day": "31"} +{"volume": 1117, "symbol": "OKTA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 61.41, "low": 61.41, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.41, "open": 61.41, "day": "31"} +{"volume": 1599, "symbol": "OKTA", "ts": "2018-08-31 10:17:00", "month": "08", "high": 61.46, "low": 61.46, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.46, "open": 61.46, "day": "31"} +{"volume": 1281, "symbol": "OKTA", "ts": "2018-08-31 10:19:00", "month": "08", "high": 61.44, "low": 61.44, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.44, "open": 61.44, "day": "31"} +{"volume": 364, "symbol": "OKTA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 61.4821, "low": 61.4821, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.4821, "open": 61.4821, "day": "31"} +{"volume": 2251, "symbol": "OKTA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 61.5036, "low": 61.5036, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.5036, "open": 61.5036, "day": "31"} +{"volume": 2086, "symbol": "OKTA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 61.51, "low": 61.51, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.51, "open": 61.51, "day": "31"} +{"volume": 1776, "symbol": "OKTA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 61.4782, "low": 61.4782, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.4782, "open": 61.4782, "day": "31"} +{"volume": 961, "symbol": "OKTA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 61.395, "low": 61.395, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.395, "open": 61.395, "day": "31"} +{"volume": 515, "symbol": "EVBG", "ts": "2018-08-31 10:01:00", "month": "08", "high": 59.405, "low": 59.405, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.405, "open": 59.405, "day": "31"} +{"volume": 817, "symbol": "EVBG", "ts": "2018-08-31 10:04:00", "month": "08", "high": 59.48, "low": 59.4699, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.48, "open": 59.4699, "day": "31"} +{"volume": 670, "symbol": "EVBG", "ts": "2018-08-31 10:08:00", "month": "08", "high": 59.6799, "low": 59.565, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.6799, "open": 59.565, "day": "31"} +{"volume": 184, "symbol": "EVBG", "ts": "2018-08-31 10:11:00", "month": "08", "high": 59.68, "low": 59.68, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.68, "open": 59.68, "day": "31"} +{"volume": 300, "symbol": "EVBG", "ts": "2018-08-31 10:14:00", "month": "08", "high": 59.63, "low": 59.63, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.63, "open": 59.63, "day": "31"} +{"volume": 910, "symbol": "EVBG", "ts": "2018-08-31 10:16:00", "month": "08", "high": 59.6399, "low": 59.6399, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.6399, "open": 59.6399, "day": "31"} +{"volume": 644, "symbol": "EVBG", "ts": "2018-08-31 10:17:00", "month": "08", "high": 59.58, "low": 59.58, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.58, "open": 59.58, "day": "31"} +{"volume": 2400, "symbol": "EVBG", "ts": "2018-08-31 10:19:00", "month": "08", "high": 59.54, "low": 59.54, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.54, "open": 59.54, "day": "31"} +{"volume": 3002, "symbol": "EVBG", "ts": "2018-08-31 10:21:00", "month": "08", "high": 59.79, "low": 59.79, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.79, "open": 59.79, "day": "31"} +{"volume": 315, "symbol": "EVBG", "ts": "2018-08-31 10:22:00", "month": "08", "high": 59.94, "low": 59.94, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.94, "open": 59.94, "day": "31"} +{"volume": 100, "symbol": "EVBG", "ts": "2018-08-31 10:25:00", "month": "08", "high": 59.835, "low": 59.835, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.835, "open": 59.835, "day": "31"} +{"volume": 621, "symbol": "EVBG", "ts": "2018-08-31 10:28:00", "month": "08", "high": 59.95, "low": 59.95, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.95, "open": 59.95, "day": "31"} +{"volume": 800, "symbol": "EVBG", "ts": "2018-08-31 10:29:00", "month": "08", "high": 59.95, "low": 59.95, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.95, "open": 59.95, "day": "31"} +{"volume": 1612, "symbol": "RETA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 86.8826, "low": 86.461, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.8826, "open": 86.461, "day": "31"} +{"volume": 1715, "symbol": "RETA", "ts": "2018-08-31 10:06:00", "month": "08", "high": 86.8634, "low": 86.8634, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.8634, "open": 86.8634, "day": "31"} +{"volume": 1390, "symbol": "RETA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 87.4898, "low": 87.4898, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.4898, "open": 87.4898, "day": "31"} +{"volume": 895, "symbol": "RETA", "ts": "2018-08-31 10:09:00", "month": "08", "high": 87.2466, "low": 87.2466, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.2466, "open": 87.2466, "day": "31"} +{"volume": 1578, "symbol": "RETA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 86.79, "low": 86.79, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.79, "open": 86.79, "day": "31"} +{"volume": 557, "symbol": "RETA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 86.47, "low": 86.47, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.47, "open": 86.47, "day": "31"} +{"volume": 1923, "symbol": "RETA", "ts": "2018-08-31 10:18:00", "month": "08", "high": 86.3454, "low": 86.3454, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.3454, "open": 86.3454, "day": "31"} +{"volume": 100, "symbol": "RETA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 86.2285, "low": 86.2285, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.2285, "open": 86.2285, "day": "31"} +{"volume": 289, "symbol": "RETA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 86.06, "low": 86.06, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.06, "open": 86.06, "day": "31"} +{"volume": 513, "symbol": "RETA", "ts": "2018-08-31 10:25:00", "month": "08", "high": 86.27, "low": 86.27, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.27, "open": 86.27, "day": "31"} +{"volume": 400, "symbol": "RETA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 86.11, "low": 86.1, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.1, "open": 86.11, "day": "31"} +{"volume": 1404, "symbol": "UIS", "ts": "2018-08-31 10:01:00", "month": "08", "high": 18.525, "low": 18.525, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.525, "open": 18.525, "day": "31"} +{"volume": 2033, "symbol": "UIS", "ts": "2018-08-31 10:02:00", "month": "08", "high": 18.525, "low": 18.525, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.525, "open": 18.525, "day": "31"} +{"volume": 1968, "symbol": "UIS", "ts": "2018-08-31 10:04:00", "month": "08", "high": 18.525, "low": 18.525, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.525, "open": 18.525, "day": "31"} +{"volume": 11473, "symbol": "UIS", "ts": "2018-08-31 10:06:00", "month": "08", "high": 18.55, "low": 18.525, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.525, "day": "31"} +{"volume": 6452, "symbol": "UIS", "ts": "2018-08-31 10:07:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 1914, "symbol": "UIS", "ts": "2018-08-31 10:08:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 2844, "symbol": "UIS", "ts": "2018-08-31 10:09:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 1128, "symbol": "UIS", "ts": "2018-08-31 10:10:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 2023, "symbol": "UIS", "ts": "2018-08-31 10:11:00", "month": "08", "high": 18.5625, "low": 18.5625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.5625, "open": 18.5625, "day": "31"} +{"volume": 3581, "symbol": "UIS", "ts": "2018-08-31 10:12:00", "month": "08", "high": 18.5625, "low": 18.55, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.5625, "open": 18.55, "day": "31"} +{"volume": 1256, "symbol": "UIS", "ts": "2018-08-31 10:13:00", "month": "08", "high": 18.55, "low": 18.55, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.55, "day": "31"} +{"volume": 13692, "symbol": "UIS", "ts": "2018-08-31 10:14:00", "month": "08", "high": 18.6, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.575, "day": "31"} +{"volume": 8457, "symbol": "UIS", "ts": "2018-08-31 10:15:00", "month": "08", "high": 18.6, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.575, "day": "31"} +{"volume": 38271, "symbol": "UIS", "ts": "2018-08-31 10:16:00", "month": "08", "high": 18.575, "low": 18.5512, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 1335, "symbol": "UIS", "ts": "2018-08-31 10:17:00", "month": "08", "high": 18.55, "low": 18.55, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.55, "day": "31"} +{"volume": 3186, "symbol": "UIS", "ts": "2018-08-31 10:18:00", "month": "08", "high": 18.6, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.6, "day": "31"} +{"volume": 1605, "symbol": "UIS", "ts": "2018-08-31 10:19:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 2893, "symbol": "UIS", "ts": "2018-08-31 10:20:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 4259, "symbol": "UIS", "ts": "2018-08-31 10:21:00", "month": "08", "high": 18.6, "low": 18.55, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.55, "day": "31"} +{"volume": 2430, "symbol": "UIS", "ts": "2018-08-31 10:22:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 2005, "symbol": "UIS", "ts": "2018-08-31 10:23:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 1703, "symbol": "UIS", "ts": "2018-08-31 10:24:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 1206, "symbol": "UIS", "ts": "2018-08-31 10:25:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 7524, "symbol": "UIS", "ts": "2018-08-31 10:26:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 2945, "symbol": "UIS", "ts": "2018-08-31 10:27:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 1302, "symbol": "UIS", "ts": "2018-08-31 10:28:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 1404, "symbol": "UIS", "ts": "2018-08-31 10:29:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 3848, "symbol": "ETSY", "ts": "2018-08-31 10:02:00", "month": "08", "high": 48.3, "low": 48.19, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.3, "open": 48.19, "day": "31"} +{"volume": 3334, "symbol": "ETSY", "ts": "2018-08-31 10:05:00", "month": "08", "high": 48.38, "low": 48.38, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.38, "open": 48.38, "day": "31"} +{"volume": 10796, "symbol": "ETSY", "ts": "2018-08-31 10:06:00", "month": "08", "high": 48.27, "low": 48.265, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.265, "open": 48.27, "day": "31"} +{"volume": 3725, "symbol": "ETSY", "ts": "2018-08-31 10:07:00", "month": "08", "high": 48.2316, "low": 48.2316, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.2316, "open": 48.2316, "day": "31"} +{"volume": 5278, "symbol": "ETSY", "ts": "2018-08-31 10:08:00", "month": "08", "high": 48.31, "low": 48.31, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.31, "open": 48.31, "day": "31"} +{"volume": 11241, "symbol": "ETSY", "ts": "2018-08-31 10:09:00", "month": "08", "high": 48.51, "low": 48.35, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.51, "open": 48.35, "day": "31"} +{"volume": 10839, "symbol": "ETSY", "ts": "2018-08-31 10:10:00", "month": "08", "high": 48.38, "low": 48.38, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.38, "open": 48.38, "day": "31"} +{"volume": 1645, "symbol": "ETSY", "ts": "2018-08-31 10:12:00", "month": "08", "high": 48.425, "low": 48.425, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.425, "open": 48.425, "day": "31"} +{"volume": 2851, "symbol": "ETSY", "ts": "2018-08-31 10:13:00", "month": "08", "high": 48.455, "low": 48.455, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.455, "open": 48.455, "day": "31"} +{"volume": 1824, "symbol": "ETSY", "ts": "2018-08-31 10:14:00", "month": "08", "high": 48.4277, "low": 48.4277, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.4277, "open": 48.4277, "day": "31"} +{"volume": 29804, "symbol": "ETSY", "ts": "2018-08-31 10:15:00", "month": "08", "high": 48.43, "low": 48.43, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.43, "open": 48.43, "day": "31"} +{"volume": 1498, "symbol": "ETSY", "ts": "2018-08-31 10:17:00", "month": "08", "high": 48.405, "low": 48.405, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.405, "open": 48.405, "day": "31"} +{"volume": 494, "symbol": "ETSY", "ts": "2018-08-31 10:18:00", "month": "08", "high": 48.3861, "low": 48.3861, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.3861, "open": 48.3861, "day": "31"} +{"volume": 3244, "symbol": "ETSY", "ts": "2018-08-31 10:19:00", "month": "08", "high": 48.37, "low": 48.37, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.37, "open": 48.37, "day": "31"} +{"volume": 5022, "symbol": "ETSY", "ts": "2018-08-31 10:20:00", "month": "08", "high": 48.49, "low": 48.46, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.49, "open": 48.46, "day": "31"} +{"volume": 598, "symbol": "ETSY", "ts": "2018-08-31 10:21:00", "month": "08", "high": 48.4654, "low": 48.4654, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.4654, "open": 48.4654, "day": "31"} +{"volume": 2270, "symbol": "ETSY", "ts": "2018-08-31 10:23:00", "month": "08", "high": 48.55, "low": 48.55, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.55, "open": 48.55, "day": "31"} +{"volume": 2855, "symbol": "ETSY", "ts": "2018-08-31 10:24:00", "month": "08", "high": 48.6462, "low": 48.6462, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.6462, "open": 48.6462, "day": "31"} +{"volume": 3082, "symbol": "ETSY", "ts": "2018-08-31 10:26:00", "month": "08", "high": 48.65, "low": 48.65, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.65, "open": 48.65, "day": "31"} +{"volume": 415, "symbol": "ETSY", "ts": "2018-08-31 10:27:00", "month": "08", "high": 48.67, "low": 48.67, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.67, "open": 48.67, "day": "31"} +{"volume": 1809, "symbol": "ETSY", "ts": "2018-08-31 10:29:00", "month": "08", "high": 48.62, "low": 48.62, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.62, "open": 48.62, "day": "31"} +{"volume": 1769, "symbol": "TWLO", "ts": "2018-08-31 10:02:00", "month": "08", "high": 80.3033, "low": 80.3033, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.3033, "open": 80.3033, "day": "31"} +{"volume": 2832, "symbol": "TWLO", "ts": "2018-08-31 10:03:00", "month": "08", "high": 80.5199, "low": 80.5199, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.5199, "open": 80.5199, "day": "31"} +{"volume": 2036, "symbol": "TWLO", "ts": "2018-08-31 10:04:00", "month": "08", "high": 80.5, "low": 80.5, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.5, "open": 80.5, "day": "31"} +{"volume": 2490, "symbol": "TWLO", "ts": "2018-08-31 10:05:00", "month": "08", "high": 80.5784, "low": 80.5784, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.5784, "open": 80.5784, "day": "31"} +{"volume": 1331, "symbol": "TWLO", "ts": "2018-08-31 10:07:00", "month": "08", "high": 80.65, "low": 80.65, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.65, "open": 80.65, "day": "31"} +{"volume": 1836, "symbol": "TWLO", "ts": "2018-08-31 10:08:00", "month": "08", "high": 80.48, "low": 80.48, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.48, "open": 80.48, "day": "31"} +{"volume": 6238, "symbol": "TWLO", "ts": "2018-08-31 10:09:00", "month": "08", "high": 80.3548, "low": 80.3548, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.3548, "open": 80.3548, "day": "31"} +{"volume": 47283, "symbol": "TWLO", "ts": "2018-08-31 10:10:00", "month": "08", "high": 80.375, "low": 80.1, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.1, "open": 80.375, "day": "31"} +{"volume": 789, "symbol": "TWLO", "ts": "2018-08-31 10:11:00", "month": "08", "high": 80.23, "low": 80.23, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.23, "open": 80.23, "day": "31"} +{"volume": 1781, "symbol": "TWLO", "ts": "2018-08-31 10:14:00", "month": "08", "high": 80.371, "low": 80.371, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.371, "open": 80.371, "day": "31"} +{"volume": 1680, "symbol": "TWLO", "ts": "2018-08-31 10:15:00", "month": "08", "high": 80.54, "low": 80.54, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.54, "open": 80.54, "day": "31"} +{"volume": 3201, "symbol": "TWLO", "ts": "2018-08-31 10:17:00", "month": "08", "high": 80.55, "low": 80.55, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.55, "open": 80.55, "day": "31"} +{"volume": 3233, "symbol": "TWLO", "ts": "2018-08-31 10:18:00", "month": "08", "high": 80.62, "low": 80.62, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.62, "open": 80.62, "day": "31"} +{"volume": 1717, "symbol": "TWLO", "ts": "2018-08-31 10:19:00", "month": "08", "high": 80.65, "low": 80.65, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.65, "open": 80.65, "day": "31"} +{"volume": 16376, "symbol": "TWLO", "ts": "2018-08-31 10:20:00", "month": "08", "high": 80.63, "low": 80.46, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.46, "open": 80.63, "day": "31"} +{"volume": 3751, "symbol": "TWLO", "ts": "2018-08-31 10:21:00", "month": "08", "high": 80.4605, "low": 80.4605, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.4605, "open": 80.4605, "day": "31"} +{"volume": 8078, "symbol": "TWLO", "ts": "2018-08-31 10:22:00", "month": "08", "high": 80.46, "low": 80.46, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.46, "open": 80.46, "day": "31"} +{"volume": 1985, "symbol": "TWLO", "ts": "2018-08-31 10:23:00", "month": "08", "high": 80.33, "low": 80.33, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.33, "open": 80.33, "day": "31"} +{"volume": 4284, "symbol": "TWLO", "ts": "2018-08-31 10:24:00", "month": "08", "high": 80.3382, "low": 80.3382, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.3382, "open": 80.3382, "day": "31"} +{"volume": 15090, "symbol": "TWLO", "ts": "2018-08-31 10:25:00", "month": "08", "high": 80.41, "low": 80.3, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.35, "open": 80.3, "day": "31"} +{"volume": 13201, "symbol": "TWLO", "ts": "2018-08-31 10:26:00", "month": "08", "high": 80.3, "low": 80.2, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.3, "open": 80.2, "day": "31"} +{"volume": 15124, "symbol": "TWLO", "ts": "2018-08-31 10:27:00", "month": "08", "high": 80.43, "low": 80.32, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.43, "open": 80.32, "day": "31"} +{"volume": 12513, "symbol": "TWLO", "ts": "2018-08-31 10:28:00", "month": "08", "high": 80.466, "low": 80.28, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.28, "open": 80.466, "day": "31"} +{"volume": 2641, "symbol": "TWLO", "ts": "2018-08-31 10:29:00", "month": "08", "high": 80.2504, "low": 80.2504, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.2504, "open": 80.2504, "day": "31"} +{"volume": 2600, "symbol": "AMED", "ts": "2018-08-31 10:01:00", "month": "08", "high": 121.995, "low": 121.995, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.995, "open": 121.995, "day": "31"} +{"volume": 1723, "symbol": "AMED", "ts": "2018-08-31 10:03:00", "month": "08", "high": 121.58, "low": 121.58, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.58, "open": 121.58, "day": "31"} +{"volume": 2906, "symbol": "AMED", "ts": "2018-08-31 10:04:00", "month": "08", "high": 121.68, "low": 121.68, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.68, "open": 121.68, "day": "31"} +{"volume": 2750, "symbol": "AMED", "ts": "2018-08-31 10:05:00", "month": "08", "high": 121.525, "low": 121.525, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.525, "open": 121.525, "day": "31"} +{"volume": 1710, "symbol": "AMED", "ts": "2018-08-31 10:06:00", "month": "08", "high": 121.53, "low": 121.53, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.53, "open": 121.53, "day": "31"} +{"volume": 1184, "symbol": "AMED", "ts": "2018-08-31 10:07:00", "month": "08", "high": 121.45, "low": 121.45, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.45, "open": 121.45, "day": "31"} +{"volume": 1503, "symbol": "AMED", "ts": "2018-08-31 10:08:00", "month": "08", "high": 121.5, "low": 121.5, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.5, "open": 121.5, "day": "31"} +{"volume": 2811, "symbol": "AMED", "ts": "2018-08-31 10:11:00", "month": "08", "high": 121.5, "low": 121.395, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.5, "open": 121.395, "day": "31"} +{"volume": 1617, "symbol": "AMED", "ts": "2018-08-31 10:14:00", "month": "08", "high": 121.3558, "low": 121.33, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.3558, "open": 121.33, "day": "31"} +{"volume": 2425, "symbol": "AMED", "ts": "2018-08-31 10:16:00", "month": "08", "high": 121.5185, "low": 121.5185, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.5185, "open": 121.5185, "day": "31"} +{"volume": 3503, "symbol": "AMED", "ts": "2018-08-31 10:17:00", "month": "08", "high": 121.51, "low": 121.51, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.51, "open": 121.51, "day": "31"} +{"volume": 677, "symbol": "AMED", "ts": "2018-08-31 10:18:00", "month": "08", "high": 121.575, "low": 121.575, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.575, "open": 121.575, "day": "31"} +{"volume": 321, "symbol": "AMED", "ts": "2018-08-31 10:19:00", "month": "08", "high": 121.696, "low": 121.696, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.696, "open": 121.696, "day": "31"} +{"volume": 1093, "symbol": "AMED", "ts": "2018-08-31 10:21:00", "month": "08", "high": 121.58, "low": 121.58, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.58, "open": 121.58, "day": "31"} +{"volume": 2399, "symbol": "AMED", "ts": "2018-08-31 10:22:00", "month": "08", "high": 121.6765, "low": 121.6765, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.6765, "open": 121.6765, "day": "31"} +{"volume": 2386, "symbol": "AMED", "ts": "2018-08-31 10:23:00", "month": "08", "high": 121.48, "low": 121.48, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.48, "open": 121.48, "day": "31"} +{"volume": 1689, "symbol": "AMED", "ts": "2018-08-31 10:24:00", "month": "08", "high": 121.6, "low": 121.6, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.6, "open": 121.6, "day": "31"} +{"volume": 1721, "symbol": "AMED", "ts": "2018-08-31 10:26:00", "month": "08", "high": 121.525, "low": 121.525, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.525, "open": 121.525, "day": "31"} +{"volume": 3391, "symbol": "AMED", "ts": "2018-08-31 10:27:00", "month": "08", "high": 121.5, "low": 121.5, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.5, "open": 121.5, "day": "31"} +{"volume": 2823, "symbol": "AMED", "ts": "2018-08-31 10:28:00", "month": "08", "high": 121.57, "low": 121.57, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.57, "open": 121.57, "day": "31"} +{"volume": 1620, "symbol": "USAT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 16.0, "low": 16.0, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.0, "open": 16.0, "day": "31"} +{"volume": 2826, "symbol": "USAT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 16.095, "low": 16.095, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.095, "open": 16.095, "day": "31"} +{"volume": 2626, "symbol": "USAT", "ts": "2018-08-31 10:07:00", "month": "08", "high": 16.0912, "low": 16.0912, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.0912, "open": 16.0912, "day": "31"} +{"volume": 2636, "symbol": "USAT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 16.0248, "low": 16.0248, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.0248, "open": 16.0248, "day": "31"} +{"volume": 1542, "symbol": "USAT", "ts": "2018-08-31 10:11:00", "month": "08", "high": 16.1, "low": 16.1, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.1, "open": 16.1, "day": "31"} +{"volume": 825, "symbol": "USAT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 16.15, "low": 16.15, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.15, "open": 16.15, "day": "31"} +{"volume": 1101, "symbol": "USAT", "ts": "2018-08-31 10:19:00", "month": "08", "high": 16.1, "low": 16.1, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.1, "open": 16.1, "day": "31"} +{"volume": 3450, "symbol": "USAT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 16.145, "low": 16.145, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.145, "open": 16.145, "day": "31"} +{"volume": 405, "symbol": "USAT", "ts": "2018-08-31 10:25:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 3106, "symbol": "USAT", "ts": "2018-08-31 10:27:00", "month": "08", "high": 16.086, "low": 16.086, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.086, "open": 16.086, "day": "31"} +{"volume": 221, "symbol": "USAT", "ts": "2018-08-31 10:29:00", "month": "08", "high": 16.1, "low": 16.1, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.1, "open": 16.1, "day": "31"} +{"volume": 2721, "symbol": "RGNX", "ts": "2018-08-31 10:01:00", "month": "08", "high": 70.25, "low": 70.25, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.25, "open": 70.25, "day": "31"} +{"volume": 1119, "symbol": "RGNX", "ts": "2018-08-31 10:03:00", "month": "08", "high": 70.35, "low": 70.35, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.35, "open": 70.35, "day": "31"} +{"volume": 1198, "symbol": "RGNX", "ts": "2018-08-31 10:04:00", "month": "08", "high": 70.45, "low": 70.45, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.45, "open": 70.45, "day": "31"} +{"volume": 2195, "symbol": "RGNX", "ts": "2018-08-31 10:06:00", "month": "08", "high": 70.45, "low": 70.45, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.45, "open": 70.45, "day": "31"} +{"volume": 1712, "symbol": "RGNX", "ts": "2018-08-31 10:07:00", "month": "08", "high": 70.5182, "low": 70.5182, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.5182, "open": 70.5182, "day": "31"} +{"volume": 1506, "symbol": "RGNX", "ts": "2018-08-31 10:08:00", "month": "08", "high": 70.6, "low": 70.6, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.6, "open": 70.6, "day": "31"} +{"volume": 530, "symbol": "RGNX", "ts": "2018-08-31 10:09:00", "month": "08", "high": 70.75, "low": 70.75, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.75, "open": 70.75, "day": "31"} +{"volume": 1573, "symbol": "RGNX", "ts": "2018-08-31 10:11:00", "month": "08", "high": 70.9, "low": 70.9, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.9, "open": 70.9, "day": "31"} +{"volume": 3062, "symbol": "RGNX", "ts": "2018-08-31 10:13:00", "month": "08", "high": 70.975, "low": 70.85, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.975, "open": 70.85, "day": "31"} +{"volume": 1561, "symbol": "RGNX", "ts": "2018-08-31 10:15:00", "month": "08", "high": 71.4, "low": 71.4, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.4, "open": 71.4, "day": "31"} +{"volume": 1686, "symbol": "RGNX", "ts": "2018-08-31 10:16:00", "month": "08", "high": 71.3, "low": 71.3, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.3, "open": 71.3, "day": "31"} +{"volume": 1535, "symbol": "RGNX", "ts": "2018-08-31 10:17:00", "month": "08", "high": 71.35, "low": 71.35, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.35, "open": 71.35, "day": "31"} +{"volume": 2387, "symbol": "RGNX", "ts": "2018-08-31 10:19:00", "month": "08", "high": 71.4, "low": 71.4, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.4, "open": 71.4, "day": "31"} +{"volume": 1293, "symbol": "RGNX", "ts": "2018-08-31 10:20:00", "month": "08", "high": 71.4923, "low": 71.4923, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.4923, "open": 71.4923, "day": "31"} +{"volume": 1297, "symbol": "RGNX", "ts": "2018-08-31 10:22:00", "month": "08", "high": 71.55, "low": 71.55, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.55, "open": 71.55, "day": "31"} +{"volume": 5291, "symbol": "RGNX", "ts": "2018-08-31 10:23:00", "month": "08", "high": 71.5, "low": 71.3, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.3, "open": 71.5, "day": "31"} +{"volume": 762, "symbol": "RGNX", "ts": "2018-08-31 10:25:00", "month": "08", "high": 71.2, "low": 71.2, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.2, "open": 71.2, "day": "31"} +{"volume": 2641, "symbol": "RGNX", "ts": "2018-08-31 10:26:00", "month": "08", "high": 71.25, "low": 71.15, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.15, "open": 71.25, "day": "31"} +{"volume": 1676, "symbol": "RGNX", "ts": "2018-08-31 10:28:00", "month": "08", "high": 71.1, "low": 71.1, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.1, "open": 71.1, "day": "31"} +{"volume": 713, "symbol": "HAE", "ts": "2018-08-31 10:01:00", "month": "08", "high": 109.56, "low": 109.56, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.56, "open": 109.56, "day": "31"} +{"volume": 781, "symbol": "HAE", "ts": "2018-08-31 10:04:00", "month": "08", "high": 109.39, "low": 109.39, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.39, "open": 109.39, "day": "31"} +{"volume": 2716, "symbol": "HAE", "ts": "2018-08-31 10:05:00", "month": "08", "high": 109.8, "low": 109.8, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.8, "open": 109.8, "day": "31"} +{"volume": 998, "symbol": "HAE", "ts": "2018-08-31 10:07:00", "month": "08", "high": 109.48, "low": 109.48, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.48, "open": 109.48, "day": "31"} +{"volume": 233, "symbol": "HAE", "ts": "2018-08-31 10:09:00", "month": "08", "high": 109.57, "low": 109.57, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.57, "open": 109.57, "day": "31"} +{"volume": 503, "symbol": "HAE", "ts": "2018-08-31 10:12:00", "month": "08", "high": 109.26, "low": 109.26, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.26, "open": 109.26, "day": "31"} +{"volume": 673, "symbol": "HAE", "ts": "2018-08-31 10:13:00", "month": "08", "high": 109.55, "low": 109.55, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.55, "open": 109.55, "day": "31"} +{"volume": 272, "symbol": "HAE", "ts": "2018-08-31 10:15:00", "month": "08", "high": 109.38, "low": 109.38, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.38, "open": 109.38, "day": "31"} +{"volume": 1727, "symbol": "HAE", "ts": "2018-08-31 10:18:00", "month": "08", "high": 109.44, "low": 109.44, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.44, "open": 109.44, "day": "31"} +{"volume": 1185, "symbol": "HAE", "ts": "2018-08-31 10:21:00", "month": "08", "high": 109.61, "low": 109.61, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.61, "open": 109.61, "day": "31"} +{"volume": 122, "symbol": "HAE", "ts": "2018-08-31 10:23:00", "month": "08", "high": 109.64, "low": 109.64, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.64, "open": 109.64, "day": "31"} +{"volume": 555, "symbol": "HAE", "ts": "2018-08-31 10:26:00", "month": "08", "high": 109.446, "low": 109.446, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.446, "open": 109.446, "day": "31"} +{"volume": 1736, "symbol": "HAE", "ts": "2018-08-31 10:29:00", "month": "08", "high": 109.5, "low": 109.5, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.5, "open": 109.5, "day": "31"} +{"volume": 10664, "symbol": "NVTA", "ts": "2018-08-31 10:01:00", "month": "08", "high": 14.97, "low": 14.94, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.97, "day": "31"} +{"volume": 51507, "symbol": "NVTA", "ts": "2018-08-31 10:02:00", "month": "08", "high": 15.18, "low": 14.93, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.15, "open": 14.93, "day": "31"} +{"volume": 28991, "symbol": "NVTA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 15.3101, "low": 15.19, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.29, "open": 15.19, "day": "31"} +{"volume": 37604, "symbol": "NVTA", "ts": "2018-08-31 10:04:00", "month": "08", "high": 15.31, "low": 15.13, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.13, "open": 15.31, "day": "31"} +{"volume": 18153, "symbol": "NVTA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 15.1027, "low": 15.08, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.1027, "open": 15.09, "day": "31"} +{"volume": 12926, "symbol": "NVTA", "ts": "2018-08-31 10:06:00", "month": "08", "high": 15.13, "low": 15.06, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.08, "open": 15.13, "day": "31"} +{"volume": 5335, "symbol": "NVTA", "ts": "2018-08-31 10:07:00", "month": "08", "high": 15.08, "low": 15.08, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.08, "open": 15.08, "day": "31"} +{"volume": 38818, "symbol": "NVTA", "ts": "2018-08-31 10:08:00", "month": "08", "high": 15.2027, "low": 15.1, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.18, "open": 15.1, "day": "31"} +{"volume": 27232, "symbol": "NVTA", "ts": "2018-08-31 10:09:00", "month": "08", "high": 15.15, "low": 15.0, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.15, "day": "31"} +{"volume": 4948, "symbol": "NVTA", "ts": "2018-08-31 10:10:00", "month": "08", "high": 15.01, "low": 15.01, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.01, "open": 15.01, "day": "31"} +{"volume": 10289, "symbol": "NVTA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 14.995, "low": 14.99, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.995, "open": 14.99, "day": "31"} +{"volume": 4840, "symbol": "NVTA", "ts": "2018-08-31 10:12:00", "month": "08", "high": 15.05, "low": 15.05, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.05, "open": 15.05, "day": "31"} +{"volume": 6677, "symbol": "NVTA", "ts": "2018-08-31 10:13:00", "month": "08", "high": 15.0, "low": 14.9996, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 14.9996, "day": "31"} +{"volume": 27104, "symbol": "NVTA", "ts": "2018-08-31 10:14:00", "month": "08", "high": 15.0, "low": 14.89, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.89, "open": 15.0, "day": "31"} +{"volume": 2908, "symbol": "NVTA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 14.935, "low": 14.935, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.935, "open": 14.935, "day": "31"} +{"volume": 13326, "symbol": "NVTA", "ts": "2018-08-31 10:16:00", "month": "08", "high": 14.925, "low": 14.89, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.89, "open": 14.925, "day": "31"} +{"volume": 3037, "symbol": "NVTA", "ts": "2018-08-31 10:17:00", "month": "08", "high": 14.88, "low": 14.88, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.88, "open": 14.88, "day": "31"} +{"volume": 7797, "symbol": "NVTA", "ts": "2018-08-31 10:18:00", "month": "08", "high": 14.87, "low": 14.86, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.87, "open": 14.86, "day": "31"} +{"volume": 2205, "symbol": "NVTA", "ts": "2018-08-31 10:19:00", "month": "08", "high": 14.85, "low": 14.85, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.85, "open": 14.85, "day": "31"} +{"volume": 14459, "symbol": "NVTA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 14.79, "low": 14.76, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.76, "open": 14.79, "day": "31"} +{"volume": 7714, "symbol": "NVTA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 14.765, "low": 14.73, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.73, "open": 14.765, "day": "31"} +{"volume": 19407, "symbol": "NVTA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 14.815, "low": 14.785, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.815, "open": 14.81, "day": "31"} +{"volume": 9984, "symbol": "NVTA", "ts": "2018-08-31 10:23:00", "month": "08", "high": 14.9, "low": 14.88, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9, "open": 14.88, "day": "31"} +{"volume": 9014, "symbol": "NVTA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 14.9, "low": 14.9, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9, "open": 14.9, "day": "31"} +{"volume": 1802, "symbol": "NVTA", "ts": "2018-08-31 10:25:00", "month": "08", "high": 14.9, "low": 14.9, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9, "open": 14.9, "day": "31"} +{"volume": 5806, "symbol": "NVTA", "ts": "2018-08-31 10:26:00", "month": "08", "high": 14.89, "low": 14.89, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.89, "open": 14.89, "day": "31"} +{"volume": 5524, "symbol": "NVTA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 14.9815, "low": 14.9, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9815, "open": 14.9, "day": "31"} +{"volume": 5626, "symbol": "NVTA", "ts": "2018-08-31 10:28:00", "month": "08", "high": 14.92, "low": 14.92, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.92, "open": 14.92, "day": "31"} +{"volume": 2685, "symbol": "NVTA", "ts": "2018-08-31 10:29:00", "month": "08", "high": 14.98, "low": 14.98, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.98, "open": 14.98, "day": "31"} +{"volume": 235, "symbol": "TPB", "ts": "2018-08-31 10:07:00", "month": "08", "high": 34.155, "low": 34.155, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 34.155, "open": 34.155, "day": "31"} +{"volume": 920, "symbol": "TPB", "ts": "2018-08-31 10:10:00", "month": "08", "high": 34.0016, "low": 34.0016, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 34.0016, "open": 34.0016, "day": "31"} +{"volume": 395, "symbol": "TPB", "ts": "2018-08-31 10:18:00", "month": "08", "high": 33.99, "low": 33.99, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.99, "open": 33.99, "day": "31"} +{"volume": 218, "symbol": "TPB", "ts": "2018-08-31 10:25:00", "month": "08", "high": 34.01, "low": 34.01, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 34.01, "open": 34.01, "day": "31"} +{"volume": 100, "symbol": "TPB", "ts": "2018-08-31 10:28:00", "month": "08", "high": 34.0, "low": 34.0, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 34.0, "open": 34.0, "day": "31"} +{"volume": 225, "symbol": "TPB", "ts": "2018-08-31 10:29:00", "month": "08", "high": 33.92, "low": 33.92, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.92, "open": 33.92, "day": "31"} +{"volume": 400, "symbol": "NTRA", "ts": "2018-08-31 10:01:00", "month": "08", "high": 27.51, "low": 27.51, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.51, "open": 27.51, "day": "31"} +{"volume": 275, "symbol": "NTRA", "ts": "2018-08-31 10:03:00", "month": "08", "high": 27.58, "low": 27.58, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.58, "open": 27.58, "day": "31"} +{"volume": 240, "symbol": "NTRA", "ts": "2018-08-31 10:05:00", "month": "08", "high": 27.57, "low": 27.57, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.57, "open": 27.57, "day": "31"} +{"volume": 502, "symbol": "NTRA", "ts": "2018-08-31 10:09:00", "month": "08", "high": 27.59, "low": 27.59, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.59, "open": 27.59, "day": "31"} +{"volume": 302, "symbol": "NTRA", "ts": "2018-08-31 10:11:00", "month": "08", "high": 27.65, "low": 27.65, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.65, "open": 27.65, "day": "31"} +{"volume": 400, "symbol": "NTRA", "ts": "2018-08-31 10:13:00", "month": "08", "high": 27.6, "low": 27.6, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.6, "open": 27.6, "day": "31"} +{"volume": 500, "symbol": "NTRA", "ts": "2018-08-31 10:15:00", "month": "08", "high": 27.68, "low": 27.67, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.68, "open": 27.67, "day": "31"} +{"volume": 347, "symbol": "NTRA", "ts": "2018-08-31 10:20:00", "month": "08", "high": 27.62, "low": 27.62, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.62, "open": 27.62, "day": "31"} +{"volume": 1703, "symbol": "NTRA", "ts": "2018-08-31 10:21:00", "month": "08", "high": 27.62, "low": 27.62, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.62, "open": 27.62, "day": "31"} +{"volume": 1266, "symbol": "NTRA", "ts": "2018-08-31 10:22:00", "month": "08", "high": 27.58, "low": 27.58, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.58, "open": 27.58, "day": "31"} +{"volume": 734, "symbol": "NTRA", "ts": "2018-08-31 10:24:00", "month": "08", "high": 27.54, "low": 27.54, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.54, "open": 27.54, "day": "31"} +{"volume": 2140, "symbol": "NTRA", "ts": "2018-08-31 10:26:00", "month": "08", "high": 27.5, "low": 27.5, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.5, "open": 27.5, "day": "31"} +{"volume": 3175, "symbol": "NTRA", "ts": "2018-08-31 10:27:00", "month": "08", "high": 27.5, "low": 27.5, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.5, "open": 27.5, "day": "31"} +{"volume": 2554, "symbol": "WTI", "ts": "2018-08-31 10:02:00", "month": "08", "high": 6.78, "low": 6.78, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.78, "open": 6.78, "day": "31"} +{"volume": 8500, "symbol": "WTI", "ts": "2018-08-31 10:03:00", "month": "08", "high": 6.77, "low": 6.77, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.77, "open": 6.77, "day": "31"} +{"volume": 1822, "symbol": "WTI", "ts": "2018-08-31 10:04:00", "month": "08", "high": 6.78, "low": 6.78, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.78, "open": 6.78, "day": "31"} +{"volume": 11393, "symbol": "WTI", "ts": "2018-08-31 10:05:00", "month": "08", "high": 6.7747, "low": 6.7747, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.7747, "open": 6.7747, "day": "31"} +{"volume": 3312, "symbol": "WTI", "ts": "2018-08-31 10:07:00", "month": "08", "high": 6.79, "low": 6.79, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.79, "open": 6.79, "day": "31"} +{"volume": 2236, "symbol": "WTI", "ts": "2018-08-31 10:08:00", "month": "08", "high": 6.78, "low": 6.78, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.78, "open": 6.78, "day": "31"} +{"volume": 6075, "symbol": "WTI", "ts": "2018-08-31 10:09:00", "month": "08", "high": 6.77, "low": 6.77, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.77, "open": 6.77, "day": "31"} +{"volume": 2912, "symbol": "WTI", "ts": "2018-08-31 10:10:00", "month": "08", "high": 6.7801, "low": 6.7801, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.7801, "open": 6.7801, "day": "31"} +{"volume": 11285, "symbol": "WTI", "ts": "2018-08-31 10:11:00", "month": "08", "high": 6.79, "low": 6.79, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.79, "open": 6.79, "day": "31"} +{"volume": 25230, "symbol": "WTI", "ts": "2018-08-31 10:12:00", "month": "08", "high": 6.79, "low": 6.79, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.79, "open": 6.79, "day": "31"} +{"volume": 18819, "symbol": "WTI", "ts": "2018-08-31 10:13:00", "month": "08", "high": 6.8099, "low": 6.8, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.8, "open": 6.8099, "day": "31"} +{"volume": 8570, "symbol": "WTI", "ts": "2018-08-31 10:15:00", "month": "08", "high": 6.805, "low": 6.805, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.805, "open": 6.805, "day": "31"} +{"volume": 5996, "symbol": "WTI", "ts": "2018-08-31 10:16:00", "month": "08", "high": 6.805, "low": 6.805, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.805, "open": 6.805, "day": "31"} +{"volume": 4200, "symbol": "WTI", "ts": "2018-08-31 10:17:00", "month": "08", "high": 6.79, "low": 6.79, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.79, "open": 6.79, "day": "31"} +{"volume": 985, "symbol": "WTI", "ts": "2018-08-31 10:18:00", "month": "08", "high": 6.79, "low": 6.79, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.79, "open": 6.79, "day": "31"} +{"volume": 3296, "symbol": "WTI", "ts": "2018-08-31 10:19:00", "month": "08", "high": 6.8, "low": 6.8, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.8, "open": 6.8, "day": "31"} +{"volume": 3254, "symbol": "WTI", "ts": "2018-08-31 10:21:00", "month": "08", "high": 6.7997, "low": 6.7997, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.7997, "open": 6.7997, "day": "31"} +{"volume": 2900, "symbol": "WTI", "ts": "2018-08-31 10:22:00", "month": "08", "high": 6.805, "low": 6.805, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.805, "open": 6.805, "day": "31"} +{"volume": 2940, "symbol": "WTI", "ts": "2018-08-31 10:23:00", "month": "08", "high": 6.81, "low": 6.81, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.81, "open": 6.81, "day": "31"} +{"volume": 5671, "symbol": "WTI", "ts": "2018-08-31 10:25:00", "month": "08", "high": 6.815, "low": 6.81, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.815, "open": 6.81, "day": "31"} +{"volume": 2479, "symbol": "WTI", "ts": "2018-08-31 10:27:00", "month": "08", "high": 6.8, "low": 6.8, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.8, "open": 6.8, "day": "31"} +{"volume": 1998, "symbol": "WTI", "ts": "2018-08-31 10:29:00", "month": "08", "high": 6.795, "low": 6.795, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.795, "open": 6.795, "day": "31"} +{"volume": 500, "symbol": "RCM", "ts": "2018-08-31 10:03:00", "month": "08", "high": 9.945, "low": 9.945, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.945, "open": 9.945, "day": "31"} +{"volume": 505, "symbol": "RCM", "ts": "2018-08-31 10:04:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 204, "symbol": "RCM", "ts": "2018-08-31 10:05:00", "month": "08", "high": 9.94, "low": 9.94, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 9.94, "day": "31"} +{"volume": 200, "symbol": "RCM", "ts": "2018-08-31 10:07:00", "month": "08", "high": 9.94, "low": 9.94, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 9.94, "day": "31"} +{"volume": 700, "symbol": "RCM", "ts": "2018-08-31 10:08:00", "month": "08", "high": 9.95, "low": 9.95, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.95, "open": 9.95, "day": "31"} +{"volume": 703, "symbol": "RCM", "ts": "2018-08-31 10:09:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 5025, "symbol": "RCM", "ts": "2018-08-31 10:12:00", "month": "08", "high": 9.92, "low": 9.9, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.92, "open": 9.9, "day": "31"} +{"volume": 802, "symbol": "RCM", "ts": "2018-08-31 10:15:00", "month": "08", "high": 9.905, "low": 9.905, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.905, "open": 9.905, "day": "31"} +{"volume": 251, "symbol": "RCM", "ts": "2018-08-31 10:18:00", "month": "08", "high": 9.905, "low": 9.905, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.905, "open": 9.905, "day": "31"} +{"volume": 1505, "symbol": "RCM", "ts": "2018-08-31 10:19:00", "month": "08", "high": 9.92, "low": 9.92, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.92, "open": 9.92, "day": "31"} +{"volume": 303, "symbol": "RCM", "ts": "2018-08-31 10:21:00", "month": "08", "high": 9.92, "low": 9.92, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.92, "open": 9.92, "day": "31"} +{"volume": 1569, "symbol": "RCM", "ts": "2018-08-31 10:23:00", "month": "08", "high": 9.92, "low": 9.92, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.92, "open": 9.92, "day": "31"} +{"volume": 2205, "symbol": "RCM", "ts": "2018-08-31 10:25:00", "month": "08", "high": 9.94, "low": 9.94, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 9.94, "day": "31"} +{"volume": 700, "symbol": "RCM", "ts": "2018-08-31 10:29:00", "month": "08", "high": 9.94, "low": 9.94, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 9.94, "day": "31"} +{"volume": 108818, "symbol": "LULU", "ts": "2018-08-31 10:01:00", "month": "08", "high": 157.3212, "low": 156.64, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.73, "open": 157.11, "day": "31"} +{"volume": 101339, "symbol": "LULU", "ts": "2018-08-31 10:02:00", "month": "08", "high": 157.04, "low": 156.5, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.765, "open": 156.5, "day": "31"} +{"volume": 72214, "symbol": "LULU", "ts": "2018-08-31 10:03:00", "month": "08", "high": 157.17, "low": 156.9, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.13, "open": 156.9, "day": "31"} +{"volume": 66014, "symbol": "LULU", "ts": "2018-08-31 10:04:00", "month": "08", "high": 157.58, "low": 157.09, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.58, "open": 157.09, "day": "31"} +{"volume": 70575, "symbol": "LULU", "ts": "2018-08-31 10:05:00", "month": "08", "high": 157.88, "low": 157.59, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.7429, "open": 157.59, "day": "31"} +{"volume": 54747, "symbol": "LULU", "ts": "2018-08-31 10:06:00", "month": "08", "high": 157.74, "low": 157.04, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.2, "open": 157.74, "day": "31"} +{"volume": 57075, "symbol": "LULU", "ts": "2018-08-31 10:07:00", "month": "08", "high": 157.3664, "low": 156.75, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.75, "open": 157.105, "day": "31"} +{"volume": 43952, "symbol": "LULU", "ts": "2018-08-31 10:08:00", "month": "08", "high": 156.94, "low": 156.44, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.6114, "open": 156.78, "day": "31"} +{"volume": 42121, "symbol": "LULU", "ts": "2018-08-31 10:09:00", "month": "08", "high": 157.1635, "low": 156.55, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.02, "open": 156.55, "day": "31"} +{"volume": 31452, "symbol": "LULU", "ts": "2018-08-31 10:10:00", "month": "08", "high": 157.23, "low": 156.821, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.821, "open": 156.99, "day": "31"} +{"volume": 22123, "symbol": "LULU", "ts": "2018-08-31 10:11:00", "month": "08", "high": 157.05, "low": 156.9, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.9, "open": 156.96, "day": "31"} +{"volume": 27992, "symbol": "LULU", "ts": "2018-08-31 10:12:00", "month": "08", "high": 157.35, "low": 156.97, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.253, "open": 156.97, "day": "31"} +{"volume": 54442, "symbol": "LULU", "ts": "2018-08-31 10:13:00", "month": "08", "high": 157.6015, "low": 157.18, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.39, "open": 157.23, "day": "31"} +{"volume": 39023, "symbol": "LULU", "ts": "2018-08-31 10:14:00", "month": "08", "high": 157.5418, "low": 157.32, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.485, "open": 157.32, "day": "31"} +{"volume": 19495, "symbol": "LULU", "ts": "2018-08-31 10:15:00", "month": "08", "high": 157.44, "low": 157.27, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.27, "open": 157.44, "day": "31"} +{"volume": 63781, "symbol": "LULU", "ts": "2018-08-31 10:16:00", "month": "08", "high": 157.71, "low": 157.24, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.61, "open": 157.24, "day": "31"} +{"volume": 38118, "symbol": "LULU", "ts": "2018-08-31 10:17:00", "month": "08", "high": 157.839, "low": 157.61, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.78, "open": 157.67, "day": "31"} +{"volume": 30222, "symbol": "LULU", "ts": "2018-08-31 10:18:00", "month": "08", "high": 157.82, "low": 157.5735, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.8, "open": 157.72, "day": "31"} +{"volume": 25938, "symbol": "LULU", "ts": "2018-08-31 10:19:00", "month": "08", "high": 157.85, "low": 157.5839, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.85, "open": 157.72, "day": "31"} +{"volume": 35280, "symbol": "LULU", "ts": "2018-08-31 10:20:00", "month": "08", "high": 157.97, "low": 157.83, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.83, "open": 157.91, "day": "31"} +{"volume": 29698, "symbol": "LULU", "ts": "2018-08-31 10:21:00", "month": "08", "high": 157.91, "low": 157.77, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.77, "open": 157.82, "day": "31"} +{"volume": 46921, "symbol": "LULU", "ts": "2018-08-31 10:22:00", "month": "08", "high": 157.6918, "low": 157.09, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.1057, "open": 157.6918, "day": "31"} +{"volume": 33242, "symbol": "LULU", "ts": "2018-08-31 10:23:00", "month": "08", "high": 157.19, "low": 156.781, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.781, "open": 157.1249, "day": "31"} +{"volume": 33069, "symbol": "LULU", "ts": "2018-08-31 10:24:00", "month": "08", "high": 157.07, "low": 156.74, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.85, "open": 156.91, "day": "31"} +{"volume": 21490, "symbol": "LULU", "ts": "2018-08-31 10:25:00", "month": "08", "high": 157.23, "low": 157.05, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.17, "open": 157.0883, "day": "31"} +{"volume": 42717, "symbol": "LULU", "ts": "2018-08-31 10:26:00", "month": "08", "high": 157.1952, "low": 156.68, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.82, "open": 157.18, "day": "31"} +{"volume": 55025, "symbol": "LULU", "ts": "2018-08-31 10:27:00", "month": "08", "high": 156.94, "low": 156.66, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.822, "open": 156.94, "day": "31"} +{"volume": 120501, "symbol": "LULU", "ts": "2018-08-31 10:28:00", "month": "08", "high": 156.87, "low": 155.8135, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 155.961, "open": 156.87, "day": "31"} +{"volume": 59160, "symbol": "LULU", "ts": "2018-08-31 10:29:00", "month": "08", "high": 155.9827, "low": 155.75, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 155.9, "open": 155.87, "day": "31"} +{"volume": 64445, "symbol": "YEXT", "ts": "2018-08-31 10:01:00", "month": "08", "high": 25.18, "low": 24.815, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.815, "open": 25.18, "day": "31"} +{"volume": 3252, "symbol": "YEXT", "ts": "2018-08-31 10:02:00", "month": "08", "high": 24.99, "low": 24.99, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.99, "open": 24.99, "day": "31"} +{"volume": 11838, "symbol": "YEXT", "ts": "2018-08-31 10:03:00", "month": "08", "high": 25.28, "low": 25.1362, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.28, "open": 25.1362, "day": "31"} +{"volume": 13753, "symbol": "YEXT", "ts": "2018-08-31 10:04:00", "month": "08", "high": 25.2212, "low": 25.16, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.2212, "open": 25.16, "day": "31"} +{"volume": 9884, "symbol": "YEXT", "ts": "2018-08-31 10:05:00", "month": "08", "high": 25.33, "low": 25.3237, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3237, "open": 25.33, "day": "31"} +{"volume": 64582, "symbol": "YEXT", "ts": "2018-08-31 10:07:00", "month": "08", "high": 25.3, "low": 25.22, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.22, "day": "31"} +{"volume": 3205, "symbol": "YEXT", "ts": "2018-08-31 10:08:00", "month": "08", "high": 25.3, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.3, "day": "31"} +{"volume": 7389, "symbol": "YEXT", "ts": "2018-08-31 10:09:00", "month": "08", "high": 25.3, "low": 25.29, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.29, "day": "31"} +{"volume": 4593, "symbol": "YEXT", "ts": "2018-08-31 10:10:00", "month": "08", "high": 25.26, "low": 25.26, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.26, "open": 25.26, "day": "31"} +{"volume": 8549, "symbol": "YEXT", "ts": "2018-08-31 10:11:00", "month": "08", "high": 25.2269, "low": 25.22, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.22, "open": 25.2269, "day": "31"} +{"volume": 3400, "symbol": "YEXT", "ts": "2018-08-31 10:12:00", "month": "08", "high": 25.265, "low": 25.265, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.265, "open": 25.265, "day": "31"} +{"volume": 1565, "symbol": "YEXT", "ts": "2018-08-31 10:13:00", "month": "08", "high": 25.3, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.3, "day": "31"} +{"volume": 2561, "symbol": "YEXT", "ts": "2018-08-31 10:14:00", "month": "08", "high": 25.26, "low": 25.26, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.26, "open": 25.26, "day": "31"} +{"volume": 1451, "symbol": "YEXT", "ts": "2018-08-31 10:15:00", "month": "08", "high": 25.28, "low": 25.28, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.28, "open": 25.28, "day": "31"} +{"volume": 19311, "symbol": "YEXT", "ts": "2018-08-31 10:16:00", "month": "08", "high": 25.34, "low": 25.3205, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.33, "open": 25.34, "day": "31"} +{"volume": 3510, "symbol": "YEXT", "ts": "2018-08-31 10:17:00", "month": "08", "high": 25.2595, "low": 25.2595, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.2595, "open": 25.2595, "day": "31"} +{"volume": 10360, "symbol": "YEXT", "ts": "2018-08-31 10:18:00", "month": "08", "high": 25.32, "low": 25.31, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.31, "open": 25.32, "day": "31"} +{"volume": 8865, "symbol": "YEXT", "ts": "2018-08-31 10:19:00", "month": "08", "high": 25.34, "low": 25.33, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.34, "open": 25.33, "day": "31"} +{"volume": 45505, "symbol": "YEXT", "ts": "2018-08-31 10:20:00", "month": "08", "high": 25.34, "low": 25.34, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.34, "open": 25.34, "day": "31"} +{"volume": 40967, "symbol": "YEXT", "ts": "2018-08-31 10:21:00", "month": "08", "high": 25.4878, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.485, "open": 25.3, "day": "31"} +{"volume": 10892, "symbol": "YEXT", "ts": "2018-08-31 10:22:00", "month": "08", "high": 25.45, "low": 25.39, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.39, "open": 25.45, "day": "31"} +{"volume": 6658, "symbol": "YEXT", "ts": "2018-08-31 10:23:00", "month": "08", "high": 25.4281, "low": 25.4281, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.4281, "open": 25.4281, "day": "31"} +{"volume": 2039, "symbol": "YEXT", "ts": "2018-08-31 10:24:00", "month": "08", "high": 25.475, "low": 25.475, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.475, "open": 25.475, "day": "31"} +{"volume": 3621, "symbol": "YEXT", "ts": "2018-08-31 10:25:00", "month": "08", "high": 25.41, "low": 25.41, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.41, "open": 25.41, "day": "31"} +{"volume": 6851, "symbol": "YEXT", "ts": "2018-08-31 10:26:00", "month": "08", "high": 25.47, "low": 25.43, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.47, "open": 25.43, "day": "31"} +{"volume": 3420, "symbol": "YEXT", "ts": "2018-08-31 10:27:00", "month": "08", "high": 25.44, "low": 25.44, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.44, "open": 25.44, "day": "31"} +{"volume": 5539, "symbol": "YEXT", "ts": "2018-08-31 10:28:00", "month": "08", "high": 25.43, "low": 25.43, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.43, "open": 25.43, "day": "31"} +{"volume": 2799, "symbol": "YEXT", "ts": "2018-08-31 10:29:00", "month": "08", "high": 25.47, "low": 25.47, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.47, "open": 25.47, "day": "31"} +{"volume": 1161, "symbol": "GRUB", "ts": "2018-08-31 10:01:00", "month": "08", "high": 142.5637, "low": 142.5637, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.5637, "open": 142.5637, "day": "31"} +{"volume": 752, "symbol": "GRUB", "ts": "2018-08-31 10:03:00", "month": "08", "high": 142.6022, "low": 142.6022, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.6022, "open": 142.6022, "day": "31"} +{"volume": 2416, "symbol": "GRUB", "ts": "2018-08-31 10:04:00", "month": "08", "high": 142.54, "low": 142.54, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.54, "open": 142.54, "day": "31"} +{"volume": 1066, "symbol": "GRUB", "ts": "2018-08-31 10:05:00", "month": "08", "high": 142.704, "low": 142.704, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.704, "open": 142.704, "day": "31"} +{"volume": 1563, "symbol": "GRUB", "ts": "2018-08-31 10:07:00", "month": "08", "high": 142.6417, "low": 142.6417, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.6417, "open": 142.6417, "day": "31"} +{"volume": 2161, "symbol": "GRUB", "ts": "2018-08-31 10:08:00", "month": "08", "high": 142.38, "low": 142.38, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.38, "open": 142.38, "day": "31"} +{"volume": 1129, "symbol": "GRUB", "ts": "2018-08-31 10:09:00", "month": "08", "high": 142.425, "low": 142.425, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.425, "open": 142.425, "day": "31"} +{"volume": 305, "symbol": "GRUB", "ts": "2018-08-31 10:10:00", "month": "08", "high": 142.27, "low": 142.27, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.27, "open": 142.27, "day": "31"} +{"volume": 534, "symbol": "GRUB", "ts": "2018-08-31 10:12:00", "month": "08", "high": 142.3177, "low": 142.3177, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.3177, "open": 142.3177, "day": "31"} +{"volume": 546, "symbol": "GRUB", "ts": "2018-08-31 10:13:00", "month": "08", "high": 142.305, "low": 142.305, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.305, "open": 142.305, "day": "31"} +{"volume": 2033, "symbol": "GRUB", "ts": "2018-08-31 10:15:00", "month": "08", "high": 142.95, "low": 142.95, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.95, "open": 142.95, "day": "31"} +{"volume": 457, "symbol": "GRUB", "ts": "2018-08-31 10:16:00", "month": "08", "high": 142.915, "low": 142.915, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.915, "open": 142.915, "day": "31"} +{"volume": 1982, "symbol": "GRUB", "ts": "2018-08-31 10:18:00", "month": "08", "high": 143.08, "low": 143.08, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.08, "open": 143.08, "day": "31"} +{"volume": 1331, "symbol": "GRUB", "ts": "2018-08-31 10:20:00", "month": "08", "high": 142.909, "low": 142.909, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.909, "open": 142.909, "day": "31"} +{"volume": 730, "symbol": "GRUB", "ts": "2018-08-31 10:21:00", "month": "08", "high": 142.7, "low": 142.7, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.7, "open": 142.7, "day": "31"} +{"volume": 1021, "symbol": "GRUB", "ts": "2018-08-31 10:23:00", "month": "08", "high": 142.715, "low": 142.715, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.715, "open": 142.715, "day": "31"} +{"volume": 1948, "symbol": "GRUB", "ts": "2018-08-31 10:25:00", "month": "08", "high": 142.71, "low": 142.71, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.71, "open": 142.71, "day": "31"} +{"volume": 643, "symbol": "GRUB", "ts": "2018-08-31 10:27:00", "month": "08", "high": 142.61, "low": 142.61, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.61, "open": 142.61, "day": "31"} +{"volume": 1456, "symbol": "GRUB", "ts": "2018-08-31 10:28:00", "month": "08", "high": 142.76, "low": 142.76, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.76, "open": 142.76, "day": "31"} +{"volume": 1991, "symbol": "GRUB", "ts": "2018-08-31 10:29:00", "month": "08", "high": 142.54, "low": 142.54, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.54, "open": 142.54, "day": "31"} +{"volume": 1451, "symbol": "DXCM", "ts": "2018-08-31 10:02:00", "month": "08", "high": 145.0741, "low": 145.0741, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 145.0741, "open": 145.0741, "day": "31"} +{"volume": 1606, "symbol": "DXCM", "ts": "2018-08-31 10:03:00", "month": "08", "high": 145.1, "low": 145.1, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 145.1, "open": 145.1, "day": "31"} +{"volume": 2662, "symbol": "DXCM", "ts": "2018-08-31 10:05:00", "month": "08", "high": 145.09, "low": 145.09, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 145.09, "open": 145.09, "day": "31"} +{"volume": 6929, "symbol": "DXCM", "ts": "2018-08-31 10:06:00", "month": "08", "high": 144.99, "low": 144.7, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.7, "open": 144.99, "day": "31"} +{"volume": 463, "symbol": "DXCM", "ts": "2018-08-31 10:08:00", "month": "08", "high": 144.575, "low": 144.575, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.575, "open": 144.575, "day": "31"} +{"volume": 1659, "symbol": "DXCM", "ts": "2018-08-31 10:09:00", "month": "08", "high": 144.45, "low": 144.45, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.45, "open": 144.45, "day": "31"} +{"volume": 1790, "symbol": "DXCM", "ts": "2018-08-31 10:11:00", "month": "08", "high": 144.585, "low": 144.575, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.575, "open": 144.585, "day": "31"} +{"volume": 2237, "symbol": "DXCM", "ts": "2018-08-31 10:13:00", "month": "08", "high": 144.49, "low": 144.49, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.49, "open": 144.49, "day": "31"} +{"volume": 1245, "symbol": "DXCM", "ts": "2018-08-31 10:14:00", "month": "08", "high": 144.4581, "low": 144.4581, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.4581, "open": 144.4581, "day": "31"} +{"volume": 525, "symbol": "DXCM", "ts": "2018-08-31 10:17:00", "month": "08", "high": 144.38, "low": 144.38, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.38, "open": 144.38, "day": "31"} +{"volume": 724, "symbol": "DXCM", "ts": "2018-08-31 10:20:00", "month": "08", "high": 144.455, "low": 144.455, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.455, "open": 144.455, "day": "31"} +{"volume": 1532, "symbol": "DXCM", "ts": "2018-08-31 10:21:00", "month": "08", "high": 144.5984, "low": 144.5984, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.5984, "open": 144.5984, "day": "31"} +{"volume": 1113, "symbol": "DXCM", "ts": "2018-08-31 10:24:00", "month": "08", "high": 144.73, "low": 144.73, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.73, "open": 144.73, "day": "31"} +{"volume": 430, "symbol": "DXCM", "ts": "2018-08-31 10:26:00", "month": "08", "high": 145.03, "low": 145.03, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 145.03, "open": 145.03, "day": "31"} +{"volume": 2336, "symbol": "DXCM", "ts": "2018-08-31 10:27:00", "month": "08", "high": 144.74, "low": 144.74, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.74, "open": 144.74, "day": "31"} +{"volume": 958, "symbol": "DXCM", "ts": "2018-08-31 10:28:00", "month": "08", "high": 144.69, "low": 144.69, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.69, "open": 144.69, "day": "31"} +{"volume": 802, "symbol": "QURE", "ts": "2018-08-31 10:01:00", "month": "08", "high": 42.13, "low": 42.13, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.13, "open": 42.13, "day": "31"} +{"volume": 100, "symbol": "QURE", "ts": "2018-08-31 10:03:00", "month": "08", "high": 42.14, "low": 42.14, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.14, "open": 42.14, "day": "31"} +{"volume": 210, "symbol": "QURE", "ts": "2018-08-31 10:06:00", "month": "08", "high": 42.13, "low": 42.13, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.13, "open": 42.13, "day": "31"} +{"volume": 321, "symbol": "QURE", "ts": "2018-08-31 10:08:00", "month": "08", "high": 42.1216, "low": 42.1216, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.1216, "open": 42.1216, "day": "31"} +{"volume": 301, "symbol": "QURE", "ts": "2018-08-31 10:11:00", "month": "08", "high": 42.1, "low": 42.1, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.1, "open": 42.1, "day": "31"} +{"volume": 301, "symbol": "QURE", "ts": "2018-08-31 10:12:00", "month": "08", "high": 41.9723, "low": 41.9723, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.9723, "open": 41.9723, "day": "31"} +{"volume": 506, "symbol": "QURE", "ts": "2018-08-31 10:15:00", "month": "08", "high": 42.06, "low": 42.06, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.06, "open": 42.06, "day": "31"} +{"volume": 203, "symbol": "QURE", "ts": "2018-08-31 10:17:00", "month": "08", "high": 41.95, "low": 41.95, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.95, "open": 41.95, "day": "31"} +{"volume": 536, "symbol": "QURE", "ts": "2018-08-31 10:21:00", "month": "08", "high": 42.01, "low": 42.01, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.01, "open": 42.01, "day": "31"} +{"volume": 101, "symbol": "QURE", "ts": "2018-08-31 10:24:00", "month": "08", "high": 42.08, "low": 42.08, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.08, "open": 42.08, "day": "31"} +{"volume": 101, "symbol": "QURE", "ts": "2018-08-31 10:27:00", "month": "08", "high": 42.04, "low": 42.04, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.04, "open": 42.04, "day": "31"} +{"volume": 630, "symbol": "QURE", "ts": "2018-08-31 10:28:00", "month": "08", "high": 42.04, "low": 42.04, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.04, "open": 42.04, "day": "31"} +{"volume": 15364, "symbol": "CRM", "ts": "2018-08-31 10:01:00", "month": "08", "high": 152.71, "low": 152.64, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.69, "open": 152.71, "day": "31"} +{"volume": 13801, "symbol": "CRM", "ts": "2018-08-31 10:02:00", "month": "08", "high": 152.76, "low": 152.64, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.76, "open": 152.64, "day": "31"} +{"volume": 21007, "symbol": "CRM", "ts": "2018-08-31 10:03:00", "month": "08", "high": 152.83, "low": 152.76, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.83, "open": 152.7635, "day": "31"} +{"volume": 4166, "symbol": "CRM", "ts": "2018-08-31 10:04:00", "month": "08", "high": 152.85, "low": 152.85, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.85, "open": 152.85, "day": "31"} +{"volume": 19082, "symbol": "CRM", "ts": "2018-08-31 10:05:00", "month": "08", "high": 152.8999, "low": 152.81, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.85, "open": 152.88, "day": "31"} +{"volume": 20237, "symbol": "CRM", "ts": "2018-08-31 10:06:00", "month": "08", "high": 152.84, "low": 152.801, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.81, "open": 152.82, "day": "31"} +{"volume": 9962, "symbol": "CRM", "ts": "2018-08-31 10:07:00", "month": "08", "high": 152.82, "low": 152.75, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.82, "open": 152.79, "day": "31"} +{"volume": 10032, "symbol": "CRM", "ts": "2018-08-31 10:08:00", "month": "08", "high": 152.9, "low": 152.81, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.81, "open": 152.9, "day": "31"} +{"volume": 17233, "symbol": "CRM", "ts": "2018-08-31 10:09:00", "month": "08", "high": 152.9, "low": 152.745, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.9, "open": 152.77, "day": "31"} +{"volume": 16706, "symbol": "CRM", "ts": "2018-08-31 10:10:00", "month": "08", "high": 152.89, "low": 152.83, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.89, "open": 152.84, "day": "31"} +{"volume": 19852, "symbol": "CRM", "ts": "2018-08-31 10:11:00", "month": "08", "high": 152.83, "low": 152.77, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.79, "open": 152.83, "day": "31"} +{"volume": 19058, "symbol": "CRM", "ts": "2018-08-31 10:12:00", "month": "08", "high": 152.78, "low": 152.65, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.78, "open": 152.71, "day": "31"} +{"volume": 9767, "symbol": "CRM", "ts": "2018-08-31 10:13:00", "month": "08", "high": 152.94, "low": 152.875, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.94, "open": 152.88, "day": "31"} +{"volume": 16875, "symbol": "CRM", "ts": "2018-08-31 10:14:00", "month": "08", "high": 153.115, "low": 153.0, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.08, "open": 153.03, "day": "31"} +{"volume": 8557, "symbol": "CRM", "ts": "2018-08-31 10:15:00", "month": "08", "high": 153.08, "low": 153.05, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.075, "open": 153.05, "day": "31"} +{"volume": 12625, "symbol": "CRM", "ts": "2018-08-31 10:16:00", "month": "08", "high": 153.18, "low": 153.11, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.18, "open": 153.11, "day": "31"} +{"volume": 14545, "symbol": "CRM", "ts": "2018-08-31 10:17:00", "month": "08", "high": 153.24, "low": 153.18, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.24, "open": 153.24, "day": "31"} +{"volume": 16819, "symbol": "CRM", "ts": "2018-08-31 10:18:00", "month": "08", "high": 153.25, "low": 153.11, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.11, "open": 153.25, "day": "31"} +{"volume": 10313, "symbol": "CRM", "ts": "2018-08-31 10:19:00", "month": "08", "high": 153.11, "low": 153.09, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.09, "open": 153.1006, "day": "31"} +{"volume": 6413, "symbol": "CRM", "ts": "2018-08-31 10:20:00", "month": "08", "high": 153.14, "low": 153.08, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.08, "open": 153.14, "day": "31"} +{"volume": 9176, "symbol": "CRM", "ts": "2018-08-31 10:21:00", "month": "08", "high": 153.18, "low": 153.14, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.14, "open": 153.16, "day": "31"} +{"volume": 8861, "symbol": "CRM", "ts": "2018-08-31 10:22:00", "month": "08", "high": 153.21, "low": 153.17, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.21, "open": 153.17, "day": "31"} +{"volume": 19287, "symbol": "CRM", "ts": "2018-08-31 10:23:00", "month": "08", "high": 153.29, "low": 153.23, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.29, "open": 153.23, "day": "31"} +{"volume": 15887, "symbol": "CRM", "ts": "2018-08-31 10:24:00", "month": "08", "high": 153.37, "low": 153.33, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.37, "open": 153.33, "day": "31"} +{"volume": 19552, "symbol": "CRM", "ts": "2018-08-31 10:25:00", "month": "08", "high": 153.34, "low": 153.2769, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.315, "open": 153.34, "day": "31"} +{"volume": 11182, "symbol": "CRM", "ts": "2018-08-31 10:26:00", "month": "08", "high": 153.29, "low": 153.2, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.2, "open": 153.29, "day": "31"} +{"volume": 10294, "symbol": "CRM", "ts": "2018-08-31 10:27:00", "month": "08", "high": 153.14, "low": 153.0607, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.0607, "open": 153.14, "day": "31"} +{"volume": 6031, "symbol": "CRM", "ts": "2018-08-31 10:28:00", "month": "08", "high": 153.0, "low": 152.88, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.88, "open": 153.0, "day": "31"} +{"volume": 6057, "symbol": "CRM", "ts": "2018-08-31 10:29:00", "month": "08", "high": 152.88, "low": 152.87, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.87, "open": 152.88, "day": "31"} diff --git a/docker/demo/data/batch_2.json b/docker/demo/data/batch_2.json new file mode 100644 index 0000000000000..392f6432ee7b4 --- /dev/null +++ b/docker/demo/data/batch_2.json @@ -0,0 +1,1668 @@ +{"volume": 24834, "symbol": "MSFT", "ts": "2018-08-31 10:31:00", "month": "08", "high": 112.665, "low": 112.6472, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.665, "open": 112.6472, "day": "31"} +{"volume": 33205, "symbol": "MSFT", "ts": "2018-08-31 10:32:00", "month": "08", "high": 112.675, "low": 112.64, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.67, "open": 112.65, "day": "31"} +{"volume": 59232, "symbol": "MSFT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 112.71, "low": 112.6, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.6, "open": 112.68, "day": "31"} +{"volume": 35439, "symbol": "MSFT", "ts": "2018-08-31 10:34:00", "month": "08", "high": 112.6399, "low": 112.57, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.625, "open": 112.61, "day": "31"} +{"volume": 47799, "symbol": "MSFT", "ts": "2018-08-31 10:35:00", "month": "08", "high": 112.6216, "low": 112.54, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.555, "open": 112.6216, "day": "31"} +{"volume": 20886, "symbol": "MSFT", "ts": "2018-08-31 10:36:00", "month": "08", "high": 112.56, "low": 112.5447, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.5447, "open": 112.55, "day": "31"} +{"volume": 30714, "symbol": "MSFT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 112.6, "low": 112.56, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.585, "open": 112.56, "day": "31"} +{"volume": 32687, "symbol": "MSFT", "ts": "2018-08-31 10:38:00", "month": "08", "high": 112.59, "low": 112.52, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.52, "open": 112.59, "day": "31"} +{"volume": 46630, "symbol": "MSFT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 112.59, "low": 112.5, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.59, "open": 112.53, "day": "31"} +{"volume": 24633, "symbol": "MSFT", "ts": "2018-08-31 10:40:00", "month": "08", "high": 112.6275, "low": 112.585, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.6275, "open": 112.585, "day": "31"} +{"volume": 34110, "symbol": "MSFT", "ts": "2018-08-31 10:41:00", "month": "08", "high": 112.66, "low": 112.6399, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.66, "open": 112.6399, "day": "31"} +{"volume": 43770, "symbol": "MSFT", "ts": "2018-08-31 10:42:00", "month": "08", "high": 112.64, "low": 112.56, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.56, "open": 112.64, "day": "31"} +{"volume": 20708, "symbol": "MSFT", "ts": "2018-08-31 10:43:00", "month": "08", "high": 112.635, "low": 112.58, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.635, "open": 112.58, "day": "31"} +{"volume": 45825, "symbol": "MSFT", "ts": "2018-08-31 10:44:00", "month": "08", "high": 112.61, "low": 112.58, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.6, "open": 112.61, "day": "31"} +{"volume": 30108, "symbol": "MSFT", "ts": "2018-08-31 10:45:00", "month": "08", "high": 112.59, "low": 112.57, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.59, "open": 112.58, "day": "31"} +{"volume": 35795, "symbol": "MSFT", "ts": "2018-08-31 10:46:00", "month": "08", "high": 112.63, "low": 112.61, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.63, "open": 112.61, "day": "31"} +{"volume": 36408, "symbol": "MSFT", "ts": "2018-08-31 10:47:00", "month": "08", "high": 112.63, "low": 112.57, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.58, "open": 112.63, "day": "31"} +{"volume": 50346, "symbol": "MSFT", "ts": "2018-08-31 10:48:00", "month": "08", "high": 112.57, "low": 112.53, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.53, "open": 112.57, "day": "31"} +{"volume": 26997, "symbol": "MSFT", "ts": "2018-08-31 10:49:00", "month": "08", "high": 112.62, "low": 112.5499, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.605, "open": 112.5499, "day": "31"} +{"volume": 18002, "symbol": "MSFT", "ts": "2018-08-31 10:50:00", "month": "08", "high": 112.6, "low": 112.5832, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.6, "open": 112.59, "day": "31"} +{"volume": 23973, "symbol": "MSFT", "ts": "2018-08-31 10:51:00", "month": "08", "high": 112.6051, "low": 112.5856, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.6051, "open": 112.6, "day": "31"} +{"volume": 34169, "symbol": "MSFT", "ts": "2018-08-31 10:52:00", "month": "08", "high": 112.64, "low": 112.62, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.621, "open": 112.62, "day": "31"} +{"volume": 14876, "symbol": "MSFT", "ts": "2018-08-31 10:53:00", "month": "08", "high": 112.635, "low": 112.614, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.623, "open": 112.635, "day": "31"} +{"volume": 26865, "symbol": "MSFT", "ts": "2018-08-31 10:54:00", "month": "08", "high": 112.7, "low": 112.64, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.7, "open": 112.64, "day": "31"} +{"volume": 45867, "symbol": "MSFT", "ts": "2018-08-31 10:55:00", "month": "08", "high": 112.72, "low": 112.6855, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.69, "open": 112.7, "day": "31"} +{"volume": 26695, "symbol": "MSFT", "ts": "2018-08-31 10:56:00", "month": "08", "high": 112.69, "low": 112.655, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.69, "open": 112.67, "day": "31"} +{"volume": 32264, "symbol": "MSFT", "ts": "2018-08-31 10:57:00", "month": "08", "high": 112.75, "low": 112.68, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.73, "open": 112.69, "day": "31"} +{"volume": 43699, "symbol": "MSFT", "ts": "2018-08-31 10:58:00", "month": "08", "high": 112.73, "low": 112.68, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.7052, "open": 112.71, "day": "31"} +{"volume": 40212, "symbol": "MSFT", "ts": "2018-08-31 10:59:00", "month": "08", "high": 112.715, "low": 112.69, "key": "MSFT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 112.7, "open": 112.69, "day": "31"} +{"volume": 90721, "symbol": "AAPL", "ts": "2018-08-31 10:31:00", "month": "08", "high": 227.62, "low": 227.45, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.48, "open": 227.53, "day": "31"} +{"volume": 178422, "symbol": "AAPL", "ts": "2018-08-31 10:32:00", "month": "08", "high": 227.4732, "low": 227.3101, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.46, "open": 227.4732, "day": "31"} +{"volume": 116569, "symbol": "AAPL", "ts": "2018-08-31 10:33:00", "month": "08", "high": 227.65, "low": 227.47, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.53, "open": 227.47, "day": "31"} +{"volume": 134284, "symbol": "AAPL", "ts": "2018-08-31 10:34:00", "month": "08", "high": 227.5516, "low": 227.29, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.4485, "open": 227.5516, "day": "31"} +{"volume": 71296, "symbol": "AAPL", "ts": "2018-08-31 10:35:00", "month": "08", "high": 227.61, "low": 227.45, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.56, "open": 227.45, "day": "31"} +{"volume": 72464, "symbol": "AAPL", "ts": "2018-08-31 10:36:00", "month": "08", "high": 227.79, "low": 227.56, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.78, "open": 227.56, "day": "31"} +{"volume": 95153, "symbol": "AAPL", "ts": "2018-08-31 10:37:00", "month": "08", "high": 227.96, "low": 227.8, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.88, "open": 227.8, "day": "31"} +{"volume": 68350, "symbol": "AAPL", "ts": "2018-08-31 10:38:00", "month": "08", "high": 227.91, "low": 227.81, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.88, "open": 227.88, "day": "31"} +{"volume": 91381, "symbol": "AAPL", "ts": "2018-08-31 10:39:00", "month": "08", "high": 227.93, "low": 227.77, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.86, "open": 227.899, "day": "31"} +{"volume": 90281, "symbol": "AAPL", "ts": "2018-08-31 10:40:00", "month": "08", "high": 227.93, "low": 227.84, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.8894, "open": 227.869, "day": "31"} +{"volume": 150043, "symbol": "AAPL", "ts": "2018-08-31 10:41:00", "month": "08", "high": 228.2, "low": 227.9, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.145, "open": 227.9, "day": "31"} +{"volume": 85057, "symbol": "AAPL", "ts": "2018-08-31 10:42:00", "month": "08", "high": 228.2299, "low": 228.1, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.2, "open": 228.11, "day": "31"} +{"volume": 128426, "symbol": "AAPL", "ts": "2018-08-31 10:43:00", "month": "08", "high": 228.37, "low": 228.235, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.29, "open": 228.235, "day": "31"} +{"volume": 116251, "symbol": "AAPL", "ts": "2018-08-31 10:44:00", "month": "08", "high": 228.3005, "low": 228.14, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.18, "open": 228.2831, "day": "31"} +{"volume": 87280, "symbol": "AAPL", "ts": "2018-08-31 10:45:00", "month": "08", "high": 228.19, "low": 228.0104, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.02, "open": 228.19, "day": "31"} +{"volume": 122901, "symbol": "AAPL", "ts": "2018-08-31 10:46:00", "month": "08", "high": 228.01, "low": 227.7379, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.75, "open": 228.01, "day": "31"} +{"volume": 64193, "symbol": "AAPL", "ts": "2018-08-31 10:47:00", "month": "08", "high": 227.8, "low": 227.7202, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.7202, "open": 227.73, "day": "31"} +{"volume": 50433, "symbol": "AAPL", "ts": "2018-08-31 10:48:00", "month": "08", "high": 227.76, "low": 227.62, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.71, "open": 227.76, "day": "31"} +{"volume": 62847, "symbol": "AAPL", "ts": "2018-08-31 10:49:00", "month": "08", "high": 227.79, "low": 227.74, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.74, "open": 227.75, "day": "31"} +{"volume": 63634, "symbol": "AAPL", "ts": "2018-08-31 10:50:00", "month": "08", "high": 227.79, "low": 227.69, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.78, "open": 227.73, "day": "31"} +{"volume": 56473, "symbol": "AAPL", "ts": "2018-08-31 10:51:00", "month": "08", "high": 227.8469, "low": 227.74, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.8469, "open": 227.76, "day": "31"} +{"volume": 119771, "symbol": "AAPL", "ts": "2018-08-31 10:52:00", "month": "08", "high": 227.89, "low": 227.69, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.78, "open": 227.8269, "day": "31"} +{"volume": 38287, "symbol": "AAPL", "ts": "2018-08-31 10:53:00", "month": "08", "high": 227.815, "low": 227.77, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.79, "open": 227.77, "day": "31"} +{"volume": 109627, "symbol": "AAPL", "ts": "2018-08-31 10:54:00", "month": "08", "high": 228.21, "low": 227.82, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.21, "open": 227.82, "day": "31"} +{"volume": 107808, "symbol": "AAPL", "ts": "2018-08-31 10:55:00", "month": "08", "high": 228.35, "low": 228.2103, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.24, "open": 228.2103, "day": "31"} +{"volume": 82233, "symbol": "AAPL", "ts": "2018-08-31 10:56:00", "month": "08", "high": 228.28, "low": 228.161, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.27, "open": 228.28, "day": "31"} +{"volume": 53235, "symbol": "AAPL", "ts": "2018-08-31 10:57:00", "month": "08", "high": 228.27, "low": 228.17, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.2586, "open": 228.25, "day": "31"} +{"volume": 77733, "symbol": "AAPL", "ts": "2018-08-31 10:58:00", "month": "08", "high": 228.23, "low": 228.04, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.12, "open": 228.23, "day": "31"} +{"volume": 38710, "symbol": "AAPL", "ts": "2018-08-31 10:59:00", "month": "08", "high": 228.12, "low": 228.0, "key": "AAPL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.04, "open": 228.12, "day": "31"} +{"volume": 2333, "symbol": "GOOG", "ts": "2018-08-31 10:32:00", "month": "08", "high": 1230.3101, "low": 1230.3101, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.3101, "open": 1230.3101, "day": "31"} +{"volume": 1432, "symbol": "GOOG", "ts": "2018-08-31 10:33:00", "month": "08", "high": 1230.115, "low": 1230.115, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1230.115, "open": 1230.115, "day": "31"} +{"volume": 3107, "symbol": "GOOG", "ts": "2018-08-31 10:34:00", "month": "08", "high": 1229.51, "low": 1229.3893, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.3893, "open": 1229.51, "day": "31"} +{"volume": 2337, "symbol": "GOOG", "ts": "2018-08-31 10:35:00", "month": "08", "high": 1229.9126, "low": 1229.9126, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.9126, "open": 1229.9126, "day": "31"} +{"volume": 4520, "symbol": "GOOG", "ts": "2018-08-31 10:36:00", "month": "08", "high": 1229.41, "low": 1229.11, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.41, "open": 1229.11, "day": "31"} +{"volume": 1926, "symbol": "GOOG", "ts": "2018-08-31 10:37:00", "month": "08", "high": 1229.52, "low": 1229.52, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.52, "open": 1229.52, "day": "31"} +{"volume": 884, "symbol": "GOOG", "ts": "2018-08-31 10:38:00", "month": "08", "high": 1229.595, "low": 1229.595, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.595, "open": 1229.595, "day": "31"} +{"volume": 3376, "symbol": "GOOG", "ts": "2018-08-31 10:39:00", "month": "08", "high": 1228.96, "low": 1228.78, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.96, "open": 1228.78, "day": "31"} +{"volume": 3049, "symbol": "GOOG", "ts": "2018-08-31 10:40:00", "month": "08", "high": 1229.6703, "low": 1229.6703, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.6703, "open": 1229.6703, "day": "31"} +{"volume": 2137, "symbol": "GOOG", "ts": "2018-08-31 10:41:00", "month": "08", "high": 1229.6, "low": 1229.6, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.6, "open": 1229.6, "day": "31"} +{"volume": 2938, "symbol": "GOOG", "ts": "2018-08-31 10:42:00", "month": "08", "high": 1229.6, "low": 1228.65, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.65, "open": 1229.6, "day": "31"} +{"volume": 2766, "symbol": "GOOG", "ts": "2018-08-31 10:43:00", "month": "08", "high": 1228.4973, "low": 1228.4973, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.4973, "open": 1228.4973, "day": "31"} +{"volume": 3444, "symbol": "GOOG", "ts": "2018-08-31 10:44:00", "month": "08", "high": 1228.47, "low": 1227.88, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1227.88, "open": 1228.47, "day": "31"} +{"volume": 2880, "symbol": "GOOG", "ts": "2018-08-31 10:45:00", "month": "08", "high": 1227.88, "low": 1227.88, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1227.88, "open": 1227.88, "day": "31"} +{"volume": 1175, "symbol": "GOOG", "ts": "2018-08-31 10:46:00", "month": "08", "high": 1228.28, "low": 1228.28, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.28, "open": 1228.28, "day": "31"} +{"volume": 850, "symbol": "GOOG", "ts": "2018-08-31 10:47:00", "month": "08", "high": 1228.29, "low": 1228.29, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.29, "open": 1228.29, "day": "31"} +{"volume": 4279, "symbol": "GOOG", "ts": "2018-08-31 10:48:00", "month": "08", "high": 1228.11, "low": 1227.61, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.11, "open": 1227.61, "day": "31"} +{"volume": 1162, "symbol": "GOOG", "ts": "2018-08-31 10:49:00", "month": "08", "high": 1228.78, "low": 1228.78, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.78, "open": 1228.78, "day": "31"} +{"volume": 1306, "symbol": "GOOG", "ts": "2018-08-31 10:50:00", "month": "08", "high": 1228.7828, "low": 1228.7828, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.7828, "open": 1228.7828, "day": "31"} +{"volume": 2789, "symbol": "GOOG", "ts": "2018-08-31 10:51:00", "month": "08", "high": 1228.0699, "low": 1228.0699, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.0699, "open": 1228.0699, "day": "31"} +{"volume": 943, "symbol": "GOOG", "ts": "2018-08-31 10:52:00", "month": "08", "high": 1227.97, "low": 1227.97, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1227.97, "open": 1227.97, "day": "31"} +{"volume": 928, "symbol": "GOOG", "ts": "2018-08-31 10:53:00", "month": "08", "high": 1227.7404, "low": 1227.7404, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1227.7404, "open": 1227.7404, "day": "31"} +{"volume": 3896, "symbol": "GOOG", "ts": "2018-08-31 10:54:00", "month": "08", "high": 1229.0, "low": 1228.58, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1229.0, "open": 1228.58, "day": "31"} +{"volume": 3748, "symbol": "GOOG", "ts": "2018-08-31 10:56:00", "month": "08", "high": 1228.45, "low": 1228.45, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.45, "open": 1228.45, "day": "31"} +{"volume": 1796, "symbol": "GOOG", "ts": "2018-08-31 10:57:00", "month": "08", "high": 1228.0, "low": 1228.0, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1228.0, "open": 1228.0, "day": "31"} +{"volume": 7109, "symbol": "GOOG", "ts": "2018-08-31 10:58:00", "month": "08", "high": 1227.65, "low": 1226.98, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1226.98, "open": 1227.65, "day": "31"} +{"volume": 9021, "symbol": "GOOG", "ts": "2018-08-31 10:59:00", "month": "08", "high": 1227.34, "low": 1227.1993, "key": "GOOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1227.215, "open": 1227.1993, "day": "31"} +{"volume": 96863, "symbol": "FB", "ts": "2018-08-31 10:31:00", "month": "08", "high": 176.57, "low": 176.4, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.54, "open": 176.4224, "day": "31"} +{"volume": 66358, "symbol": "FB", "ts": "2018-08-31 10:32:00", "month": "08", "high": 176.51, "low": 176.25, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.25, "open": 176.51, "day": "31"} +{"volume": 53305, "symbol": "FB", "ts": "2018-08-31 10:33:00", "month": "08", "high": 176.26, "low": 176.1002, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.16, "open": 176.26, "day": "31"} +{"volume": 55364, "symbol": "FB", "ts": "2018-08-31 10:34:00", "month": "08", "high": 176.09, "low": 176.0, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.09, "open": 176.09, "day": "31"} +{"volume": 45463, "symbol": "FB", "ts": "2018-08-31 10:35:00", "month": "08", "high": 176.11, "low": 175.93, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.96, "open": 176.11, "day": "31"} +{"volume": 48978, "symbol": "FB", "ts": "2018-08-31 10:36:00", "month": "08", "high": 176.04, "low": 175.96, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0, "open": 175.9601, "day": "31"} +{"volume": 37586, "symbol": "FB", "ts": "2018-08-31 10:37:00", "month": "08", "high": 176.08, "low": 175.93, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.08, "open": 175.93, "day": "31"} +{"volume": 44297, "symbol": "FB", "ts": "2018-08-31 10:38:00", "month": "08", "high": 176.1, "low": 176.0, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0, "open": 176.085, "day": "31"} +{"volume": 21702, "symbol": "FB", "ts": "2018-08-31 10:39:00", "month": "08", "high": 176.06, "low": 176.0, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.06, "open": 176.025, "day": "31"} +{"volume": 43814, "symbol": "FB", "ts": "2018-08-31 10:40:00", "month": "08", "high": 176.21, "low": 176.0799, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.1391, "open": 176.0799, "day": "31"} +{"volume": 42931, "symbol": "FB", "ts": "2018-08-31 10:41:00", "month": "08", "high": 176.28, "low": 176.14, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.28, "open": 176.14, "day": "31"} +{"volume": 54422, "symbol": "FB", "ts": "2018-08-31 10:42:00", "month": "08", "high": 176.2299, "low": 175.94, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.94, "open": 176.2299, "day": "31"} +{"volume": 41483, "symbol": "FB", "ts": "2018-08-31 10:43:00", "month": "08", "high": 176.12, "low": 175.94, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.1, "open": 175.94, "day": "31"} +{"volume": 28170, "symbol": "FB", "ts": "2018-08-31 10:44:00", "month": "08", "high": 176.11, "low": 175.96, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.96, "open": 176.11, "day": "31"} +{"volume": 33343, "symbol": "FB", "ts": "2018-08-31 10:45:00", "month": "08", "high": 176.0, "low": 175.95, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0, "open": 175.96, "day": "31"} +{"volume": 23999, "symbol": "FB", "ts": "2018-08-31 10:46:00", "month": "08", "high": 176.13, "low": 176.0174, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0174, "open": 176.04, "day": "31"} +{"volume": 38147, "symbol": "FB", "ts": "2018-08-31 10:47:00", "month": "08", "high": 175.98, "low": 175.87, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.87, "open": 175.98, "day": "31"} +{"volume": 41551, "symbol": "FB", "ts": "2018-08-31 10:48:00", "month": "08", "high": 175.93, "low": 175.82, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.9266, "open": 175.85, "day": "31"} +{"volume": 36036, "symbol": "FB", "ts": "2018-08-31 10:49:00", "month": "08", "high": 176.1285, "low": 175.975, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.025, "open": 175.975, "day": "31"} +{"volume": 26106, "symbol": "FB", "ts": "2018-08-31 10:50:00", "month": "08", "high": 176.04, "low": 175.9856, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.039, "open": 176.0, "day": "31"} +{"volume": 24152, "symbol": "FB", "ts": "2018-08-31 10:51:00", "month": "08", "high": 176.02, "low": 175.97, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.97, "open": 176.01, "day": "31"} +{"volume": 24621, "symbol": "FB", "ts": "2018-08-31 10:52:00", "month": "08", "high": 176.03, "low": 175.92, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.92, "open": 175.98, "day": "31"} +{"volume": 21011, "symbol": "FB", "ts": "2018-08-31 10:53:00", "month": "08", "high": 175.9501, "low": 175.855, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.94, "open": 175.9049, "day": "31"} +{"volume": 28578, "symbol": "FB", "ts": "2018-08-31 10:54:00", "month": "08", "high": 176.02, "low": 175.909, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.02, "open": 175.916, "day": "31"} +{"volume": 34036, "symbol": "FB", "ts": "2018-08-31 10:55:00", "month": "08", "high": 176.05, "low": 175.9, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.9, "open": 176.0397, "day": "31"} +{"volume": 55472, "symbol": "FB", "ts": "2018-08-31 10:56:00", "month": "08", "high": 176.0139, "low": 175.85, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0139, "open": 175.905, "day": "31"} +{"volume": 26265, "symbol": "FB", "ts": "2018-08-31 10:57:00", "month": "08", "high": 176.08, "low": 175.98, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0217, "open": 175.98, "day": "31"} +{"volume": 42822, "symbol": "FB", "ts": "2018-08-31 10:58:00", "month": "08", "high": 176.0411, "low": 175.95, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 176.0411, "open": 175.95, "day": "31"} +{"volume": 58174, "symbol": "FB", "ts": "2018-08-31 10:59:00", "month": "08", "high": 175.99, "low": 175.911, "key": "FB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 175.97, "open": 175.99, "day": "31"} +{"volume": 11241, "symbol": "NFLX", "ts": "2018-08-31 10:31:00", "month": "08", "high": 374.13, "low": 373.92, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.13, "open": 373.92, "day": "31"} +{"volume": 21050, "symbol": "NFLX", "ts": "2018-08-31 10:32:00", "month": "08", "high": 374.01, "low": 373.6, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.62, "open": 374.01, "day": "31"} +{"volume": 12048, "symbol": "NFLX", "ts": "2018-08-31 10:33:00", "month": "08", "high": 373.9, "low": 373.78, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.83, "open": 373.78, "day": "31"} +{"volume": 13070, "symbol": "NFLX", "ts": "2018-08-31 10:34:00", "month": "08", "high": 373.71, "low": 373.63, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.71, "open": 373.63, "day": "31"} +{"volume": 20072, "symbol": "NFLX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 373.7, "low": 373.65, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.65, "open": 373.7, "day": "31"} +{"volume": 22403, "symbol": "NFLX", "ts": "2018-08-31 10:36:00", "month": "08", "high": 373.58, "low": 373.4, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.4, "open": 373.56, "day": "31"} +{"volume": 26307, "symbol": "NFLX", "ts": "2018-08-31 10:37:00", "month": "08", "high": 373.86, "low": 373.3575, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.86, "open": 373.3575, "day": "31"} +{"volume": 10084, "symbol": "NFLX", "ts": "2018-08-31 10:38:00", "month": "08", "high": 373.915, "low": 373.82, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.82, "open": 373.915, "day": "31"} +{"volume": 8692, "symbol": "NFLX", "ts": "2018-08-31 10:39:00", "month": "08", "high": 373.755, "low": 373.53, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.5521, "open": 373.755, "day": "31"} +{"volume": 15388, "symbol": "NFLX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 373.88, "low": 373.7, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.85, "open": 373.7, "day": "31"} +{"volume": 13611, "symbol": "NFLX", "ts": "2018-08-31 10:41:00", "month": "08", "high": 374.04, "low": 373.79, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.04, "open": 373.79, "day": "31"} +{"volume": 17813, "symbol": "NFLX", "ts": "2018-08-31 10:42:00", "month": "08", "high": 373.84, "low": 373.45, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.699, "open": 373.84, "day": "31"} +{"volume": 6314, "symbol": "NFLX", "ts": "2018-08-31 10:43:00", "month": "08", "high": 373.65, "low": 373.64, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.65, "open": 373.64, "day": "31"} +{"volume": 3023, "symbol": "NFLX", "ts": "2018-08-31 10:44:00", "month": "08", "high": 373.66, "low": 373.66, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.66, "open": 373.66, "day": "31"} +{"volume": 7408, "symbol": "NFLX", "ts": "2018-08-31 10:45:00", "month": "08", "high": 373.7, "low": 373.69, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 373.7, "open": 373.69, "day": "31"} +{"volume": 55690, "symbol": "NFLX", "ts": "2018-08-31 10:46:00", "month": "08", "high": 374.23, "low": 373.8992, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.0664, "open": 373.8992, "day": "31"} +{"volume": 5948, "symbol": "NFLX", "ts": "2018-08-31 10:47:00", "month": "08", "high": 374.14, "low": 374.0152, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.0152, "open": 374.14, "day": "31"} +{"volume": 29335, "symbol": "NFLX", "ts": "2018-08-31 10:48:00", "month": "08", "high": 374.13, "low": 373.8255, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.13, "open": 374.04, "day": "31"} +{"volume": 23033, "symbol": "NFLX", "ts": "2018-08-31 10:49:00", "month": "08", "high": 374.49, "low": 374.0432, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.49, "open": 374.12, "day": "31"} +{"volume": 21979, "symbol": "NFLX", "ts": "2018-08-31 10:50:00", "month": "08", "high": 374.65, "low": 374.42, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.6099, "open": 374.54, "day": "31"} +{"volume": 13548, "symbol": "NFLX", "ts": "2018-08-31 10:51:00", "month": "08", "high": 374.57, "low": 374.5, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.57, "open": 374.5, "day": "31"} +{"volume": 36528, "symbol": "NFLX", "ts": "2018-08-31 10:52:00", "month": "08", "high": 375.19, "low": 374.41, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.9116, "open": 374.6, "day": "31"} +{"volume": 11385, "symbol": "NFLX", "ts": "2018-08-31 10:53:00", "month": "08", "high": 374.86, "low": 374.82, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.86, "open": 374.82, "day": "31"} +{"volume": 11996, "symbol": "NFLX", "ts": "2018-08-31 10:54:00", "month": "08", "high": 375.1297, "low": 374.89, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 375.1297, "open": 374.89, "day": "31"} +{"volume": 18209, "symbol": "NFLX", "ts": "2018-08-31 10:55:00", "month": "08", "high": 375.1771, "low": 374.96, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.96, "open": 375.1592, "day": "31"} +{"volume": 21202, "symbol": "NFLX", "ts": "2018-08-31 10:56:00", "month": "08", "high": 375.0483, "low": 374.83, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 375.02, "open": 374.84, "day": "31"} +{"volume": 10954, "symbol": "NFLX", "ts": "2018-08-31 10:57:00", "month": "08", "high": 375.0, "low": 374.691, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.91, "open": 375.0, "day": "31"} +{"volume": 15919, "symbol": "NFLX", "ts": "2018-08-31 10:58:00", "month": "08", "high": 374.904, "low": 374.73, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.904, "open": 374.8223, "day": "31"} +{"volume": 6362, "symbol": "NFLX", "ts": "2018-08-31 10:59:00", "month": "08", "high": 374.99, "low": 374.93, "key": "NFLX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 374.99, "open": 374.93, "day": "31"} +{"volume": 10875, "symbol": "TSLA", "ts": "2018-08-31 10:31:00", "month": "08", "high": 300.86, "low": 300.75, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.81, "open": 300.75, "day": "31"} +{"volume": 18069, "symbol": "TSLA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 300.55, "low": 300.27, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.32, "open": 300.55, "day": "31"} +{"volume": 5263, "symbol": "TSLA", "ts": "2018-08-31 10:33:00", "month": "08", "high": 300.6057, "low": 300.56, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.6057, "open": 300.56, "day": "31"} +{"volume": 13351, "symbol": "TSLA", "ts": "2018-08-31 10:34:00", "month": "08", "high": 300.99, "low": 300.89, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.89, "open": 300.9, "day": "31"} +{"volume": 8522, "symbol": "TSLA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 301.08, "low": 300.675, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.675, "open": 301.08, "day": "31"} +{"volume": 16171, "symbol": "TSLA", "ts": "2018-08-31 10:36:00", "month": "08", "high": 300.8, "low": 300.56, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.75, "open": 300.56, "day": "31"} +{"volume": 30432, "symbol": "TSLA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 300.82, "low": 300.43, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.51, "open": 300.82, "day": "31"} +{"volume": 14135, "symbol": "TSLA", "ts": "2018-08-31 10:38:00", "month": "08", "high": 300.6, "low": 300.5, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.52, "open": 300.5126, "day": "31"} +{"volume": 13047, "symbol": "TSLA", "ts": "2018-08-31 10:39:00", "month": "08", "high": 300.4883, "low": 300.42, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.44, "open": 300.44, "day": "31"} +{"volume": 68130, "symbol": "TSLA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 300.62, "low": 299.89, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.62, "open": 300.28, "day": "31"} +{"volume": 14664, "symbol": "TSLA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 300.4, "low": 300.2325, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.399, "open": 300.4, "day": "31"} +{"volume": 41991, "symbol": "TSLA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 300.44, "low": 299.9, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.0, "open": 300.12, "day": "31"} +{"volume": 7700, "symbol": "TSLA", "ts": "2018-08-31 10:43:00", "month": "08", "high": 300.0849, "low": 299.86, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.86, "open": 300.0849, "day": "31"} +{"volume": 31260, "symbol": "TSLA", "ts": "2018-08-31 10:44:00", "month": "08", "high": 299.86, "low": 299.51, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.6101, "open": 299.86, "day": "31"} +{"volume": 10325, "symbol": "TSLA", "ts": "2018-08-31 10:45:00", "month": "08", "high": 299.76, "low": 299.6033, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.6033, "open": 299.71, "day": "31"} +{"volume": 18424, "symbol": "TSLA", "ts": "2018-08-31 10:46:00", "month": "08", "high": 299.86, "low": 299.695, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.775, "open": 299.695, "day": "31"} +{"volume": 13044, "symbol": "TSLA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 299.9899, "low": 299.9, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.9899, "open": 299.9, "day": "31"} +{"volume": 17112, "symbol": "TSLA", "ts": "2018-08-31 10:48:00", "month": "08", "high": 300.05, "low": 299.61, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.66, "open": 300.05, "day": "31"} +{"volume": 23649, "symbol": "TSLA", "ts": "2018-08-31 10:49:00", "month": "08", "high": 299.75, "low": 299.36, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.75, "open": 299.5, "day": "31"} +{"volume": 5287, "symbol": "TSLA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 299.73, "low": 299.7, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.73, "open": 299.7, "day": "31"} +{"volume": 3798, "symbol": "TSLA", "ts": "2018-08-31 10:51:00", "month": "08", "high": 299.73, "low": 299.73, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.73, "open": 299.73, "day": "31"} +{"volume": 8172, "symbol": "TSLA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 299.8489, "low": 299.68, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 299.8489, "open": 299.68, "day": "31"} +{"volume": 14957, "symbol": "TSLA", "ts": "2018-08-31 10:53:00", "month": "08", "high": 300.15, "low": 300.0, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.15, "open": 300.0, "day": "31"} +{"volume": 21749, "symbol": "TSLA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 300.46, "low": 300.06, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.435, "open": 300.14, "day": "31"} +{"volume": 12970, "symbol": "TSLA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 300.4, "low": 300.14, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.4, "open": 300.3, "day": "31"} +{"volume": 6972, "symbol": "TSLA", "ts": "2018-08-31 10:56:00", "month": "08", "high": 300.21, "low": 300.15, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.21, "open": 300.15, "day": "31"} +{"volume": 7396, "symbol": "TSLA", "ts": "2018-08-31 10:57:00", "month": "08", "high": 300.3, "low": 300.19, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.2, "open": 300.19, "day": "31"} +{"volume": 5592, "symbol": "TSLA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 300.345, "low": 300.345, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.345, "open": 300.345, "day": "31"} +{"volume": 11145, "symbol": "TSLA", "ts": "2018-08-31 10:59:00", "month": "08", "high": 300.4273, "low": 300.29, "key": "TSLA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 300.4, "open": 300.29, "day": "31"} +{"volume": 87261, "symbol": "F", "ts": "2018-08-31 10:31:00", "month": "08", "high": 9.63, "low": 9.625, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.625, "open": 9.63, "day": "31"} +{"volume": 73465, "symbol": "F", "ts": "2018-08-31 10:32:00", "month": "08", "high": 9.63, "low": 9.6268, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6268, "open": 9.63, "day": "31"} +{"volume": 71918, "symbol": "F", "ts": "2018-08-31 10:33:00", "month": "08", "high": 9.6309, "low": 9.63, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6309, "open": 9.63, "day": "31"} +{"volume": 127343, "symbol": "F", "ts": "2018-08-31 10:34:00", "month": "08", "high": 9.635, "low": 9.63, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.63, "open": 9.635, "day": "31"} +{"volume": 26147, "symbol": "F", "ts": "2018-08-31 10:35:00", "month": "08", "high": 9.625, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.625, "open": 9.62, "day": "31"} +{"volume": 35335, "symbol": "F", "ts": "2018-08-31 10:36:00", "month": "08", "high": 9.625, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.625, "open": 9.62, "day": "31"} +{"volume": 535655, "symbol": "F", "ts": "2018-08-31 10:37:00", "month": "08", "high": 9.6257, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6226, "open": 9.625, "day": "31"} +{"volume": 788816, "symbol": "F", "ts": "2018-08-31 10:38:00", "month": "08", "high": 9.64, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.62, "open": 9.63, "day": "31"} +{"volume": 497510, "symbol": "F", "ts": "2018-08-31 10:39:00", "month": "08", "high": 9.63, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.62, "open": 9.6219, "day": "31"} +{"volume": 114450, "symbol": "F", "ts": "2018-08-31 10:40:00", "month": "08", "high": 9.6262, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6257, "open": 9.62, "day": "31"} +{"volume": 301438, "symbol": "F", "ts": "2018-08-31 10:41:00", "month": "08", "high": 9.6229, "low": 9.62, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.62, "open": 9.6229, "day": "31"} +{"volume": 812290, "symbol": "F", "ts": "2018-08-31 10:42:00", "month": "08", "high": 9.6121, "low": 9.59, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.59, "open": 9.61, "day": "31"} +{"volume": 125934, "symbol": "F", "ts": "2018-08-31 10:43:00", "month": "08", "high": 9.6085, "low": 9.595, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6085, "open": 9.595, "day": "31"} +{"volume": 13591, "symbol": "F", "ts": "2018-08-31 10:44:00", "month": "08", "high": 9.605, "low": 9.605, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.605, "open": 9.605, "day": "31"} +{"volume": 73926, "symbol": "F", "ts": "2018-08-31 10:45:00", "month": "08", "high": 9.61, "low": 9.6, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.61, "day": "31"} +{"volume": 87410, "symbol": "F", "ts": "2018-08-31 10:46:00", "month": "08", "high": 9.61, "low": 9.6, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.61, "day": "31"} +{"volume": 1409915, "symbol": "F", "ts": "2018-08-31 10:47:00", "month": "08", "high": 9.595, "low": 9.57, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.59, "open": 9.59, "day": "31"} +{"volume": 38672, "symbol": "F", "ts": "2018-08-31 10:48:00", "month": "08", "high": 9.6, "low": 9.593, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.593, "day": "31"} +{"volume": 64756, "symbol": "F", "ts": "2018-08-31 10:49:00", "month": "08", "high": 9.597, "low": 9.5933, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.597, "open": 9.5933, "day": "31"} +{"volume": 235322, "symbol": "F", "ts": "2018-08-31 10:50:00", "month": "08", "high": 9.594, "low": 9.58, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.58, "open": 9.594, "day": "31"} +{"volume": 1031235, "symbol": "F", "ts": "2018-08-31 10:51:00", "month": "08", "high": 9.59, "low": 9.565, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.59, "open": 9.59, "day": "31"} +{"volume": 56352, "symbol": "F", "ts": "2018-08-31 10:52:00", "month": "08", "high": 9.6, "low": 9.59, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.59, "open": 9.6, "day": "31"} +{"volume": 61175, "symbol": "F", "ts": "2018-08-31 10:53:00", "month": "08", "high": 9.6, "low": 9.59, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.6, "open": 9.591, "day": "31"} +{"volume": 64689, "symbol": "F", "ts": "2018-08-31 10:54:00", "month": "08", "high": 9.6, "low": 9.595, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.5956, "open": 9.6, "day": "31"} +{"volume": 215872, "symbol": "F", "ts": "2018-08-31 10:55:00", "month": "08", "high": 9.595, "low": 9.585, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.585, "open": 9.595, "day": "31"} +{"volume": 65343, "symbol": "F", "ts": "2018-08-31 10:56:00", "month": "08", "high": 9.5853, "low": 9.5843, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.5853, "open": 9.5843, "day": "31"} +{"volume": 32913, "symbol": "F", "ts": "2018-08-31 10:57:00", "month": "08", "high": 9.585, "low": 9.585, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.585, "open": 9.585, "day": "31"} +{"volume": 37055, "symbol": "F", "ts": "2018-08-31 10:58:00", "month": "08", "high": 9.5841, "low": 9.5831, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.5831, "open": 9.5833, "day": "31"} +{"volume": 24700, "symbol": "F", "ts": "2018-08-31 10:59:00", "month": "08", "high": 9.585, "low": 9.58, "key": "F_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.585, "open": 9.585, "day": "31"} +{"volume": 7572, "symbol": "AMZN", "ts": "2018-08-31 10:31:00", "month": "08", "high": 2017.3094, "low": 2017.0, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.0, "open": 2017.3094, "day": "31"} +{"volume": 8314, "symbol": "AMZN", "ts": "2018-08-31 10:32:00", "month": "08", "high": 2016.7627, "low": 2016.47, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.47, "open": 2016.7627, "day": "31"} +{"volume": 9396, "symbol": "AMZN", "ts": "2018-08-31 10:33:00", "month": "08", "high": 2017.4301, "low": 2016.8, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.8, "open": 2017.429, "day": "31"} +{"volume": 6725, "symbol": "AMZN", "ts": "2018-08-31 10:34:00", "month": "08", "high": 2017.01, "low": 2016.16, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.3835, "open": 2017.01, "day": "31"} +{"volume": 7856, "symbol": "AMZN", "ts": "2018-08-31 10:35:00", "month": "08", "high": 2017.2446, "low": 2016.96, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.01, "open": 2016.96, "day": "31"} +{"volume": 8410, "symbol": "AMZN", "ts": "2018-08-31 10:36:00", "month": "08", "high": 2018.27, "low": 2017.59, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.27, "open": 2017.59, "day": "31"} +{"volume": 7004, "symbol": "AMZN", "ts": "2018-08-31 10:37:00", "month": "08", "high": 2018.96, "low": 2018.89, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.96, "open": 2018.89, "day": "31"} +{"volume": 8236, "symbol": "AMZN", "ts": "2018-08-31 10:38:00", "month": "08", "high": 2019.0503, "low": 2018.47, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.47, "open": 2018.76, "day": "31"} +{"volume": 4986, "symbol": "AMZN", "ts": "2018-08-31 10:39:00", "month": "08", "high": 2019.3553, "low": 2019.0, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.3553, "open": 2019.0, "day": "31"} +{"volume": 13023, "symbol": "AMZN", "ts": "2018-08-31 10:40:00", "month": "08", "high": 2020.38, "low": 2019.766, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2020.38, "open": 2019.766, "day": "31"} +{"volume": 14731, "symbol": "AMZN", "ts": "2018-08-31 10:41:00", "month": "08", "high": 2021.33, "low": 2020.5217, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2021.21, "open": 2020.595, "day": "31"} +{"volume": 15884, "symbol": "AMZN", "ts": "2018-08-31 10:42:00", "month": "08", "high": 2020.46, "low": 2019.39, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.67, "open": 2020.46, "day": "31"} +{"volume": 5791, "symbol": "AMZN", "ts": "2018-08-31 10:43:00", "month": "08", "high": 2019.76, "low": 2019.4324, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.76, "open": 2019.4324, "day": "31"} +{"volume": 5457, "symbol": "AMZN", "ts": "2018-08-31 10:44:00", "month": "08", "high": 2019.86, "low": 2019.3, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.3, "open": 2019.86, "day": "31"} +{"volume": 6065, "symbol": "AMZN", "ts": "2018-08-31 10:45:00", "month": "08", "high": 2019.58, "low": 2018.99, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.58, "open": 2018.99, "day": "31"} +{"volume": 2632, "symbol": "AMZN", "ts": "2018-08-31 10:46:00", "month": "08", "high": 2019.59, "low": 2019.59, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.59, "open": 2019.59, "day": "31"} +{"volume": 4796, "symbol": "AMZN", "ts": "2018-08-31 10:47:00", "month": "08", "high": 2020.33, "low": 2020.0653, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2020.0653, "open": 2020.33, "day": "31"} +{"volume": 7175, "symbol": "AMZN", "ts": "2018-08-31 10:48:00", "month": "08", "high": 2020.0822, "low": 2018.5558, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.5558, "open": 2020.0822, "day": "31"} +{"volume": 9635, "symbol": "AMZN", "ts": "2018-08-31 10:49:00", "month": "08", "high": 2019.2834, "low": 2019.2354, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2019.25, "open": 2019.28, "day": "31"} +{"volume": 10195, "symbol": "AMZN", "ts": "2018-08-31 10:50:00", "month": "08", "high": 2018.38, "low": 2017.681, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.681, "open": 2018.38, "day": "31"} +{"volume": 13022, "symbol": "AMZN", "ts": "2018-08-31 10:51:00", "month": "08", "high": 2017.275, "low": 2016.9399, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.9399, "open": 2017.1, "day": "31"} +{"volume": 10402, "symbol": "AMZN", "ts": "2018-08-31 10:52:00", "month": "08", "high": 2017.155, "low": 2016.29, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.155, "open": 2016.29, "day": "31"} +{"volume": 7419, "symbol": "AMZN", "ts": "2018-08-31 10:53:00", "month": "08", "high": 2017.5192, "low": 2016.26, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2016.26, "open": 2017.5192, "day": "31"} +{"volume": 4009, "symbol": "AMZN", "ts": "2018-08-31 10:54:00", "month": "08", "high": 2017.24, "low": 2016.9301, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.24, "open": 2016.9301, "day": "31"} +{"volume": 5348, "symbol": "AMZN", "ts": "2018-08-31 10:55:00", "month": "08", "high": 2018.9, "low": 2018.1113, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.9, "open": 2018.1113, "day": "31"} +{"volume": 6122, "symbol": "AMZN", "ts": "2018-08-31 10:56:00", "month": "08", "high": 2018.15, "low": 2018.09, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.09, "open": 2018.15, "day": "31"} +{"volume": 4655, "symbol": "AMZN", "ts": "2018-08-31 10:57:00", "month": "08", "high": 2018.67, "low": 2018.5601, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2018.5601, "open": 2018.67, "day": "31"} +{"volume": 6832, "symbol": "AMZN", "ts": "2018-08-31 10:58:00", "month": "08", "high": 2018.3199, "low": 2017.755, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.755, "open": 2018.3199, "day": "31"} +{"volume": 1018, "symbol": "AMZN", "ts": "2018-08-31 10:59:00", "month": "08", "high": 2017.63, "low": 2017.63, "key": "AMZN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 2017.63, "open": 2017.63, "day": "31"} +{"volume": 11153, "symbol": "NVDA", "ts": "2018-08-31 10:31:00", "month": "08", "high": 280.0, "low": 279.95, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.0, "open": 279.95, "day": "31"} +{"volume": 11703, "symbol": "NVDA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 279.93, "low": 279.8768, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.9275, "open": 279.8768, "day": "31"} +{"volume": 22260, "symbol": "NVDA", "ts": "2018-08-31 10:33:00", "month": "08", "high": 280.05, "low": 280.006, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.006, "open": 280.04, "day": "31"} +{"volume": 11295, "symbol": "NVDA", "ts": "2018-08-31 10:34:00", "month": "08", "high": 279.95, "low": 279.691, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.84, "open": 279.95, "day": "31"} +{"volume": 7102, "symbol": "NVDA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 279.88, "low": 279.78, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.78, "open": 279.88, "day": "31"} +{"volume": 6405, "symbol": "NVDA", "ts": "2018-08-31 10:36:00", "month": "08", "high": 279.96, "low": 279.87, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 279.96, "open": 279.87, "day": "31"} +{"volume": 16325, "symbol": "NVDA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 280.1, "low": 279.95, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.1, "open": 279.95, "day": "31"} +{"volume": 2577, "symbol": "NVDA", "ts": "2018-08-31 10:38:00", "month": "08", "high": 280.11, "low": 280.11, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.11, "open": 280.11, "day": "31"} +{"volume": 8331, "symbol": "NVDA", "ts": "2018-08-31 10:39:00", "month": "08", "high": 280.05, "low": 280.034, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.05, "open": 280.034, "day": "31"} +{"volume": 10446, "symbol": "NVDA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 280.15, "low": 280.14, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.149, "open": 280.14, "day": "31"} +{"volume": 34047, "symbol": "NVDA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 280.332, "low": 280.19, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.332, "open": 280.19, "day": "31"} +{"volume": 15859, "symbol": "NVDA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 280.329, "low": 280.14, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.14, "open": 280.329, "day": "31"} +{"volume": 19459, "symbol": "NVDA", "ts": "2018-08-31 10:43:00", "month": "08", "high": 280.33, "low": 280.2092, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.33, "open": 280.2092, "day": "31"} +{"volume": 27953, "symbol": "NVDA", "ts": "2018-08-31 10:44:00", "month": "08", "high": 280.3371, "low": 280.2, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.3371, "open": 280.33, "day": "31"} +{"volume": 21675, "symbol": "NVDA", "ts": "2018-08-31 10:45:00", "month": "08", "high": 280.4, "low": 280.3, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.355, "open": 280.4, "day": "31"} +{"volume": 17297, "symbol": "NVDA", "ts": "2018-08-31 10:46:00", "month": "08", "high": 280.4, "low": 280.37, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.3768, "open": 280.4, "day": "31"} +{"volume": 45411, "symbol": "NVDA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 280.51, "low": 280.4, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.51, "open": 280.4, "day": "31"} +{"volume": 21891, "symbol": "NVDA", "ts": "2018-08-31 10:48:00", "month": "08", "high": 280.575, "low": 280.44, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.49, "open": 280.575, "day": "31"} +{"volume": 12360, "symbol": "NVDA", "ts": "2018-08-31 10:49:00", "month": "08", "high": 280.495, "low": 280.465, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.495, "open": 280.465, "day": "31"} +{"volume": 16968, "symbol": "NVDA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 280.59, "low": 280.51, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.55, "open": 280.51, "day": "31"} +{"volume": 8447, "symbol": "NVDA", "ts": "2018-08-31 10:51:00", "month": "08", "high": 280.51, "low": 280.5098, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.51, "open": 280.5098, "day": "31"} +{"volume": 15095, "symbol": "NVDA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 280.525, "low": 280.45, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.45, "open": 280.525, "day": "31"} +{"volume": 10052, "symbol": "NVDA", "ts": "2018-08-31 10:53:00", "month": "08", "high": 280.54, "low": 280.47, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.47, "open": 280.5, "day": "31"} +{"volume": 53055, "symbol": "NVDA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 280.93, "low": 280.6, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.93, "open": 280.6, "day": "31"} +{"volume": 36392, "symbol": "NVDA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 280.95, "low": 280.73, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.73, "open": 280.93, "day": "31"} +{"volume": 21777, "symbol": "NVDA", "ts": "2018-08-31 10:56:00", "month": "08", "high": 281.0, "low": 280.5502, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.9, "open": 280.5502, "day": "31"} +{"volume": 6877, "symbol": "NVDA", "ts": "2018-08-31 10:57:00", "month": "08", "high": 280.88, "low": 280.7754, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.7754, "open": 280.88, "day": "31"} +{"volume": 12248, "symbol": "NVDA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 280.81, "low": 280.6338, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.72, "open": 280.81, "day": "31"} +{"volume": 4862, "symbol": "NVDA", "ts": "2018-08-31 10:59:00", "month": "08", "high": 280.75, "low": 280.71, "key": "NVDA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 280.71, "open": 280.75, "day": "31"} +{"volume": 31472, "symbol": "INTC", "ts": "2018-08-31 10:31:00", "month": "08", "high": 48.36, "low": 48.33, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.33, "open": 48.35, "day": "31"} +{"volume": 22865, "symbol": "INTC", "ts": "2018-08-31 10:32:00", "month": "08", "high": 48.35, "low": 48.325, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.35, "open": 48.325, "day": "31"} +{"volume": 43879, "symbol": "INTC", "ts": "2018-08-31 10:33:00", "month": "08", "high": 48.37, "low": 48.34, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.34, "open": 48.36, "day": "31"} +{"volume": 56207, "symbol": "INTC", "ts": "2018-08-31 10:34:00", "month": "08", "high": 48.34, "low": 48.32, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.32, "open": 48.32, "day": "31"} +{"volume": 19785, "symbol": "INTC", "ts": "2018-08-31 10:35:00", "month": "08", "high": 48.325, "low": 48.304, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.32, "open": 48.304, "day": "31"} +{"volume": 13980, "symbol": "INTC", "ts": "2018-08-31 10:36:00", "month": "08", "high": 48.3458, "low": 48.31, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.3458, "open": 48.31, "day": "31"} +{"volume": 19220, "symbol": "INTC", "ts": "2018-08-31 10:37:00", "month": "08", "high": 48.35, "low": 48.34, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.34, "open": 48.34, "day": "31"} +{"volume": 31972, "symbol": "INTC", "ts": "2018-08-31 10:38:00", "month": "08", "high": 48.37, "low": 48.346, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.35, "open": 48.346, "day": "31"} +{"volume": 23892, "symbol": "INTC", "ts": "2018-08-31 10:39:00", "month": "08", "high": 48.355, "low": 48.335, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.355, "open": 48.35, "day": "31"} +{"volume": 12795, "symbol": "INTC", "ts": "2018-08-31 10:40:00", "month": "08", "high": 48.37, "low": 48.365, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.37, "open": 48.365, "day": "31"} +{"volume": 30214, "symbol": "INTC", "ts": "2018-08-31 10:41:00", "month": "08", "high": 48.41, "low": 48.38, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.41, "open": 48.38, "day": "31"} +{"volume": 38504, "symbol": "INTC", "ts": "2018-08-31 10:42:00", "month": "08", "high": 48.4, "low": 48.3522, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.3522, "open": 48.4, "day": "31"} +{"volume": 11113, "symbol": "INTC", "ts": "2018-08-31 10:43:00", "month": "08", "high": 48.38, "low": 48.365, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.38, "open": 48.365, "day": "31"} +{"volume": 20862, "symbol": "INTC", "ts": "2018-08-31 10:44:00", "month": "08", "high": 48.3901, "low": 48.385, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.39, "open": 48.3901, "day": "31"} +{"volume": 25634, "symbol": "INTC", "ts": "2018-08-31 10:45:00", "month": "08", "high": 48.39, "low": 48.37, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.375, "open": 48.39, "day": "31"} +{"volume": 23987, "symbol": "INTC", "ts": "2018-08-31 10:46:00", "month": "08", "high": 48.4, "low": 48.375, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.375, "open": 48.4, "day": "31"} +{"volume": 38194, "symbol": "INTC", "ts": "2018-08-31 10:47:00", "month": "08", "high": 48.365, "low": 48.34, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.355, "open": 48.365, "day": "31"} +{"volume": 23323, "symbol": "INTC", "ts": "2018-08-31 10:48:00", "month": "08", "high": 48.38, "low": 48.34, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.345, "open": 48.38, "day": "31"} +{"volume": 22203, "symbol": "INTC", "ts": "2018-08-31 10:49:00", "month": "08", "high": 48.37, "low": 48.345, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.345, "open": 48.35, "day": "31"} +{"volume": 15571, "symbol": "INTC", "ts": "2018-08-31 10:50:00", "month": "08", "high": 48.35, "low": 48.34, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.34, "open": 48.35, "day": "31"} +{"volume": 19877, "symbol": "INTC", "ts": "2018-08-31 10:51:00", "month": "08", "high": 48.36, "low": 48.32, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.35, "open": 48.32, "day": "31"} +{"volume": 23827, "symbol": "INTC", "ts": "2018-08-31 10:52:00", "month": "08", "high": 48.39, "low": 48.3664, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.37, "open": 48.3664, "day": "31"} +{"volume": 24501, "symbol": "INTC", "ts": "2018-08-31 10:53:00", "month": "08", "high": 48.39, "low": 48.37, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.38, "open": 48.37, "day": "31"} +{"volume": 51740, "symbol": "INTC", "ts": "2018-08-31 10:54:00", "month": "08", "high": 48.44, "low": 48.39, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.43, "open": 48.39, "day": "31"} +{"volume": 30172, "symbol": "INTC", "ts": "2018-08-31 10:55:00", "month": "08", "high": 48.444, "low": 48.43, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.44, "open": 48.444, "day": "31"} +{"volume": 15095, "symbol": "INTC", "ts": "2018-08-31 10:56:00", "month": "08", "high": 48.42, "low": 48.4, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.411, "open": 48.42, "day": "31"} +{"volume": 51434, "symbol": "INTC", "ts": "2018-08-31 10:57:00", "month": "08", "high": 48.429, "low": 48.401, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.41, "open": 48.429, "day": "31"} +{"volume": 37761, "symbol": "INTC", "ts": "2018-08-31 10:58:00", "month": "08", "high": 48.41, "low": 48.39, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.4, "open": 48.41, "day": "31"} +{"volume": 17199, "symbol": "INTC", "ts": "2018-08-31 10:59:00", "month": "08", "high": 48.395, "low": 48.39, "key": "INTC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.39, "open": 48.395, "day": "31"} +{"volume": 31450, "symbol": "TNDM", "ts": "2018-08-31 10:31:00", "month": "08", "high": 45.18, "low": 45.0, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.18, "open": 45.0, "day": "31"} +{"volume": 11371, "symbol": "TNDM", "ts": "2018-08-31 10:32:00", "month": "08", "high": 45.1732, "low": 45.11, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.17, "open": 45.1732, "day": "31"} +{"volume": 8592, "symbol": "TNDM", "ts": "2018-08-31 10:33:00", "month": "08", "high": 45.0817, "low": 45.0817, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.0817, "open": 45.0817, "day": "31"} +{"volume": 11494, "symbol": "TNDM", "ts": "2018-08-31 10:34:00", "month": "08", "high": 45.24, "low": 45.235, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.24, "open": 45.24, "day": "31"} +{"volume": 40590, "symbol": "TNDM", "ts": "2018-08-31 10:35:00", "month": "08", "high": 45.3, "low": 45.19, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.3, "open": 45.23, "day": "31"} +{"volume": 9006, "symbol": "TNDM", "ts": "2018-08-31 10:36:00", "month": "08", "high": 45.38, "low": 45.3, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.38, "open": 45.3, "day": "31"} +{"volume": 13927, "symbol": "TNDM", "ts": "2018-08-31 10:37:00", "month": "08", "high": 45.4092, "low": 45.3562, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.4092, "open": 45.3562, "day": "31"} +{"volume": 16546, "symbol": "TNDM", "ts": "2018-08-31 10:38:00", "month": "08", "high": 45.6072, "low": 45.45, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.6072, "open": 45.45, "day": "31"} +{"volume": 8481, "symbol": "TNDM", "ts": "2018-08-31 10:39:00", "month": "08", "high": 45.56, "low": 45.44, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.44, "open": 45.56, "day": "31"} +{"volume": 17933, "symbol": "TNDM", "ts": "2018-08-31 10:40:00", "month": "08", "high": 45.58, "low": 45.47, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.58, "open": 45.49, "day": "31"} +{"volume": 11093, "symbol": "TNDM", "ts": "2018-08-31 10:41:00", "month": "08", "high": 45.65, "low": 45.5567, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.65, "open": 45.56, "day": "31"} +{"volume": 11691, "symbol": "TNDM", "ts": "2018-08-31 10:42:00", "month": "08", "high": 45.79, "low": 45.7002, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7002, "open": 45.79, "day": "31"} +{"volume": 14152, "symbol": "TNDM", "ts": "2018-08-31 10:43:00", "month": "08", "high": 45.75, "low": 45.64, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.64, "open": 45.75, "day": "31"} +{"volume": 44335, "symbol": "TNDM", "ts": "2018-08-31 10:44:00", "month": "08", "high": 45.86, "low": 45.71, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.85, "open": 45.71, "day": "31"} +{"volume": 19322, "symbol": "TNDM", "ts": "2018-08-31 10:45:00", "month": "08", "high": 45.9777, "low": 45.94, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.9777, "open": 45.94, "day": "31"} +{"volume": 25949, "symbol": "TNDM", "ts": "2018-08-31 10:46:00", "month": "08", "high": 45.9558, "low": 45.86, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.9, "open": 45.9558, "day": "31"} +{"volume": 15523, "symbol": "TNDM", "ts": "2018-08-31 10:47:00", "month": "08", "high": 45.84, "low": 45.82, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.84, "open": 45.82, "day": "31"} +{"volume": 11003, "symbol": "TNDM", "ts": "2018-08-31 10:48:00", "month": "08", "high": 45.89, "low": 45.86, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.86, "open": 45.89, "day": "31"} +{"volume": 7833, "symbol": "TNDM", "ts": "2018-08-31 10:49:00", "month": "08", "high": 45.929, "low": 45.86, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.929, "open": 45.86, "day": "31"} +{"volume": 23389, "symbol": "TNDM", "ts": "2018-08-31 10:50:00", "month": "08", "high": 45.9684, "low": 45.92, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.935, "open": 45.92, "day": "31"} +{"volume": 28686, "symbol": "TNDM", "ts": "2018-08-31 10:51:00", "month": "08", "high": 45.89, "low": 45.745, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.89, "open": 45.8, "day": "31"} +{"volume": 12373, "symbol": "TNDM", "ts": "2018-08-31 10:52:00", "month": "08", "high": 45.83, "low": 45.778, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.778, "open": 45.83, "day": "31"} +{"volume": 15252, "symbol": "TNDM", "ts": "2018-08-31 10:53:00", "month": "08", "high": 45.84, "low": 45.7999, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7999, "open": 45.8, "day": "31"} +{"volume": 5585, "symbol": "TNDM", "ts": "2018-08-31 10:54:00", "month": "08", "high": 45.7854, "low": 45.7854, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7854, "open": 45.7854, "day": "31"} +{"volume": 17284, "symbol": "TNDM", "ts": "2018-08-31 10:55:00", "month": "08", "high": 45.79, "low": 45.75, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.77, "open": 45.79, "day": "31"} +{"volume": 10376, "symbol": "TNDM", "ts": "2018-08-31 10:56:00", "month": "08", "high": 45.78, "low": 45.78, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.78, "open": 45.78, "day": "31"} +{"volume": 25580, "symbol": "TNDM", "ts": "2018-08-31 10:57:00", "month": "08", "high": 45.65, "low": 45.52, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.52, "open": 45.62, "day": "31"} +{"volume": 15068, "symbol": "TNDM", "ts": "2018-08-31 10:58:00", "month": "08", "high": 45.59, "low": 45.59, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.59, "open": 45.59, "day": "31"} +{"volume": 10711, "symbol": "TNDM", "ts": "2018-08-31 10:59:00", "month": "08", "high": 45.74, "low": 45.66, "key": "TNDM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.74, "open": 45.66, "day": "31"} +{"volume": 7160, "symbol": "CDNA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 23.9168, "low": 23.8407, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.8407, "open": 23.9168, "day": "31"} +{"volume": 2377, "symbol": "CDNA", "ts": "2018-08-31 10:33:00", "month": "08", "high": 23.86, "low": 23.86, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.86, "open": 23.86, "day": "31"} +{"volume": 2192, "symbol": "CDNA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 23.9067, "low": 23.9067, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.9067, "open": 23.9067, "day": "31"} +{"volume": 2750, "symbol": "CDNA", "ts": "2018-08-31 10:36:00", "month": "08", "high": 23.89, "low": 23.89, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.89, "open": 23.89, "day": "31"} +{"volume": 851, "symbol": "CDNA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 23.905, "low": 23.905, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.905, "open": 23.905, "day": "31"} +{"volume": 1874, "symbol": "CDNA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 23.92, "low": 23.92, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.92, "open": 23.92, "day": "31"} +{"volume": 1629, "symbol": "CDNA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 23.86, "low": 23.86, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.86, "open": 23.86, "day": "31"} +{"volume": 103, "symbol": "CDNA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 23.86, "low": 23.86, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.86, "open": 23.86, "day": "31"} +{"volume": 210, "symbol": "CDNA", "ts": "2018-08-31 10:43:00", "month": "08", "high": 23.87, "low": 23.87, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.87, "open": 23.87, "day": "31"} +{"volume": 5773, "symbol": "CDNA", "ts": "2018-08-31 10:45:00", "month": "08", "high": 23.94, "low": 23.94, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.94, "open": 23.94, "day": "31"} +{"volume": 7602, "symbol": "CDNA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 23.82, "low": 23.82, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.82, "open": 23.82, "day": "31"} +{"volume": 503, "symbol": "CDNA", "ts": "2018-08-31 10:48:00", "month": "08", "high": 23.85, "low": 23.85, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.85, "open": 23.85, "day": "31"} +{"volume": 1427, "symbol": "CDNA", "ts": "2018-08-31 10:51:00", "month": "08", "high": 23.8, "low": 23.8, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.8, "open": 23.8, "day": "31"} +{"volume": 1614, "symbol": "CDNA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 23.8, "low": 23.8, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.8, "open": 23.8, "day": "31"} +{"volume": 1528, "symbol": "CDNA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 23.81, "low": 23.81, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.81, "open": 23.81, "day": "31"} +{"volume": 2251, "symbol": "CDNA", "ts": "2018-08-31 10:56:00", "month": "08", "high": 23.8188, "low": 23.8188, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.8188, "open": 23.8188, "day": "31"} +{"volume": 2050, "symbol": "CDNA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 23.86, "low": 23.86, "key": "CDNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.86, "open": 23.86, "day": "31"} +{"volume": 1667, "symbol": "IIN", "ts": "2018-08-31 10:38:00", "month": "08", "high": 74.1983, "low": 74.1983, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.1983, "open": 74.1983, "day": "31"} +{"volume": 297, "symbol": "IIN", "ts": "2018-08-31 10:41:00", "month": "08", "high": 74.175, "low": 74.175, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.175, "open": 74.175, "day": "31"} +{"volume": 3623, "symbol": "IIN", "ts": "2018-08-31 10:43:00", "month": "08", "high": 74.525, "low": 74.525, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.525, "open": 74.525, "day": "31"} +{"volume": 1834, "symbol": "IIN", "ts": "2018-08-31 10:45:00", "month": "08", "high": 74.3052, "low": 74.3052, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.3052, "open": 74.3052, "day": "31"} +{"volume": 475, "symbol": "IIN", "ts": "2018-08-31 10:46:00", "month": "08", "high": 74.37, "low": 74.37, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.37, "open": 74.37, "day": "31"} +{"volume": 1543, "symbol": "IIN", "ts": "2018-08-31 10:48:00", "month": "08", "high": 74.5, "low": 74.444, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.444, "open": 74.5, "day": "31"} +{"volume": 1096, "symbol": "IIN", "ts": "2018-08-31 10:51:00", "month": "08", "high": 74.4852, "low": 74.4852, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.4852, "open": 74.4852, "day": "31"} +{"volume": 1218, "symbol": "IIN", "ts": "2018-08-31 10:53:00", "month": "08", "high": 74.25, "low": 74.25, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.25, "open": 74.25, "day": "31"} +{"volume": 2304, "symbol": "IIN", "ts": "2018-08-31 10:54:00", "month": "08", "high": 74.2997, "low": 74.2997, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.2997, "open": 74.2997, "day": "31"} +{"volume": 2272, "symbol": "IIN", "ts": "2018-08-31 10:58:00", "month": "08", "high": 74.25, "low": 74.25, "key": "IIN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.25, "open": 74.25, "day": "31"} +{"volume": 100, "symbol": "TPNL", "ts": "2018-08-31 10:35:00", "month": "08", "high": 3.36, "low": 3.36, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 100, "symbol": "TPNL", "ts": "2018-08-31 10:42:00", "month": "08", "high": 3.35, "low": 3.35, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.35, "open": 3.35, "day": "31"} +{"volume": 100, "symbol": "TPNL", "ts": "2018-08-31 10:48:00", "month": "08", "high": 3.36, "low": 3.36, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 100, "symbol": "TPNL", "ts": "2018-08-31 10:56:00", "month": "08", "high": 3.38, "low": 3.38, "key": "TPNL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.38, "open": 3.38, "day": "31"} +{"volume": 44806, "symbol": "CGC", "ts": "2018-08-31 10:31:00", "month": "08", "high": 45.78, "low": 45.59, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.59, "open": 45.78, "day": "31"} +{"volume": 94271, "symbol": "CGC", "ts": "2018-08-31 10:32:00", "month": "08", "high": 45.61, "low": 45.5487, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.61, "open": 45.6, "day": "31"} +{"volume": 38084, "symbol": "CGC", "ts": "2018-08-31 10:33:00", "month": "08", "high": 45.67, "low": 45.54, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.65, "open": 45.54, "day": "31"} +{"volume": 26385, "symbol": "CGC", "ts": "2018-08-31 10:34:00", "month": "08", "high": 45.59, "low": 45.55, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.55, "open": 45.59, "day": "31"} +{"volume": 33161, "symbol": "CGC", "ts": "2018-08-31 10:35:00", "month": "08", "high": 45.71, "low": 45.6, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.71, "open": 45.6, "day": "31"} +{"volume": 48849, "symbol": "CGC", "ts": "2018-08-31 10:36:00", "month": "08", "high": 45.836, "low": 45.75, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.836, "open": 45.75, "day": "31"} +{"volume": 39699, "symbol": "CGC", "ts": "2018-08-31 10:37:00", "month": "08", "high": 45.87, "low": 45.8012, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.85, "open": 45.86, "day": "31"} +{"volume": 31371, "symbol": "CGC", "ts": "2018-08-31 10:38:00", "month": "08", "high": 45.805, "low": 45.7, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.7, "open": 45.805, "day": "31"} +{"volume": 20969, "symbol": "CGC", "ts": "2018-08-31 10:39:00", "month": "08", "high": 45.777, "low": 45.6624, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.74, "open": 45.777, "day": "31"} +{"volume": 19774, "symbol": "CGC", "ts": "2018-08-31 10:40:00", "month": "08", "high": 45.829, "low": 45.71, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.76, "open": 45.71, "day": "31"} +{"volume": 44557, "symbol": "CGC", "ts": "2018-08-31 10:41:00", "month": "08", "high": 45.8198, "low": 45.8001, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.8198, "open": 45.8001, "day": "31"} +{"volume": 25502, "symbol": "CGC", "ts": "2018-08-31 10:42:00", "month": "08", "high": 45.77, "low": 45.69, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.6929, "open": 45.77, "day": "31"} +{"volume": 37897, "symbol": "CGC", "ts": "2018-08-31 10:43:00", "month": "08", "high": 45.6566, "low": 45.545, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.545, "open": 45.6566, "day": "31"} +{"volume": 85090, "symbol": "CGC", "ts": "2018-08-31 10:44:00", "month": "08", "high": 45.5411, "low": 45.45, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.5411, "open": 45.5, "day": "31"} +{"volume": 36182, "symbol": "CGC", "ts": "2018-08-31 10:45:00", "month": "08", "high": 45.6168, "low": 45.55, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.6168, "open": 45.5803, "day": "31"} +{"volume": 43554, "symbol": "CGC", "ts": "2018-08-31 10:46:00", "month": "08", "high": 45.68, "low": 45.58, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.6448, "open": 45.65, "day": "31"} +{"volume": 8535, "symbol": "CGC", "ts": "2018-08-31 10:47:00", "month": "08", "high": 45.62, "low": 45.6, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.6, "open": 45.62, "day": "31"} +{"volume": 17917, "symbol": "CGC", "ts": "2018-08-31 10:48:00", "month": "08", "high": 45.63, "low": 45.61, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.61, "open": 45.62, "day": "31"} +{"volume": 35545, "symbol": "CGC", "ts": "2018-08-31 10:49:00", "month": "08", "high": 45.69, "low": 45.62, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.6492, "open": 45.62, "day": "31"} +{"volume": 40791, "symbol": "CGC", "ts": "2018-08-31 10:50:00", "month": "08", "high": 45.66, "low": 45.57, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.57, "open": 45.66, "day": "31"} +{"volume": 47609, "symbol": "CGC", "ts": "2018-08-31 10:51:00", "month": "08", "high": 45.562, "low": 45.47, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.53, "open": 45.54, "day": "31"} +{"volume": 11649, "symbol": "CGC", "ts": "2018-08-31 10:52:00", "month": "08", "high": 45.5439, "low": 45.4586, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.4586, "open": 45.5439, "day": "31"} +{"volume": 64399, "symbol": "CGC", "ts": "2018-08-31 10:53:00", "month": "08", "high": 45.5, "low": 45.37, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.42, "open": 45.4684, "day": "31"} +{"volume": 32142, "symbol": "CGC", "ts": "2018-08-31 10:54:00", "month": "08", "high": 45.43, "low": 45.34, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.43, "open": 45.39, "day": "31"} +{"volume": 37239, "symbol": "CGC", "ts": "2018-08-31 10:55:00", "month": "08", "high": 45.55, "low": 45.3811, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.55, "open": 45.4015, "day": "31"} +{"volume": 25593, "symbol": "CGC", "ts": "2018-08-31 10:56:00", "month": "08", "high": 45.58, "low": 45.51, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.51, "open": 45.53, "day": "31"} +{"volume": 31697, "symbol": "CGC", "ts": "2018-08-31 10:57:00", "month": "08", "high": 45.45, "low": 45.3976, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.45, "open": 45.3976, "day": "31"} +{"volume": 23189, "symbol": "CGC", "ts": "2018-08-31 10:58:00", "month": "08", "high": 45.39, "low": 45.37, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.37, "open": 45.38, "day": "31"} +{"volume": 53385, "symbol": "CGC", "ts": "2018-08-31 10:59:00", "month": "08", "high": 45.34, "low": 45.26, "key": "CGC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.29, "open": 45.3328, "day": "31"} +{"volume": 1522, "symbol": "FNKO", "ts": "2018-08-31 10:32:00", "month": "08", "high": 25.01, "low": 25.01, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.01, "open": 25.01, "day": "31"} +{"volume": 2141, "symbol": "FNKO", "ts": "2018-08-31 10:36:00", "month": "08", "high": 25.1, "low": 25.1, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.1, "open": 25.1, "day": "31"} +{"volume": 2505, "symbol": "FNKO", "ts": "2018-08-31 10:38:00", "month": "08", "high": 25.05, "low": 25.05, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.05, "open": 25.05, "day": "31"} +{"volume": 9343, "symbol": "FNKO", "ts": "2018-08-31 10:39:00", "month": "08", "high": 25.0323, "low": 25.0323, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0323, "open": 25.0323, "day": "31"} +{"volume": 1644, "symbol": "FNKO", "ts": "2018-08-31 10:40:00", "month": "08", "high": 25.07, "low": 25.07, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.07, "open": 25.07, "day": "31"} +{"volume": 1932, "symbol": "FNKO", "ts": "2018-08-31 10:41:00", "month": "08", "high": 25.0, "low": 25.0, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0, "open": 25.0, "day": "31"} +{"volume": 366, "symbol": "FNKO", "ts": "2018-08-31 10:43:00", "month": "08", "high": 24.9617, "low": 24.9617, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9617, "open": 24.9617, "day": "31"} +{"volume": 2476, "symbol": "FNKO", "ts": "2018-08-31 10:45:00", "month": "08", "high": 24.9545, "low": 24.9545, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9545, "open": 24.9545, "day": "31"} +{"volume": 493, "symbol": "FNKO", "ts": "2018-08-31 10:47:00", "month": "08", "high": 24.96, "low": 24.96, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.96, "open": 24.96, "day": "31"} +{"volume": 583, "symbol": "FNKO", "ts": "2018-08-31 10:49:00", "month": "08", "high": 24.95, "low": 24.95, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.95, "open": 24.95, "day": "31"} +{"volume": 1043, "symbol": "FNKO", "ts": "2018-08-31 10:52:00", "month": "08", "high": 24.87, "low": 24.87, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.87, "open": 24.87, "day": "31"} +{"volume": 1890, "symbol": "FNKO", "ts": "2018-08-31 10:53:00", "month": "08", "high": 24.9589, "low": 24.9589, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9589, "open": 24.9589, "day": "31"} +{"volume": 1237, "symbol": "FNKO", "ts": "2018-08-31 10:57:00", "month": "08", "high": 25.049, "low": 25.049, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.049, "open": 25.049, "day": "31"} +{"volume": 655, "symbol": "FNKO", "ts": "2018-08-31 10:58:00", "month": "08", "high": 25.09, "low": 25.09, "key": "FNKO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.09, "open": 25.09, "day": "31"} +{"volume": 3967, "symbol": "XENE", "ts": "2018-08-31 10:35:00", "month": "08", "high": 13.2775, "low": 13.2775, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.2775, "open": 13.2775, "day": "31"} +{"volume": 701, "symbol": "XENE", "ts": "2018-08-31 10:36:00", "month": "08", "high": 13.2055, "low": 13.2055, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.2055, "open": 13.2055, "day": "31"} +{"volume": 2137, "symbol": "XENE", "ts": "2018-08-31 10:46:00", "month": "08", "high": 13.1339, "low": 13.1339, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.1339, "open": 13.1339, "day": "31"} +{"volume": 1538, "symbol": "XENE", "ts": "2018-08-31 10:48:00", "month": "08", "high": 13.0, "low": 13.0, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.0, "open": 13.0, "day": "31"} +{"volume": 860, "symbol": "XENE", "ts": "2018-08-31 10:50:00", "month": "08", "high": 13.034, "low": 13.034, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.034, "open": 13.034, "day": "31"} +{"volume": 967, "symbol": "XENE", "ts": "2018-08-31 10:53:00", "month": "08", "high": 13.0811, "low": 13.0811, "key": "XENE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.0811, "open": 13.0811, "day": "31"} +{"volume": 2184, "symbol": "SSTI", "ts": "2018-08-31 10:31:00", "month": "08", "high": 55.92, "low": 55.92, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.92, "open": 55.92, "day": "31"} +{"volume": 2362, "symbol": "SSTI", "ts": "2018-08-31 10:32:00", "month": "08", "high": 55.8, "low": 55.8, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.8, "open": 55.8, "day": "31"} +{"volume": 2188, "symbol": "SSTI", "ts": "2018-08-31 10:33:00", "month": "08", "high": 55.81, "low": 55.81, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.81, "open": 55.81, "day": "31"} +{"volume": 1572, "symbol": "SSTI", "ts": "2018-08-31 10:35:00", "month": "08", "high": 55.97, "low": 55.97, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.97, "open": 55.97, "day": "31"} +{"volume": 974, "symbol": "SSTI", "ts": "2018-08-31 10:37:00", "month": "08", "high": 55.96, "low": 55.96, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.96, "open": 55.96, "day": "31"} +{"volume": 1967, "symbol": "SSTI", "ts": "2018-08-31 10:38:00", "month": "08", "high": 55.77, "low": 55.77, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.77, "open": 55.77, "day": "31"} +{"volume": 3448, "symbol": "SSTI", "ts": "2018-08-31 10:44:00", "month": "08", "high": 56.2, "low": 55.93, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.2, "open": 55.93, "day": "31"} +{"volume": 1206, "symbol": "SSTI", "ts": "2018-08-31 10:46:00", "month": "08", "high": 56.44, "low": 56.44, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.44, "open": 56.44, "day": "31"} +{"volume": 5174, "symbol": "SSTI", "ts": "2018-08-31 10:47:00", "month": "08", "high": 56.69, "low": 56.69, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.69, "open": 56.69, "day": "31"} +{"volume": 1616, "symbol": "SSTI", "ts": "2018-08-31 10:48:00", "month": "08", "high": 56.8, "low": 56.8, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.8, "open": 56.8, "day": "31"} +{"volume": 2043, "symbol": "SSTI", "ts": "2018-08-31 10:49:00", "month": "08", "high": 56.89, "low": 56.89, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.89, "open": 56.89, "day": "31"} +{"volume": 2541, "symbol": "SSTI", "ts": "2018-08-31 10:50:00", "month": "08", "high": 56.7743, "low": 56.7743, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.7743, "open": 56.7743, "day": "31"} +{"volume": 2640, "symbol": "SSTI", "ts": "2018-08-31 10:51:00", "month": "08", "high": 56.525, "low": 56.525, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.525, "open": 56.525, "day": "31"} +{"volume": 1190, "symbol": "SSTI", "ts": "2018-08-31 10:52:00", "month": "08", "high": 56.71, "low": 56.71, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.71, "open": 56.71, "day": "31"} +{"volume": 1589, "symbol": "SSTI", "ts": "2018-08-31 10:53:00", "month": "08", "high": 56.72, "low": 56.72, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.72, "open": 56.72, "day": "31"} +{"volume": 2621, "symbol": "SSTI", "ts": "2018-08-31 10:54:00", "month": "08", "high": 56.72, "low": 56.72, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.72, "open": 56.72, "day": "31"} +{"volume": 2474, "symbol": "SSTI", "ts": "2018-08-31 10:55:00", "month": "08", "high": 56.83, "low": 56.83, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.83, "open": 56.83, "day": "31"} +{"volume": 1228, "symbol": "SSTI", "ts": "2018-08-31 10:56:00", "month": "08", "high": 56.87, "low": 56.87, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.87, "open": 56.87, "day": "31"} +{"volume": 4359, "symbol": "SSTI", "ts": "2018-08-31 10:58:00", "month": "08", "high": 57.07, "low": 56.9, "key": "SSTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.07, "open": 56.9, "day": "31"} +{"volume": 8243, "symbol": "VRS", "ts": "2018-08-31 10:31:00", "month": "08", "high": 31.155, "low": 31.155, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.155, "open": 31.155, "day": "31"} +{"volume": 1534, "symbol": "VRS", "ts": "2018-08-31 10:32:00", "month": "08", "high": 31.17, "low": 31.17, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.17, "open": 31.17, "day": "31"} +{"volume": 4252, "symbol": "VRS", "ts": "2018-08-31 10:33:00", "month": "08", "high": 31.17, "low": 31.17, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.17, "open": 31.17, "day": "31"} +{"volume": 1472, "symbol": "VRS", "ts": "2018-08-31 10:34:00", "month": "08", "high": 31.1644, "low": 31.1644, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1644, "open": 31.1644, "day": "31"} +{"volume": 11234, "symbol": "VRS", "ts": "2018-08-31 10:35:00", "month": "08", "high": 31.31, "low": 31.18, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.22, "open": 31.18, "day": "31"} +{"volume": 12705, "symbol": "VRS", "ts": "2018-08-31 10:36:00", "month": "08", "high": 31.315, "low": 31.31, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.31, "open": 31.315, "day": "31"} +{"volume": 9680, "symbol": "VRS", "ts": "2018-08-31 10:37:00", "month": "08", "high": 31.325, "low": 31.3, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.325, "open": 31.3, "day": "31"} +{"volume": 7831, "symbol": "VRS", "ts": "2018-08-31 10:38:00", "month": "08", "high": 31.38, "low": 31.38, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.38, "open": 31.38, "day": "31"} +{"volume": 2572, "symbol": "VRS", "ts": "2018-08-31 10:39:00", "month": "08", "high": 31.38, "low": 31.38, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.38, "open": 31.38, "day": "31"} +{"volume": 3341, "symbol": "VRS", "ts": "2018-08-31 10:40:00", "month": "08", "high": 31.33, "low": 31.33, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.33, "open": 31.33, "day": "31"} +{"volume": 2764, "symbol": "VRS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 31.31, "low": 31.31, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.31, "open": 31.31, "day": "31"} +{"volume": 1554, "symbol": "VRS", "ts": "2018-08-31 10:43:00", "month": "08", "high": 31.2775, "low": 31.2775, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.2775, "open": 31.2775, "day": "31"} +{"volume": 3886, "symbol": "VRS", "ts": "2018-08-31 10:45:00", "month": "08", "high": 31.27, "low": 31.27, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.27, "open": 31.27, "day": "31"} +{"volume": 1961, "symbol": "VRS", "ts": "2018-08-31 10:46:00", "month": "08", "high": 31.2387, "low": 31.2387, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.2387, "open": 31.2387, "day": "31"} +{"volume": 2298, "symbol": "VRS", "ts": "2018-08-31 10:47:00", "month": "08", "high": 31.21, "low": 31.21, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.21, "open": 31.21, "day": "31"} +{"volume": 2164, "symbol": "VRS", "ts": "2018-08-31 10:48:00", "month": "08", "high": 31.21, "low": 31.21, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.21, "open": 31.21, "day": "31"} +{"volume": 2206, "symbol": "VRS", "ts": "2018-08-31 10:50:00", "month": "08", "high": 31.15, "low": 31.15, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.15, "open": 31.15, "day": "31"} +{"volume": 6998, "symbol": "VRS", "ts": "2018-08-31 10:51:00", "month": "08", "high": 31.15, "low": 31.11, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.11, "open": 31.15, "day": "31"} +{"volume": 2996, "symbol": "VRS", "ts": "2018-08-31 10:52:00", "month": "08", "high": 31.1, "low": 31.1, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.1, "open": 31.1, "day": "31"} +{"volume": 2469, "symbol": "VRS", "ts": "2018-08-31 10:54:00", "month": "08", "high": 31.095, "low": 31.095, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.095, "open": 31.095, "day": "31"} +{"volume": 1678, "symbol": "VRS", "ts": "2018-08-31 10:55:00", "month": "08", "high": 31.08, "low": 31.08, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.08, "open": 31.08, "day": "31"} +{"volume": 4660, "symbol": "VRS", "ts": "2018-08-31 10:56:00", "month": "08", "high": 31.07, "low": 31.07, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.07, "open": 31.07, "day": "31"} +{"volume": 6616, "symbol": "VRS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 31.07, "low": 31.06, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.06, "open": 31.07, "day": "31"} +{"volume": 5097, "symbol": "VRS", "ts": "2018-08-31 10:58:00", "month": "08", "high": 31.075, "low": 31.075, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.075, "open": 31.075, "day": "31"} +{"volume": 6798, "symbol": "VRS", "ts": "2018-08-31 10:59:00", "month": "08", "high": 31.095, "low": 31.09, "key": "VRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 31.095, "open": 31.09, "day": "31"} +{"volume": 1710, "symbol": "ARWR", "ts": "2018-08-31 10:32:00", "month": "08", "high": 14.425, "low": 14.425, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.425, "open": 14.425, "day": "31"} +{"volume": 6446, "symbol": "ARWR", "ts": "2018-08-31 10:33:00", "month": "08", "high": 14.42, "low": 14.42, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.42, "open": 14.42, "day": "31"} +{"volume": 3501, "symbol": "ARWR", "ts": "2018-08-31 10:34:00", "month": "08", "high": 14.42, "low": 14.42, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.42, "open": 14.42, "day": "31"} +{"volume": 5745, "symbol": "ARWR", "ts": "2018-08-31 10:35:00", "month": "08", "high": 14.42, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.42, "open": 14.4, "day": "31"} +{"volume": 1543, "symbol": "ARWR", "ts": "2018-08-31 10:36:00", "month": "08", "high": 14.415, "low": 14.415, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.415, "open": 14.415, "day": "31"} +{"volume": 302, "symbol": "ARWR", "ts": "2018-08-31 10:38:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 5298, "symbol": "ARWR", "ts": "2018-08-31 10:39:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 3100, "symbol": "ARWR", "ts": "2018-08-31 10:40:00", "month": "08", "high": 14.415, "low": 14.415, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.415, "open": 14.415, "day": "31"} +{"volume": 2842, "symbol": "ARWR", "ts": "2018-08-31 10:41:00", "month": "08", "high": 14.45, "low": 14.45, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.45, "open": 14.45, "day": "31"} +{"volume": 1825, "symbol": "ARWR", "ts": "2018-08-31 10:42:00", "month": "08", "high": 14.47, "low": 14.47, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.47, "open": 14.47, "day": "31"} +{"volume": 1914, "symbol": "ARWR", "ts": "2018-08-31 10:43:00", "month": "08", "high": 14.5, "low": 14.5, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.5, "open": 14.5, "day": "31"} +{"volume": 3477, "symbol": "ARWR", "ts": "2018-08-31 10:44:00", "month": "08", "high": 14.45, "low": 14.45, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.45, "open": 14.45, "day": "31"} +{"volume": 4762, "symbol": "ARWR", "ts": "2018-08-31 10:45:00", "month": "08", "high": 14.44, "low": 14.42, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.42, "open": 14.44, "day": "31"} +{"volume": 2642, "symbol": "ARWR", "ts": "2018-08-31 10:46:00", "month": "08", "high": 14.42, "low": 14.42, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.42, "open": 14.42, "day": "31"} +{"volume": 2468, "symbol": "ARWR", "ts": "2018-08-31 10:48:00", "month": "08", "high": 14.39, "low": 14.39, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.39, "open": 14.39, "day": "31"} +{"volume": 6656, "symbol": "ARWR", "ts": "2018-08-31 10:49:00", "month": "08", "high": 14.4, "low": 14.385, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.385, "day": "31"} +{"volume": 1877, "symbol": "ARWR", "ts": "2018-08-31 10:50:00", "month": "08", "high": 14.39, "low": 14.39, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.39, "open": 14.39, "day": "31"} +{"volume": 101, "symbol": "ARWR", "ts": "2018-08-31 10:52:00", "month": "08", "high": 14.395, "low": 14.395, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.395, "open": 14.395, "day": "31"} +{"volume": 11999, "symbol": "ARWR", "ts": "2018-08-31 10:53:00", "month": "08", "high": 14.46, "low": 14.395, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.46, "open": 14.395, "day": "31"} +{"volume": 5055, "symbol": "ARWR", "ts": "2018-08-31 10:54:00", "month": "08", "high": 14.39, "low": 14.39, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.39, "open": 14.39, "day": "31"} +{"volume": 7372, "symbol": "ARWR", "ts": "2018-08-31 10:55:00", "month": "08", "high": 14.422, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.422, "day": "31"} +{"volume": 776, "symbol": "ARWR", "ts": "2018-08-31 10:56:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 2114, "symbol": "ARWR", "ts": "2018-08-31 10:58:00", "month": "08", "high": 14.395, "low": 14.395, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.395, "open": 14.395, "day": "31"} +{"volume": 2260, "symbol": "ARWR", "ts": "2018-08-31 10:59:00", "month": "08", "high": 14.4, "low": 14.4, "key": "ARWR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.4, "open": 14.4, "day": "31"} +{"volume": 1847, "symbol": "MRTX", "ts": "2018-08-31 10:31:00", "month": "08", "high": 56.0, "low": 56.0, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.0, "open": 56.0, "day": "31"} +{"volume": 1541, "symbol": "MRTX", "ts": "2018-08-31 10:32:00", "month": "08", "high": 55.9, "low": 55.9, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.9, "open": 55.9, "day": "31"} +{"volume": 2664, "symbol": "MRTX", "ts": "2018-08-31 10:33:00", "month": "08", "high": 55.75, "low": 55.75, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.75, "open": 55.75, "day": "31"} +{"volume": 241, "symbol": "MRTX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 55.725, "low": 55.725, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.725, "open": 55.725, "day": "31"} +{"volume": 487, "symbol": "MRTX", "ts": "2018-08-31 10:37:00", "month": "08", "high": 55.7805, "low": 55.7805, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.7805, "open": 55.7805, "day": "31"} +{"volume": 1574, "symbol": "MRTX", "ts": "2018-08-31 10:38:00", "month": "08", "high": 55.65, "low": 55.65, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.65, "open": 55.65, "day": "31"} +{"volume": 1397, "symbol": "MRTX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 55.45, "low": 55.45, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.45, "open": 55.45, "day": "31"} +{"volume": 2146, "symbol": "MRTX", "ts": "2018-08-31 10:41:00", "month": "08", "high": 55.493, "low": 55.493, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.493, "open": 55.493, "day": "31"} +{"volume": 642, "symbol": "MRTX", "ts": "2018-08-31 10:43:00", "month": "08", "high": 55.55, "low": 55.55, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.55, "open": 55.55, "day": "31"} +{"volume": 355, "symbol": "MRTX", "ts": "2018-08-31 10:44:00", "month": "08", "high": 55.65, "low": 55.65, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.65, "open": 55.65, "day": "31"} +{"volume": 1887, "symbol": "MRTX", "ts": "2018-08-31 10:45:00", "month": "08", "high": 55.6, "low": 55.6, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.6, "open": 55.6, "day": "31"} +{"volume": 2496, "symbol": "MRTX", "ts": "2018-08-31 10:48:00", "month": "08", "high": 55.4, "low": 55.4, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.4, "open": 55.4, "day": "31"} +{"volume": 1220, "symbol": "MRTX", "ts": "2018-08-31 10:49:00", "month": "08", "high": 55.25, "low": 55.25, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.25, "open": 55.25, "day": "31"} +{"volume": 127, "symbol": "MRTX", "ts": "2018-08-31 10:51:00", "month": "08", "high": 55.4, "low": 55.4, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.4, "open": 55.4, "day": "31"} +{"volume": 967, "symbol": "MRTX", "ts": "2018-08-31 10:53:00", "month": "08", "high": 55.35, "low": 55.35, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.35, "open": 55.35, "day": "31"} +{"volume": 1326, "symbol": "MRTX", "ts": "2018-08-31 10:54:00", "month": "08", "high": 55.3171, "low": 55.3171, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.3171, "open": 55.3171, "day": "31"} +{"volume": 963, "symbol": "MRTX", "ts": "2018-08-31 10:57:00", "month": "08", "high": 55.35, "low": 55.35, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.35, "open": 55.35, "day": "31"} +{"volume": 202, "symbol": "MRTX", "ts": "2018-08-31 10:58:00", "month": "08", "high": 55.3, "low": 55.3, "key": "MRTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 55.3, "open": 55.3, "day": "31"} +{"volume": 3476, "symbol": "WWE", "ts": "2018-08-31 10:32:00", "month": "08", "high": 86.38, "low": 86.37, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.37, "open": 86.38, "day": "31"} +{"volume": 4699, "symbol": "WWE", "ts": "2018-08-31 10:33:00", "month": "08", "high": 86.35, "low": 86.31, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.35, "open": 86.31, "day": "31"} +{"volume": 1305, "symbol": "WWE", "ts": "2018-08-31 10:36:00", "month": "08", "high": 86.26, "low": 86.26, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.26, "open": 86.26, "day": "31"} +{"volume": 983, "symbol": "WWE", "ts": "2018-08-31 10:37:00", "month": "08", "high": 86.2768, "low": 86.2768, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.2768, "open": 86.2768, "day": "31"} +{"volume": 2892, "symbol": "WWE", "ts": "2018-08-31 10:38:00", "month": "08", "high": 86.5755, "low": 86.5755, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.5755, "open": 86.5755, "day": "31"} +{"volume": 1776, "symbol": "WWE", "ts": "2018-08-31 10:40:00", "month": "08", "high": 86.524, "low": 86.524, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.524, "open": 86.524, "day": "31"} +{"volume": 572, "symbol": "WWE", "ts": "2018-08-31 10:42:00", "month": "08", "high": 86.6682, "low": 86.6682, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.6682, "open": 86.6682, "day": "31"} +{"volume": 1871, "symbol": "WWE", "ts": "2018-08-31 10:44:00", "month": "08", "high": 86.76, "low": 86.72, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.76, "open": 86.72, "day": "31"} +{"volume": 835, "symbol": "WWE", "ts": "2018-08-31 10:47:00", "month": "08", "high": 86.86, "low": 86.86, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.86, "open": 86.86, "day": "31"} +{"volume": 745, "symbol": "WWE", "ts": "2018-08-31 10:49:00", "month": "08", "high": 86.797, "low": 86.797, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.797, "open": 86.797, "day": "31"} +{"volume": 6573, "symbol": "WWE", "ts": "2018-08-31 10:50:00", "month": "08", "high": 86.82, "low": 86.71, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.82, "open": 86.71, "day": "31"} +{"volume": 848, "symbol": "WWE", "ts": "2018-08-31 10:52:00", "month": "08", "high": 86.79, "low": 86.79, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.79, "open": 86.79, "day": "31"} +{"volume": 6281, "symbol": "WWE", "ts": "2018-08-31 10:55:00", "month": "08", "high": 86.89, "low": 86.78, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.78, "open": 86.89, "day": "31"} +{"volume": 1740, "symbol": "WWE", "ts": "2018-08-31 10:57:00", "month": "08", "high": 86.85, "low": 86.85, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.85, "open": 86.85, "day": "31"} +{"volume": 969, "symbol": "WWE", "ts": "2018-08-31 10:58:00", "month": "08", "high": 86.9893, "low": 86.9893, "key": "WWE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.9893, "open": 86.9893, "day": "31"} +{"volume": 2139, "symbol": "RFIL", "ts": "2018-08-31 10:33:00", "month": "08", "high": 11.75, "low": 11.7, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.7, "open": 11.75, "day": "31"} +{"volume": 359, "symbol": "RFIL", "ts": "2018-08-31 10:41:00", "month": "08", "high": 11.7828, "low": 11.7828, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.7828, "open": 11.7828, "day": "31"} +{"volume": 572, "symbol": "RFIL", "ts": "2018-08-31 10:55:00", "month": "08", "high": 11.8, "low": 11.8, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.8, "open": 11.8, "day": "31"} +{"volume": 1730, "symbol": "RFIL", "ts": "2018-08-31 10:58:00", "month": "08", "high": 11.845, "low": 11.845, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.845, "open": 11.845, "day": "31"} +{"volume": 975, "symbol": "RFIL", "ts": "2018-08-31 10:59:00", "month": "08", "high": 11.8, "low": 11.8, "key": "RFIL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 11.8, "open": 11.8, "day": "31"} +{"volume": 200, "symbol": "MED", "ts": "2018-08-31 10:34:00", "month": "08", "high": 227.7322, "low": 227.7322, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.7322, "open": 227.7322, "day": "31"} +{"volume": 441, "symbol": "MED", "ts": "2018-08-31 10:35:00", "month": "08", "high": 227.6873, "low": 227.6873, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 227.6873, "open": 227.6873, "day": "31"} +{"volume": 1288, "symbol": "MED", "ts": "2018-08-31 10:45:00", "month": "08", "high": 228.526, "low": 228.526, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.526, "open": 228.526, "day": "31"} +{"volume": 1291, "symbol": "MED", "ts": "2018-08-31 10:46:00", "month": "08", "high": 228.945, "low": 228.945, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 228.945, "open": 228.945, "day": "31"} +{"volume": 2284, "symbol": "MED", "ts": "2018-08-31 10:48:00", "month": "08", "high": 229.03, "low": 229.02, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 229.02, "open": 229.03, "day": "31"} +{"volume": 356, "symbol": "MED", "ts": "2018-08-31 10:55:00", "month": "08", "high": 229.1677, "low": 229.1677, "key": "MED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 229.1677, "open": 229.1677, "day": "31"} +{"volume": 600, "symbol": "STAA", "ts": "2018-08-31 10:31:00", "month": "08", "high": 47.6062, "low": 47.6062, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.6062, "open": 47.6062, "day": "31"} +{"volume": 100, "symbol": "STAA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 47.55, "low": 47.55, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.55, "open": 47.55, "day": "31"} +{"volume": 1408, "symbol": "STAA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 47.4, "low": 47.4, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.4, "open": 47.4, "day": "31"} +{"volume": 1821, "symbol": "STAA", "ts": "2018-08-31 10:38:00", "month": "08", "high": 47.52, "low": 47.52, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.52, "open": 47.52, "day": "31"} +{"volume": 466, "symbol": "STAA", "ts": "2018-08-31 10:39:00", "month": "08", "high": 47.545, "low": 47.545, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.545, "open": 47.545, "day": "31"} +{"volume": 1734, "symbol": "STAA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 47.55, "low": 47.55, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.55, "open": 47.55, "day": "31"} +{"volume": 100, "symbol": "STAA", "ts": "2018-08-31 10:43:00", "month": "08", "high": 47.5404, "low": 47.5404, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5404, "open": 47.5404, "day": "31"} +{"volume": 1244, "symbol": "STAA", "ts": "2018-08-31 10:45:00", "month": "08", "high": 47.6, "low": 47.6, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.6, "open": 47.6, "day": "31"} +{"volume": 637, "symbol": "STAA", "ts": "2018-08-31 10:48:00", "month": "08", "high": 47.5809, "low": 47.5809, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5809, "open": 47.5809, "day": "31"} +{"volume": 3542, "symbol": "STAA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 47.4, "low": 47.4, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.4, "open": 47.4, "day": "31"} +{"volume": 1656, "symbol": "STAA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 2538, "symbol": "STAA", "ts": "2018-08-31 10:57:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 7630, "symbol": "STAA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 2510, "symbol": "STAA", "ts": "2018-08-31 10:59:00", "month": "08", "high": 47.5, "low": 47.5, "key": "STAA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 47.5, "open": 47.5, "day": "31"} +{"volume": 154, "symbol": "TRHC", "ts": "2018-08-31 10:31:00", "month": "08", "high": 86.95, "low": 86.95, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.95, "open": 86.95, "day": "31"} +{"volume": 200, "symbol": "TRHC", "ts": "2018-08-31 10:34:00", "month": "08", "high": 86.81, "low": 86.81, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.81, "open": 86.81, "day": "31"} +{"volume": 307, "symbol": "TRHC", "ts": "2018-08-31 10:35:00", "month": "08", "high": 86.92, "low": 86.92, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.92, "open": 86.92, "day": "31"} +{"volume": 100, "symbol": "TRHC", "ts": "2018-08-31 10:36:00", "month": "08", "high": 86.92, "low": 86.92, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.92, "open": 86.92, "day": "31"} +{"volume": 454, "symbol": "TRHC", "ts": "2018-08-31 10:38:00", "month": "08", "high": 86.851, "low": 86.851, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.851, "open": 86.851, "day": "31"} +{"volume": 275, "symbol": "TRHC", "ts": "2018-08-31 10:39:00", "month": "08", "high": 86.92, "low": 86.92, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 86.92, "open": 86.92, "day": "31"} +{"volume": 1680, "symbol": "TRHC", "ts": "2018-08-31 10:41:00", "month": "08", "high": 87.01, "low": 87.01, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.01, "open": 87.01, "day": "31"} +{"volume": 1623, "symbol": "TRHC", "ts": "2018-08-31 10:43:00", "month": "08", "high": 87.13, "low": 87.13, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.13, "open": 87.13, "day": "31"} +{"volume": 479, "symbol": "TRHC", "ts": "2018-08-31 10:44:00", "month": "08", "high": 87.437, "low": 87.437, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.437, "open": 87.437, "day": "31"} +{"volume": 120, "symbol": "TRHC", "ts": "2018-08-31 10:46:00", "month": "08", "high": 87.55, "low": 87.55, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.55, "open": 87.55, "day": "31"} +{"volume": 480, "symbol": "TRHC", "ts": "2018-08-31 10:48:00", "month": "08", "high": 87.55, "low": 87.55, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.55, "open": 87.55, "day": "31"} +{"volume": 200, "symbol": "TRHC", "ts": "2018-08-31 10:50:00", "month": "08", "high": 87.57, "low": 87.57, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.57, "open": 87.57, "day": "31"} +{"volume": 309, "symbol": "TRHC", "ts": "2018-08-31 10:54:00", "month": "08", "high": 87.56, "low": 87.44, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.44, "open": 87.56, "day": "31"} +{"volume": 421, "symbol": "TRHC", "ts": "2018-08-31 10:57:00", "month": "08", "high": 87.54, "low": 87.54, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.54, "open": 87.54, "day": "31"} +{"volume": 917, "symbol": "TRHC", "ts": "2018-08-31 10:59:00", "month": "08", "high": 87.495, "low": 87.495, "key": "TRHC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 87.495, "open": 87.495, "day": "31"} +{"volume": 1018, "symbol": "I", "ts": "2018-08-31 10:31:00", "month": "08", "high": 21.86, "low": 21.86, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.86, "open": 21.86, "day": "31"} +{"volume": 2227, "symbol": "I", "ts": "2018-08-31 10:32:00", "month": "08", "high": 21.91, "low": 21.91, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.91, "open": 21.91, "day": "31"} +{"volume": 2455, "symbol": "I", "ts": "2018-08-31 10:33:00", "month": "08", "high": 21.85, "low": 21.85, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.85, "open": 21.85, "day": "31"} +{"volume": 1536, "symbol": "I", "ts": "2018-08-31 10:36:00", "month": "08", "high": 21.8852, "low": 21.8852, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.8852, "open": 21.8852, "day": "31"} +{"volume": 4006, "symbol": "I", "ts": "2018-08-31 10:37:00", "month": "08", "high": 21.87, "low": 21.87, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.87, "open": 21.87, "day": "31"} +{"volume": 302, "symbol": "I", "ts": "2018-08-31 10:39:00", "month": "08", "high": 21.8999, "low": 21.8999, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.8999, "open": 21.8999, "day": "31"} +{"volume": 2401, "symbol": "I", "ts": "2018-08-31 10:40:00", "month": "08", "high": 21.88, "low": 21.88, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.88, "open": 21.88, "day": "31"} +{"volume": 2380, "symbol": "I", "ts": "2018-08-31 10:42:00", "month": "08", "high": 21.88, "low": 21.875, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.88, "open": 21.875, "day": "31"} +{"volume": 1940, "symbol": "I", "ts": "2018-08-31 10:45:00", "month": "08", "high": 21.89, "low": 21.89, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.89, "open": 21.89, "day": "31"} +{"volume": 3072, "symbol": "I", "ts": "2018-08-31 10:46:00", "month": "08", "high": 21.89, "low": 21.89, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.89, "open": 21.89, "day": "31"} +{"volume": 1005, "symbol": "I", "ts": "2018-08-31 10:49:00", "month": "08", "high": 21.87, "low": 21.87, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.87, "open": 21.87, "day": "31"} +{"volume": 1934, "symbol": "I", "ts": "2018-08-31 10:51:00", "month": "08", "high": 21.88, "low": 21.88, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.88, "open": 21.88, "day": "31"} +{"volume": 1154, "symbol": "I", "ts": "2018-08-31 10:53:00", "month": "08", "high": 21.88, "low": 21.88, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.88, "open": 21.88, "day": "31"} +{"volume": 3145, "symbol": "I", "ts": "2018-08-31 10:55:00", "month": "08", "high": 21.845, "low": 21.845, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.845, "open": 21.845, "day": "31"} +{"volume": 3068, "symbol": "I", "ts": "2018-08-31 10:56:00", "month": "08", "high": 21.87, "low": 21.87, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.87, "open": 21.87, "day": "31"} +{"volume": 2003, "symbol": "I", "ts": "2018-08-31 10:57:00", "month": "08", "high": 21.87, "low": 21.87, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.87, "open": 21.87, "day": "31"} +{"volume": 15054, "symbol": "I", "ts": "2018-08-31 10:58:00", "month": "08", "high": 21.9, "low": 21.87, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.9, "open": 21.87, "day": "31"} +{"volume": 107, "symbol": "I", "ts": "2018-08-31 10:59:00", "month": "08", "high": 21.88, "low": 21.88, "key": "I_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.88, "open": 21.88, "day": "31"} +{"volume": 2750, "symbol": "ARQL", "ts": "2018-08-31 10:31:00", "month": "08", "high": 6.6516, "low": 6.6516, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6516, "open": 6.6516, "day": "31"} +{"volume": 100, "symbol": "ARQL", "ts": "2018-08-31 10:33:00", "month": "08", "high": 6.6573, "low": 6.6573, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6573, "open": 6.6573, "day": "31"} +{"volume": 7323, "symbol": "ARQL", "ts": "2018-08-31 10:34:00", "month": "08", "high": 6.62, "low": 6.62, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.62, "open": 6.62, "day": "31"} +{"volume": 3013, "symbol": "ARQL", "ts": "2018-08-31 10:36:00", "month": "08", "high": 6.62, "low": 6.62, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.62, "open": 6.62, "day": "31"} +{"volume": 331, "symbol": "ARQL", "ts": "2018-08-31 10:37:00", "month": "08", "high": 6.61, "low": 6.61, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.61, "open": 6.61, "day": "31"} +{"volume": 1386, "symbol": "ARQL", "ts": "2018-08-31 10:39:00", "month": "08", "high": 6.62, "low": 6.62, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.62, "open": 6.62, "day": "31"} +{"volume": 1010, "symbol": "ARQL", "ts": "2018-08-31 10:41:00", "month": "08", "high": 6.61, "low": 6.61, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.61, "open": 6.61, "day": "31"} +{"volume": 7072, "symbol": "ARQL", "ts": "2018-08-31 10:42:00", "month": "08", "high": 6.59, "low": 6.59, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.59, "open": 6.59, "day": "31"} +{"volume": 8523, "symbol": "ARQL", "ts": "2018-08-31 10:44:00", "month": "08", "high": 6.62, "low": 6.62, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.62, "open": 6.62, "day": "31"} +{"volume": 2000, "symbol": "ARQL", "ts": "2018-08-31 10:46:00", "month": "08", "high": 6.61, "low": 6.61, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.61, "open": 6.61, "day": "31"} +{"volume": 2988, "symbol": "ARQL", "ts": "2018-08-31 10:47:00", "month": "08", "high": 6.5766, "low": 6.5766, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.5766, "open": 6.5766, "day": "31"} +{"volume": 2210, "symbol": "ARQL", "ts": "2018-08-31 10:50:00", "month": "08", "high": 6.6, "low": 6.6, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6, "open": 6.6, "day": "31"} +{"volume": 1860, "symbol": "ARQL", "ts": "2018-08-31 10:52:00", "month": "08", "high": 6.599, "low": 6.599, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.599, "open": 6.599, "day": "31"} +{"volume": 115, "symbol": "ARQL", "ts": "2018-08-31 10:54:00", "month": "08", "high": 6.59, "low": 6.59, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.59, "open": 6.59, "day": "31"} +{"volume": 3500, "symbol": "ARQL", "ts": "2018-08-31 10:57:00", "month": "08", "high": 6.59, "low": 6.59, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.59, "open": 6.59, "day": "31"} +{"volume": 5405, "symbol": "ARQL", "ts": "2018-08-31 10:58:00", "month": "08", "high": 6.6, "low": 6.6, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.6, "open": 6.6, "day": "31"} +{"volume": 2578, "symbol": "ARQL", "ts": "2018-08-31 10:59:00", "month": "08", "high": 6.63, "low": 6.63, "key": "ARQL_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.63, "open": 6.63, "day": "31"} +{"volume": 5003, "symbol": "DNR", "ts": "2018-08-31 10:31:00", "month": "08", "high": 5.49, "low": 5.49, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.49, "open": 5.49, "day": "31"} +{"volume": 15966, "symbol": "DNR", "ts": "2018-08-31 10:32:00", "month": "08", "high": 5.495, "low": 5.49, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.495, "open": 5.491, "day": "31"} +{"volume": 67252, "symbol": "DNR", "ts": "2018-08-31 10:33:00", "month": "08", "high": 5.52, "low": 5.495, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.52, "open": 5.495, "day": "31"} +{"volume": 36885, "symbol": "DNR", "ts": "2018-08-31 10:34:00", "month": "08", "high": 5.5165, "low": 5.51, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.51, "open": 5.515, "day": "31"} +{"volume": 2813, "symbol": "DNR", "ts": "2018-08-31 10:35:00", "month": "08", "high": 5.515, "low": 5.515, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.515, "open": 5.515, "day": "31"} +{"volume": 15690, "symbol": "DNR", "ts": "2018-08-31 10:36:00", "month": "08", "high": 5.525, "low": 5.515, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.525, "open": 5.515, "day": "31"} +{"volume": 52022, "symbol": "DNR", "ts": "2018-08-31 10:37:00", "month": "08", "high": 5.54, "low": 5.525, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.54, "open": 5.525, "day": "31"} +{"volume": 88041, "symbol": "DNR", "ts": "2018-08-31 10:38:00", "month": "08", "high": 5.55, "low": 5.535, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.55, "open": 5.54, "day": "31"} +{"volume": 4238, "symbol": "DNR", "ts": "2018-08-31 10:39:00", "month": "08", "high": 5.545, "low": 5.545, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.545, "open": 5.545, "day": "31"} +{"volume": 49392, "symbol": "DNR", "ts": "2018-08-31 10:40:00", "month": "08", "high": 5.56, "low": 5.55, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.56, "open": 5.55, "day": "31"} +{"volume": 17938, "symbol": "DNR", "ts": "2018-08-31 10:41:00", "month": "08", "high": 5.56, "low": 5.56, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.56, "open": 5.56, "day": "31"} +{"volume": 4051, "symbol": "DNR", "ts": "2018-08-31 10:42:00", "month": "08", "high": 5.56, "low": 5.56, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.56, "open": 5.56, "day": "31"} +{"volume": 34492, "symbol": "DNR", "ts": "2018-08-31 10:43:00", "month": "08", "high": 5.55, "low": 5.55, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.55, "open": 5.55, "day": "31"} +{"volume": 302940, "symbol": "DNR", "ts": "2018-08-31 10:44:00", "month": "08", "high": 5.57, "low": 5.56, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.565, "open": 5.56, "day": "31"} +{"volume": 54193, "symbol": "DNR", "ts": "2018-08-31 10:45:00", "month": "08", "high": 5.5699, "low": 5.545, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.545, "open": 5.5699, "day": "31"} +{"volume": 19030, "symbol": "DNR", "ts": "2018-08-31 10:46:00", "month": "08", "high": 5.54, "low": 5.54, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.54, "open": 5.54, "day": "31"} +{"volume": 46354, "symbol": "DNR", "ts": "2018-08-31 10:47:00", "month": "08", "high": 5.52, "low": 5.52, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.52, "open": 5.52, "day": "31"} +{"volume": 23462, "symbol": "DNR", "ts": "2018-08-31 10:48:00", "month": "08", "high": 5.515, "low": 5.515, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.515, "open": 5.515, "day": "31"} +{"volume": 5729, "symbol": "DNR", "ts": "2018-08-31 10:49:00", "month": "08", "high": 5.51, "low": 5.51, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.51, "open": 5.51, "day": "31"} +{"volume": 14752, "symbol": "DNR", "ts": "2018-08-31 10:50:00", "month": "08", "high": 5.52, "low": 5.51, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.52, "open": 5.51, "day": "31"} +{"volume": 16916, "symbol": "DNR", "ts": "2018-08-31 10:51:00", "month": "08", "high": 5.5, "low": 5.5, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.5, "open": 5.5, "day": "31"} +{"volume": 65407, "symbol": "DNR", "ts": "2018-08-31 10:52:00", "month": "08", "high": 5.5, "low": 5.4842, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.4842, "open": 5.5, "day": "31"} +{"volume": 2980, "symbol": "DNR", "ts": "2018-08-31 10:53:00", "month": "08", "high": 5.48, "low": 5.48, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.48, "open": 5.48, "day": "31"} +{"volume": 4547, "symbol": "DNR", "ts": "2018-08-31 10:54:00", "month": "08", "high": 5.4823, "low": 5.4823, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.4823, "open": 5.4823, "day": "31"} +{"volume": 27774, "symbol": "DNR", "ts": "2018-08-31 10:55:00", "month": "08", "high": 5.47, "low": 5.47, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.47, "open": 5.47, "day": "31"} +{"volume": 17150, "symbol": "DNR", "ts": "2018-08-31 10:56:00", "month": "08", "high": 5.47, "low": 5.47, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.47, "open": 5.47, "day": "31"} +{"volume": 9847, "symbol": "DNR", "ts": "2018-08-31 10:57:00", "month": "08", "high": 5.48, "low": 5.48, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.48, "open": 5.48, "day": "31"} +{"volume": 37155, "symbol": "DNR", "ts": "2018-08-31 10:58:00", "month": "08", "high": 5.48, "low": 5.47, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.47, "open": 5.48, "day": "31"} +{"volume": 26572, "symbol": "DNR", "ts": "2018-08-31 10:59:00", "month": "08", "high": 5.47, "low": 5.46, "key": "DNR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.46, "open": 5.47, "day": "31"} +{"volume": 1152, "symbol": "CVNA", "ts": "2018-08-31 10:31:00", "month": "08", "high": 62.1, "low": 62.1, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.1, "open": 62.1, "day": "31"} +{"volume": 1357, "symbol": "CVNA", "ts": "2018-08-31 10:34:00", "month": "08", "high": 61.87, "low": 61.87, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.87, "open": 61.87, "day": "31"} +{"volume": 1310, "symbol": "CVNA", "ts": "2018-08-31 10:36:00", "month": "08", "high": 61.8, "low": 61.8, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.8, "open": 61.8, "day": "31"} +{"volume": 2849, "symbol": "CVNA", "ts": "2018-08-31 10:38:00", "month": "08", "high": 62.2399, "low": 61.865, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.2399, "open": 61.865, "day": "31"} +{"volume": 350, "symbol": "CVNA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 62.12, "low": 62.12, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.12, "open": 62.12, "day": "31"} +{"volume": 1410, "symbol": "CVNA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 62.23, "low": 62.23, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.23, "open": 62.23, "day": "31"} +{"volume": 1576, "symbol": "CVNA", "ts": "2018-08-31 10:44:00", "month": "08", "high": 62.1875, "low": 62.1875, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.1875, "open": 62.1875, "day": "31"} +{"volume": 519, "symbol": "CVNA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 62.195, "low": 62.195, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.195, "open": 62.195, "day": "31"} +{"volume": 212, "symbol": "CVNA", "ts": "2018-08-31 10:49:00", "month": "08", "high": 62.21, "low": 62.21, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.21, "open": 62.21, "day": "31"} +{"volume": 2905, "symbol": "CVNA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 62.095, "low": 62.095, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.095, "open": 62.095, "day": "31"} +{"volume": 5207, "symbol": "CVNA", "ts": "2018-08-31 10:51:00", "month": "08", "high": 62.21, "low": 62.11, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.21, "open": 62.11, "day": "31"} +{"volume": 983, "symbol": "CVNA", "ts": "2018-08-31 10:53:00", "month": "08", "high": 62.17, "low": 62.17, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.17, "open": 62.17, "day": "31"} +{"volume": 204, "symbol": "CVNA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 62.19, "low": 62.19, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.19, "open": 62.19, "day": "31"} +{"volume": 1496, "symbol": "CVNA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 62.34, "low": 62.34, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.34, "open": 62.34, "day": "31"} +{"volume": 2699, "symbol": "CVNA", "ts": "2018-08-31 10:57:00", "month": "08", "high": 62.35, "low": 62.35, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.35, "open": 62.35, "day": "31"} +{"volume": 5331, "symbol": "CVNA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 62.32, "low": 62.21, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.21, "open": 62.32, "day": "31"} +{"volume": 1586, "symbol": "CVNA", "ts": "2018-08-31 10:59:00", "month": "08", "high": 62.14, "low": 62.14, "key": "CVNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 62.14, "open": 62.14, "day": "31"} +{"volume": 1350, "symbol": "BOOT", "ts": "2018-08-31 10:32:00", "month": "08", "high": 29.5, "low": 29.5, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.5, "open": 29.5, "day": "31"} +{"volume": 2455, "symbol": "BOOT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 29.4531, "low": 29.4531, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.4531, "open": 29.4531, "day": "31"} +{"volume": 2389, "symbol": "BOOT", "ts": "2018-08-31 10:35:00", "month": "08", "high": 29.45, "low": 29.45, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.45, "open": 29.45, "day": "31"} +{"volume": 1093, "symbol": "BOOT", "ts": "2018-08-31 10:36:00", "month": "08", "high": 29.38, "low": 29.38, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.38, "open": 29.38, "day": "31"} +{"volume": 1130, "symbol": "BOOT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 29.41, "low": 29.41, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.41, "open": 29.41, "day": "31"} +{"volume": 2506, "symbol": "BOOT", "ts": "2018-08-31 10:38:00", "month": "08", "high": 29.39, "low": 29.39, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.39, "open": 29.39, "day": "31"} +{"volume": 1933, "symbol": "BOOT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 29.3777, "low": 29.3777, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.3777, "open": 29.3777, "day": "31"} +{"volume": 295, "symbol": "BOOT", "ts": "2018-08-31 10:41:00", "month": "08", "high": 29.35, "low": 29.35, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.35, "open": 29.35, "day": "31"} +{"volume": 1649, "symbol": "BOOT", "ts": "2018-08-31 10:42:00", "month": "08", "high": 29.38, "low": 29.38, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.38, "open": 29.38, "day": "31"} +{"volume": 1129, "symbol": "BOOT", "ts": "2018-08-31 10:44:00", "month": "08", "high": 29.35, "low": 29.35, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.35, "open": 29.35, "day": "31"} +{"volume": 4601, "symbol": "BOOT", "ts": "2018-08-31 10:46:00", "month": "08", "high": 29.37, "low": 29.35, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.35, "open": 29.37, "day": "31"} +{"volume": 1450, "symbol": "BOOT", "ts": "2018-08-31 10:48:00", "month": "08", "high": 29.37, "low": 29.36, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.37, "open": 29.36, "day": "31"} +{"volume": 1351, "symbol": "BOOT", "ts": "2018-08-31 10:51:00", "month": "08", "high": 29.33, "low": 29.33, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.33, "open": 29.33, "day": "31"} +{"volume": 1836, "symbol": "BOOT", "ts": "2018-08-31 10:54:00", "month": "08", "high": 29.26, "low": 29.26, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.26, "open": 29.26, "day": "31"} +{"volume": 276, "symbol": "BOOT", "ts": "2018-08-31 10:55:00", "month": "08", "high": 29.26, "low": 29.26, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.26, "open": 29.26, "day": "31"} +{"volume": 2015, "symbol": "BOOT", "ts": "2018-08-31 10:56:00", "month": "08", "high": 29.32, "low": 29.31, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.32, "open": 29.31, "day": "31"} +{"volume": 400, "symbol": "BOOT", "ts": "2018-08-31 10:58:00", "month": "08", "high": 29.3, "low": 29.3, "key": "BOOT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.3, "open": 29.3, "day": "31"} +{"volume": 29813, "symbol": "SQ", "ts": "2018-08-31 10:31:00", "month": "08", "high": 88.3454, "low": 88.26, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.32, "open": 88.26, "day": "31"} +{"volume": 33241, "symbol": "SQ", "ts": "2018-08-31 10:32:00", "month": "08", "high": 88.25, "low": 88.12, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.215, "open": 88.25, "day": "31"} +{"volume": 10163, "symbol": "SQ", "ts": "2018-08-31 10:33:00", "month": "08", "high": 88.2, "low": 88.19, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.2, "open": 88.19, "day": "31"} +{"volume": 13449, "symbol": "SQ", "ts": "2018-08-31 10:34:00", "month": "08", "high": 88.2, "low": 88.08, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.08, "open": 88.2, "day": "31"} +{"volume": 13577, "symbol": "SQ", "ts": "2018-08-31 10:35:00", "month": "08", "high": 88.12, "low": 88.06, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.12, "open": 88.06, "day": "31"} +{"volume": 10022, "symbol": "SQ", "ts": "2018-08-31 10:36:00", "month": "08", "high": 88.16, "low": 88.095, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.16, "open": 88.095, "day": "31"} +{"volume": 20838, "symbol": "SQ", "ts": "2018-08-31 10:37:00", "month": "08", "high": 88.27, "low": 88.15, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.25, "open": 88.15, "day": "31"} +{"volume": 10683, "symbol": "SQ", "ts": "2018-08-31 10:38:00", "month": "08", "high": 88.29, "low": 88.24, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.24, "open": 88.29, "day": "31"} +{"volume": 18055, "symbol": "SQ", "ts": "2018-08-31 10:39:00", "month": "08", "high": 88.18, "low": 88.12, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.18, "open": 88.14, "day": "31"} +{"volume": 29571, "symbol": "SQ", "ts": "2018-08-31 10:40:00", "month": "08", "high": 88.36, "low": 88.21, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.36, "open": 88.21, "day": "31"} +{"volume": 40789, "symbol": "SQ", "ts": "2018-08-31 10:41:00", "month": "08", "high": 88.465, "low": 88.399, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.465, "open": 88.399, "day": "31"} +{"volume": 34077, "symbol": "SQ", "ts": "2018-08-31 10:42:00", "month": "08", "high": 88.47, "low": 88.4427, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.45, "open": 88.47, "day": "31"} +{"volume": 37198, "symbol": "SQ", "ts": "2018-08-31 10:43:00", "month": "08", "high": 88.6, "low": 88.42, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.6, "open": 88.42, "day": "31"} +{"volume": 27527, "symbol": "SQ", "ts": "2018-08-31 10:44:00", "month": "08", "high": 88.67, "low": 88.57, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.64, "open": 88.64, "day": "31"} +{"volume": 31863, "symbol": "SQ", "ts": "2018-08-31 10:45:00", "month": "08", "high": 88.71, "low": 88.649, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.71, "open": 88.649, "day": "31"} +{"volume": 36138, "symbol": "SQ", "ts": "2018-08-31 10:46:00", "month": "08", "high": 88.7999, "low": 88.699, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.699, "open": 88.765, "day": "31"} +{"volume": 40159, "symbol": "SQ", "ts": "2018-08-31 10:47:00", "month": "08", "high": 88.65, "low": 88.421, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.4572, "open": 88.65, "day": "31"} +{"volume": 19873, "symbol": "SQ", "ts": "2018-08-31 10:48:00", "month": "08", "high": 88.49, "low": 88.4023, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.45, "open": 88.49, "day": "31"} +{"volume": 13558, "symbol": "SQ", "ts": "2018-08-31 10:49:00", "month": "08", "high": 88.6, "low": 88.49, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.6, "open": 88.49, "day": "31"} +{"volume": 26174, "symbol": "SQ", "ts": "2018-08-31 10:50:00", "month": "08", "high": 88.66, "low": 88.56, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.56, "open": 88.66, "day": "31"} +{"volume": 26280, "symbol": "SQ", "ts": "2018-08-31 10:51:00", "month": "08", "high": 88.62, "low": 88.47, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.62, "open": 88.47, "day": "31"} +{"volume": 12664, "symbol": "SQ", "ts": "2018-08-31 10:52:00", "month": "08", "high": 88.6724, "low": 88.56, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.56, "open": 88.6724, "day": "31"} +{"volume": 12703, "symbol": "SQ", "ts": "2018-08-31 10:53:00", "month": "08", "high": 88.56, "low": 88.47, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.5161, "open": 88.56, "day": "31"} +{"volume": 20884, "symbol": "SQ", "ts": "2018-08-31 10:54:00", "month": "08", "high": 88.62, "low": 88.5, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.62, "open": 88.5399, "day": "31"} +{"volume": 44043, "symbol": "SQ", "ts": "2018-08-31 10:55:00", "month": "08", "high": 88.6963, "low": 88.62, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.62, "open": 88.67, "day": "31"} +{"volume": 23833, "symbol": "SQ", "ts": "2018-08-31 10:56:00", "month": "08", "high": 88.6586, "low": 88.62, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.6586, "open": 88.65, "day": "31"} +{"volume": 12413, "symbol": "SQ", "ts": "2018-08-31 10:57:00", "month": "08", "high": 88.6906, "low": 88.68, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.6906, "open": 88.68, "day": "31"} +{"volume": 29745, "symbol": "SQ", "ts": "2018-08-31 10:58:00", "month": "08", "high": 88.7, "low": 88.6653, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.6653, "open": 88.7, "day": "31"} +{"volume": 16748, "symbol": "SQ", "ts": "2018-08-31 10:59:00", "month": "08", "high": 88.66, "low": 88.6, "key": "SQ_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 88.62, "open": 88.66, "day": "31"} +{"volume": 1072, "symbol": "EGAN", "ts": "2018-08-31 10:34:00", "month": "08", "high": 14.7687, "low": 14.7687, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.7687, "open": 14.7687, "day": "31"} +{"volume": 2645, "symbol": "EGAN", "ts": "2018-08-31 10:35:00", "month": "08", "high": 14.65, "low": 14.65, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.65, "open": 14.65, "day": "31"} +{"volume": 541, "symbol": "EGAN", "ts": "2018-08-31 10:36:00", "month": "08", "high": 14.6, "low": 14.6, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.6, "open": 14.6, "day": "31"} +{"volume": 111, "symbol": "EGAN", "ts": "2018-08-31 10:39:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 300, "symbol": "EGAN", "ts": "2018-08-31 10:41:00", "month": "08", "high": 14.65, "low": 14.65, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.65, "open": 14.65, "day": "31"} +{"volume": 100, "symbol": "EGAN", "ts": "2018-08-31 10:42:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 594, "symbol": "EGAN", "ts": "2018-08-31 10:44:00", "month": "08", "high": 14.6, "low": 14.6, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.6, "open": 14.6, "day": "31"} +{"volume": 352, "symbol": "EGAN", "ts": "2018-08-31 10:47:00", "month": "08", "high": 14.6124, "low": 14.6124, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.6124, "open": 14.6124, "day": "31"} +{"volume": 252, "symbol": "EGAN", "ts": "2018-08-31 10:48:00", "month": "08", "high": 14.65, "low": 14.65, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.65, "open": 14.65, "day": "31"} +{"volume": 100, "symbol": "EGAN", "ts": "2018-08-31 10:50:00", "month": "08", "high": 14.65, "low": 14.65, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.65, "open": 14.65, "day": "31"} +{"volume": 565, "symbol": "EGAN", "ts": "2018-08-31 10:52:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 500, "symbol": "EGAN", "ts": "2018-08-31 10:54:00", "month": "08", "high": 14.569, "low": 14.569, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.569, "open": 14.569, "day": "31"} +{"volume": 1301, "symbol": "EGAN", "ts": "2018-08-31 10:56:00", "month": "08", "high": 14.55, "low": 14.55, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.55, "open": 14.55, "day": "31"} +{"volume": 300, "symbol": "EGAN", "ts": "2018-08-31 10:57:00", "month": "08", "high": 14.65, "low": 14.65, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.65, "open": 14.65, "day": "31"} +{"volume": 2944, "symbol": "EGAN", "ts": "2018-08-31 10:59:00", "month": "08", "high": 14.7, "low": 14.6, "key": "EGAN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.6, "open": 14.7, "day": "31"} +{"volume": 241, "symbol": "RCKT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 23.31, "low": 23.31, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.31, "open": 23.31, "day": "31"} +{"volume": 748, "symbol": "RCKT", "ts": "2018-08-31 10:43:00", "month": "08", "high": 23.4, "low": 23.4, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.4, "open": 23.4, "day": "31"} +{"volume": 201, "symbol": "RCKT", "ts": "2018-08-31 10:50:00", "month": "08", "high": 23.33, "low": 23.33, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.33, "open": 23.33, "day": "31"} +{"volume": 102, "symbol": "RCKT", "ts": "2018-08-31 10:52:00", "month": "08", "high": 23.48, "low": 23.48, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.48, "open": 23.48, "day": "31"} +{"volume": 424, "symbol": "RCKT", "ts": "2018-08-31 10:56:00", "month": "08", "high": 23.4, "low": 23.4, "key": "RCKT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.4, "open": 23.4, "day": "31"} +{"volume": 217, "symbol": "NGVC", "ts": "2018-08-31 10:33:00", "month": "08", "high": 18.9, "low": 18.9, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.9, "open": 18.9, "day": "31"} +{"volume": 117, "symbol": "NGVC", "ts": "2018-08-31 10:35:00", "month": "08", "high": 18.96, "low": 18.96, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.96, "open": 18.96, "day": "31"} +{"volume": 197, "symbol": "NGVC", "ts": "2018-08-31 10:37:00", "month": "08", "high": 19.03, "low": 19.03, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.03, "open": 19.03, "day": "31"} +{"volume": 317, "symbol": "NGVC", "ts": "2018-08-31 10:39:00", "month": "08", "high": 19.04, "low": 19.04, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.04, "open": 19.04, "day": "31"} +{"volume": 178, "symbol": "NGVC", "ts": "2018-08-31 10:41:00", "month": "08", "high": 19.04, "low": 19.04, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.04, "open": 19.04, "day": "31"} +{"volume": 4182, "symbol": "NGVC", "ts": "2018-08-31 10:47:00", "month": "08", "high": 19.04, "low": 19.04, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.04, "open": 19.04, "day": "31"} +{"volume": 1026, "symbol": "NGVC", "ts": "2018-08-31 10:48:00", "month": "08", "high": 19.1, "low": 19.1, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.1, "open": 19.1, "day": "31"} +{"volume": 538, "symbol": "NGVC", "ts": "2018-08-31 10:51:00", "month": "08", "high": 19.16, "low": 19.16, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.16, "open": 19.16, "day": "31"} +{"volume": 859, "symbol": "NGVC", "ts": "2018-08-31 10:55:00", "month": "08", "high": 19.14, "low": 19.14, "key": "NGVC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.14, "open": 19.14, "day": "31"} +{"volume": 615, "symbol": "SHSP", "ts": "2018-08-31 10:38:00", "month": "08", "high": 14.0, "low": 14.0, "key": "SHSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.0, "open": 14.0, "day": "31"} +{"volume": 200, "symbol": "SHSP", "ts": "2018-08-31 10:47:00", "month": "08", "high": 13.93, "low": 13.93, "key": "SHSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.93, "open": 13.93, "day": "31"} +{"volume": 500, "symbol": "SHSP", "ts": "2018-08-31 10:53:00", "month": "08", "high": 13.9041, "low": 13.9041, "key": "SHSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.9041, "open": 13.9041, "day": "31"} +{"volume": 101, "symbol": "SHSP", "ts": "2018-08-31 10:55:00", "month": "08", "high": 13.97, "low": 13.97, "key": "SHSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.97, "open": 13.97, "day": "31"} +{"volume": 701, "symbol": "VNCE", "ts": "2018-08-31 10:34:00", "month": "08", "high": 21.22, "low": 21.22, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.22, "open": 21.22, "day": "31"} +{"volume": 947, "symbol": "VNCE", "ts": "2018-08-31 10:37:00", "month": "08", "high": 21.3, "low": 21.3, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.3, "open": 21.3, "day": "31"} +{"volume": 4434, "symbol": "VNCE", "ts": "2018-08-31 10:39:00", "month": "08", "high": 21.1511, "low": 21.1511, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.1511, "open": 21.1511, "day": "31"} +{"volume": 791, "symbol": "VNCE", "ts": "2018-08-31 10:40:00", "month": "08", "high": 21.17, "low": 21.17, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.17, "open": 21.17, "day": "31"} +{"volume": 691, "symbol": "VNCE", "ts": "2018-08-31 10:41:00", "month": "08", "high": 21.05, "low": 21.05, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.05, "open": 21.05, "day": "31"} +{"volume": 210, "symbol": "VNCE", "ts": "2018-08-31 10:52:00", "month": "08", "high": 21.04, "low": 21.04, "key": "VNCE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 21.04, "open": 21.04, "day": "31"} +{"volume": 8627, "symbol": "CRC", "ts": "2018-08-31 10:31:00", "month": "08", "high": 40.4, "low": 40.38, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.4, "open": 40.38, "day": "31"} +{"volume": 4960, "symbol": "CRC", "ts": "2018-08-31 10:33:00", "month": "08", "high": 40.495, "low": 40.405, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.495, "open": 40.405, "day": "31"} +{"volume": 3405, "symbol": "CRC", "ts": "2018-08-31 10:34:00", "month": "08", "high": 40.49, "low": 40.49, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.49, "open": 40.49, "day": "31"} +{"volume": 9988, "symbol": "CRC", "ts": "2018-08-31 10:35:00", "month": "08", "high": 40.5, "low": 40.4987, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.5, "open": 40.4987, "day": "31"} +{"volume": 2380, "symbol": "CRC", "ts": "2018-08-31 10:36:00", "month": "08", "high": 40.61, "low": 40.61, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.61, "open": 40.61, "day": "31"} +{"volume": 1551, "symbol": "CRC", "ts": "2018-08-31 10:38:00", "month": "08", "high": 40.6, "low": 40.6, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.6, "open": 40.6, "day": "31"} +{"volume": 8155, "symbol": "CRC", "ts": "2018-08-31 10:39:00", "month": "08", "high": 40.71, "low": 40.71, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.71, "open": 40.71, "day": "31"} +{"volume": 3126, "symbol": "CRC", "ts": "2018-08-31 10:40:00", "month": "08", "high": 40.8, "low": 40.8, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.8, "open": 40.8, "day": "31"} +{"volume": 6312, "symbol": "CRC", "ts": "2018-08-31 10:41:00", "month": "08", "high": 40.8, "low": 40.8, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.8, "open": 40.8, "day": "31"} +{"volume": 3260, "symbol": "CRC", "ts": "2018-08-31 10:42:00", "month": "08", "high": 40.8845, "low": 40.8845, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.8845, "open": 40.8845, "day": "31"} +{"volume": 7184, "symbol": "CRC", "ts": "2018-08-31 10:43:00", "month": "08", "high": 40.89, "low": 40.89, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.89, "open": 40.89, "day": "31"} +{"volume": 2595, "symbol": "CRC", "ts": "2018-08-31 10:44:00", "month": "08", "high": 40.8973, "low": 40.8973, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.8973, "open": 40.8973, "day": "31"} +{"volume": 1106, "symbol": "CRC", "ts": "2018-08-31 10:45:00", "month": "08", "high": 40.91, "low": 40.91, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.91, "open": 40.91, "day": "31"} +{"volume": 4860, "symbol": "CRC", "ts": "2018-08-31 10:46:00", "month": "08", "high": 40.789, "low": 40.789, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.789, "open": 40.789, "day": "31"} +{"volume": 701, "symbol": "CRC", "ts": "2018-08-31 10:47:00", "month": "08", "high": 40.6, "low": 40.6, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.6, "open": 40.6, "day": "31"} +{"volume": 1945, "symbol": "CRC", "ts": "2018-08-31 10:49:00", "month": "08", "high": 40.58, "low": 40.58, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.58, "open": 40.58, "day": "31"} +{"volume": 2643, "symbol": "CRC", "ts": "2018-08-31 10:50:00", "month": "08", "high": 40.53, "low": 40.53, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.53, "open": 40.53, "day": "31"} +{"volume": 5138, "symbol": "CRC", "ts": "2018-08-31 10:51:00", "month": "08", "high": 40.54, "low": 40.54, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.54, "open": 40.54, "day": "31"} +{"volume": 5382, "symbol": "CRC", "ts": "2018-08-31 10:52:00", "month": "08", "high": 40.589, "low": 40.589, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.589, "open": 40.589, "day": "31"} +{"volume": 2033, "symbol": "CRC", "ts": "2018-08-31 10:53:00", "month": "08", "high": 40.48, "low": 40.48, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.48, "open": 40.48, "day": "31"} +{"volume": 1324, "symbol": "CRC", "ts": "2018-08-31 10:54:00", "month": "08", "high": 40.59, "low": 40.59, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.59, "open": 40.59, "day": "31"} +{"volume": 7728, "symbol": "CRC", "ts": "2018-08-31 10:56:00", "month": "08", "high": 40.57, "low": 40.55, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.57, "open": 40.55, "day": "31"} +{"volume": 3634, "symbol": "CRC", "ts": "2018-08-31 10:57:00", "month": "08", "high": 40.67, "low": 40.67, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.67, "open": 40.67, "day": "31"} +{"volume": 7877, "symbol": "CRC", "ts": "2018-08-31 10:58:00", "month": "08", "high": 40.76, "low": 40.7203, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.7203, "open": 40.76, "day": "31"} +{"volume": 949, "symbol": "CRC", "ts": "2018-08-31 10:59:00", "month": "08", "high": 40.779, "low": 40.779, "key": "CRC_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.779, "open": 40.779, "day": "31"} +{"volume": 2303, "symbol": "VKTX", "ts": "2018-08-31 10:31:00", "month": "08", "high": 12.58, "low": 12.58, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.58, "open": 12.58, "day": "31"} +{"volume": 5250, "symbol": "VKTX", "ts": "2018-08-31 10:32:00", "month": "08", "high": 12.6, "low": 12.6, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6, "open": 12.6, "day": "31"} +{"volume": 3288, "symbol": "VKTX", "ts": "2018-08-31 10:33:00", "month": "08", "high": 12.6182, "low": 12.6182, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6182, "open": 12.6182, "day": "31"} +{"volume": 1251, "symbol": "VKTX", "ts": "2018-08-31 10:34:00", "month": "08", "high": 12.59, "low": 12.59, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.59, "open": 12.59, "day": "31"} +{"volume": 6869, "symbol": "VKTX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 12.573, "low": 12.573, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.573, "open": 12.573, "day": "31"} +{"volume": 3866, "symbol": "VKTX", "ts": "2018-08-31 10:36:00", "month": "08", "high": 12.5901, "low": 12.5901, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.5901, "open": 12.5901, "day": "31"} +{"volume": 7355, "symbol": "VKTX", "ts": "2018-08-31 10:37:00", "month": "08", "high": 12.58, "low": 12.58, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.58, "open": 12.58, "day": "31"} +{"volume": 1663, "symbol": "VKTX", "ts": "2018-08-31 10:38:00", "month": "08", "high": 12.57, "low": 12.57, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.57, "open": 12.57, "day": "31"} +{"volume": 1205, "symbol": "VKTX", "ts": "2018-08-31 10:39:00", "month": "08", "high": 12.58, "low": 12.58, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.58, "open": 12.58, "day": "31"} +{"volume": 8777, "symbol": "VKTX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 12.6099, "low": 12.6099, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6099, "open": 12.6099, "day": "31"} +{"volume": 2217, "symbol": "VKTX", "ts": "2018-08-31 10:41:00", "month": "08", "high": 12.63, "low": 12.63, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.63, "open": 12.63, "day": "31"} +{"volume": 1613, "symbol": "VKTX", "ts": "2018-08-31 10:42:00", "month": "08", "high": 12.6481, "low": 12.6481, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6481, "open": 12.6481, "day": "31"} +{"volume": 7139, "symbol": "VKTX", "ts": "2018-08-31 10:43:00", "month": "08", "high": 12.66, "low": 12.66, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.66, "open": 12.66, "day": "31"} +{"volume": 6560, "symbol": "VKTX", "ts": "2018-08-31 10:44:00", "month": "08", "high": 12.7, "low": 12.7, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.7, "open": 12.7, "day": "31"} +{"volume": 6645, "symbol": "VKTX", "ts": "2018-08-31 10:45:00", "month": "08", "high": 12.7, "low": 12.66, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.66, "open": 12.7, "day": "31"} +{"volume": 1200, "symbol": "VKTX", "ts": "2018-08-31 10:47:00", "month": "08", "high": 12.67, "low": 12.67, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.67, "open": 12.67, "day": "31"} +{"volume": 3296, "symbol": "VKTX", "ts": "2018-08-31 10:48:00", "month": "08", "high": 12.65, "low": 12.65, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.65, "open": 12.65, "day": "31"} +{"volume": 2179, "symbol": "VKTX", "ts": "2018-08-31 10:49:00", "month": "08", "high": 12.67, "low": 12.67, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.67, "open": 12.67, "day": "31"} +{"volume": 8300, "symbol": "VKTX", "ts": "2018-08-31 10:50:00", "month": "08", "high": 12.65, "low": 12.65, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.65, "open": 12.65, "day": "31"} +{"volume": 3269, "symbol": "VKTX", "ts": "2018-08-31 10:51:00", "month": "08", "high": 12.655, "low": 12.655, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.655, "open": 12.655, "day": "31"} +{"volume": 1100, "symbol": "VKTX", "ts": "2018-08-31 10:52:00", "month": "08", "high": 12.6921, "low": 12.6921, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.6921, "open": 12.6921, "day": "31"} +{"volume": 4055, "symbol": "VKTX", "ts": "2018-08-31 10:53:00", "month": "08", "high": 12.705, "low": 12.705, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.705, "open": 12.705, "day": "31"} +{"volume": 14736, "symbol": "VKTX", "ts": "2018-08-31 10:54:00", "month": "08", "high": 12.789, "low": 12.76, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.789, "open": 12.76, "day": "31"} +{"volume": 4315, "symbol": "VKTX", "ts": "2018-08-31 10:55:00", "month": "08", "high": 12.79, "low": 12.79, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.79, "open": 12.79, "day": "31"} +{"volume": 4646, "symbol": "VKTX", "ts": "2018-08-31 10:56:00", "month": "08", "high": 12.79, "low": 12.79, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.79, "open": 12.79, "day": "31"} +{"volume": 4207, "symbol": "VKTX", "ts": "2018-08-31 10:57:00", "month": "08", "high": 12.77, "low": 12.77, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.77, "open": 12.77, "day": "31"} +{"volume": 10772, "symbol": "VKTX", "ts": "2018-08-31 10:58:00", "month": "08", "high": 12.7933, "low": 12.74, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.74, "open": 12.7933, "day": "31"} +{"volume": 2233, "symbol": "VKTX", "ts": "2018-08-31 10:59:00", "month": "08", "high": 12.73, "low": 12.73, "key": "VKTX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.73, "open": 12.73, "day": "31"} +{"volume": 100, "symbol": "CDXS", "ts": "2018-08-31 10:31:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 3502, "symbol": "CDXS", "ts": "2018-08-31 10:32:00", "month": "08", "high": 17.1968, "low": 17.1968, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1968, "open": 17.1968, "day": "31"} +{"volume": 602, "symbol": "CDXS", "ts": "2018-08-31 10:33:00", "month": "08", "high": 17.175, "low": 17.175, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.175, "open": 17.175, "day": "31"} +{"volume": 1041, "symbol": "CDXS", "ts": "2018-08-31 10:38:00", "month": "08", "high": 17.0501, "low": 17.0501, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.0501, "open": 17.0501, "day": "31"} +{"volume": 131, "symbol": "CDXS", "ts": "2018-08-31 10:40:00", "month": "08", "high": 17.15, "low": 17.15, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.15, "open": 17.15, "day": "31"} +{"volume": 105, "symbol": "CDXS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 17.15, "low": 17.15, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.15, "open": 17.15, "day": "31"} +{"volume": 1739, "symbol": "CDXS", "ts": "2018-08-31 10:44:00", "month": "08", "high": 17.2, "low": 17.15, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.15, "day": "31"} +{"volume": 100, "symbol": "CDXS", "ts": "2018-08-31 10:47:00", "month": "08", "high": 17.2, "low": 17.2, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.2, "day": "31"} +{"volume": 304, "symbol": "CDXS", "ts": "2018-08-31 10:48:00", "month": "08", "high": 17.1446, "low": 17.1446, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1446, "open": 17.1446, "day": "31"} +{"volume": 200, "symbol": "CDXS", "ts": "2018-08-31 10:51:00", "month": "08", "high": 17.1, "low": 17.1, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.1, "open": 17.1, "day": "31"} +{"volume": 137, "symbol": "CDXS", "ts": "2018-08-31 10:52:00", "month": "08", "high": 17.2, "low": 17.2, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.2, "day": "31"} +{"volume": 105, "symbol": "CDXS", "ts": "2018-08-31 10:54:00", "month": "08", "high": 17.2, "low": 17.2, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.2, "day": "31"} +{"volume": 100, "symbol": "CDXS", "ts": "2018-08-31 10:56:00", "month": "08", "high": 17.2, "low": 17.2, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.2, "open": 17.2, "day": "31"} +{"volume": 2140, "symbol": "CDXS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 17.25, "low": 17.25, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.25, "open": 17.25, "day": "31"} +{"volume": 8055, "symbol": "CDXS", "ts": "2018-08-31 10:59:00", "month": "08", "high": 17.25, "low": 17.25, "key": "CDXS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 17.25, "open": 17.25, "day": "31"} +{"volume": 100, "symbol": "LFVN", "ts": "2018-08-31 10:31:00", "month": "08", "high": 12.3, "low": 12.3, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.3, "open": 12.3, "day": "31"} +{"volume": 11956, "symbol": "LFVN", "ts": "2018-08-31 10:33:00", "month": "08", "high": 12.38, "low": 12.38, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.38, "open": 12.38, "day": "31"} +{"volume": 1897, "symbol": "LFVN", "ts": "2018-08-31 10:35:00", "month": "08", "high": 12.37, "low": 12.37, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.37, "open": 12.37, "day": "31"} +{"volume": 1604, "symbol": "LFVN", "ts": "2018-08-31 10:38:00", "month": "08", "high": 12.436, "low": 12.436, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.436, "open": 12.436, "day": "31"} +{"volume": 1200, "symbol": "LFVN", "ts": "2018-08-31 10:40:00", "month": "08", "high": 12.435, "low": 12.435, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.435, "open": 12.435, "day": "31"} +{"volume": 1132, "symbol": "LFVN", "ts": "2018-08-31 10:41:00", "month": "08", "high": 12.4717, "low": 12.4717, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.4717, "open": 12.4717, "day": "31"} +{"volume": 1042, "symbol": "LFVN", "ts": "2018-08-31 10:42:00", "month": "08", "high": 12.43, "low": 12.43, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.43, "open": 12.43, "day": "31"} +{"volume": 1000, "symbol": "LFVN", "ts": "2018-08-31 10:45:00", "month": "08", "high": 12.3727, "low": 12.3727, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.3727, "open": 12.3727, "day": "31"} +{"volume": 619, "symbol": "LFVN", "ts": "2018-08-31 10:47:00", "month": "08", "high": 12.38, "low": 12.38, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.38, "open": 12.38, "day": "31"} +{"volume": 1460, "symbol": "LFVN", "ts": "2018-08-31 10:53:00", "month": "08", "high": 12.4, "low": 12.4, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.4, "open": 12.4, "day": "31"} +{"volume": 707, "symbol": "LFVN", "ts": "2018-08-31 10:54:00", "month": "08", "high": 12.4, "low": 12.4, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.4, "open": 12.4, "day": "31"} +{"volume": 3965, "symbol": "LFVN", "ts": "2018-08-31 10:55:00", "month": "08", "high": 12.45, "low": 12.45, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.45, "open": 12.45, "day": "31"} +{"volume": 1709, "symbol": "LFVN", "ts": "2018-08-31 10:56:00", "month": "08", "high": 12.465, "low": 12.465, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.465, "open": 12.465, "day": "31"} +{"volume": 1635, "symbol": "LFVN", "ts": "2018-08-31 10:58:00", "month": "08", "high": 12.4272, "low": 12.4272, "key": "LFVN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 12.4272, "open": 12.4272, "day": "31"} +{"volume": 1226, "symbol": "SRDX", "ts": "2018-08-31 10:32:00", "month": "08", "high": 77.75, "low": 77.75, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.75, "open": 77.75, "day": "31"} +{"volume": 3168, "symbol": "SRDX", "ts": "2018-08-31 10:33:00", "month": "08", "high": 77.8, "low": 77.695, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.695, "open": 77.8, "day": "31"} +{"volume": 3640, "symbol": "SRDX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 77.5, "low": 77.5, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.5, "open": 77.5, "day": "31"} +{"volume": 1754, "symbol": "SRDX", "ts": "2018-08-31 10:38:00", "month": "08", "high": 77.47, "low": 77.47, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.47, "open": 77.47, "day": "31"} +{"volume": 1229, "symbol": "SRDX", "ts": "2018-08-31 10:39:00", "month": "08", "high": 77.5, "low": 77.5, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.5, "open": 77.5, "day": "31"} +{"volume": 1754, "symbol": "SRDX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 77.623, "low": 77.623, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 77.623, "open": 77.623, "day": "31"} +{"volume": 1700, "symbol": "SRDX", "ts": "2018-08-31 10:42:00", "month": "08", "high": 78.1, "low": 78.1, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.1, "open": 78.1, "day": "31"} +{"volume": 2360, "symbol": "SRDX", "ts": "2018-08-31 10:43:00", "month": "08", "high": 78.475, "low": 78.475, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.475, "open": 78.475, "day": "31"} +{"volume": 1827, "symbol": "SRDX", "ts": "2018-08-31 10:44:00", "month": "08", "high": 78.25, "low": 78.25, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.25, "open": 78.25, "day": "31"} +{"volume": 150, "symbol": "SRDX", "ts": "2018-08-31 10:45:00", "month": "08", "high": 78.4, "low": 78.4, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.4, "open": 78.4, "day": "31"} +{"volume": 904, "symbol": "SRDX", "ts": "2018-08-31 10:48:00", "month": "08", "high": 78.75, "low": 78.75, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.75, "open": 78.75, "day": "31"} +{"volume": 753, "symbol": "SRDX", "ts": "2018-08-31 10:49:00", "month": "08", "high": 78.65, "low": 78.65, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.65, "open": 78.65, "day": "31"} +{"volume": 2094, "symbol": "SRDX", "ts": "2018-08-31 10:51:00", "month": "08", "high": 78.85, "low": 78.85, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.85, "open": 78.85, "day": "31"} +{"volume": 1095, "symbol": "SRDX", "ts": "2018-08-31 10:52:00", "month": "08", "high": 78.5, "low": 78.5, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.5, "open": 78.5, "day": "31"} +{"volume": 363, "symbol": "SRDX", "ts": "2018-08-31 10:53:00", "month": "08", "high": 78.5, "low": 78.5, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.5, "open": 78.5, "day": "31"} +{"volume": 1847, "symbol": "SRDX", "ts": "2018-08-31 10:55:00", "month": "08", "high": 78.55, "low": 78.55, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.55, "open": 78.55, "day": "31"} +{"volume": 1399, "symbol": "SRDX", "ts": "2018-08-31 10:57:00", "month": "08", "high": 78.7, "low": 78.7, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.7, "open": 78.7, "day": "31"} +{"volume": 708, "symbol": "SRDX", "ts": "2018-08-31 10:59:00", "month": "08", "high": 78.6, "low": 78.6, "key": "SRDX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 78.6, "open": 78.6, "day": "31"} +{"volume": 200, "symbol": "LGCYP", "ts": "2018-08-31 10:31:00", "month": "08", "high": 15.5775, "low": 15.5775, "key": "LGCYP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.5775, "open": 15.5775, "day": "31"} +{"volume": 5000, "symbol": "SDPI", "ts": "2018-08-31 10:34:00", "month": "08", "high": 1.9, "low": 1.9, "key": "SDPI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1.9, "open": 1.9, "day": "31"} +{"volume": 100, "symbol": "SDPI", "ts": "2018-08-31 10:38:00", "month": "08", "high": 1.9, "low": 1.9, "key": "SDPI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1.9, "open": 1.9, "day": "31"} +{"volume": 5000, "symbol": "SDPI", "ts": "2018-08-31 10:44:00", "month": "08", "high": 1.95, "low": 1.91, "key": "SDPI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1.95, "open": 1.91, "day": "31"} +{"volume": 176, "symbol": "SDPI", "ts": "2018-08-31 10:49:00", "month": "08", "high": 1.9166, "low": 1.9166, "key": "SDPI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 1.9166, "open": 1.9166, "day": "31"} +{"volume": 389, "symbol": "MDB", "ts": "2018-08-31 10:32:00", "month": "08", "high": 71.4506, "low": 71.4506, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.4506, "open": 71.4506, "day": "31"} +{"volume": 1597, "symbol": "MDB", "ts": "2018-08-31 10:35:00", "month": "08", "high": 71.265, "low": 71.265, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.265, "open": 71.265, "day": "31"} +{"volume": 3265, "symbol": "MDB", "ts": "2018-08-31 10:36:00", "month": "08", "high": 71.07, "low": 71.07, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.07, "open": 71.07, "day": "31"} +{"volume": 3259, "symbol": "MDB", "ts": "2018-08-31 10:38:00", "month": "08", "high": 71.145, "low": 71.12, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.145, "open": 71.12, "day": "31"} +{"volume": 1146, "symbol": "MDB", "ts": "2018-08-31 10:39:00", "month": "08", "high": 71.23, "low": 71.23, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.23, "open": 71.23, "day": "31"} +{"volume": 1228, "symbol": "MDB", "ts": "2018-08-31 10:42:00", "month": "08", "high": 71.43, "low": 71.43, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.43, "open": 71.43, "day": "31"} +{"volume": 412, "symbol": "MDB", "ts": "2018-08-31 10:43:00", "month": "08", "high": 71.4356, "low": 71.4356, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.4356, "open": 71.4356, "day": "31"} +{"volume": 402, "symbol": "MDB", "ts": "2018-08-31 10:46:00", "month": "08", "high": 71.34, "low": 71.34, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.34, "open": 71.34, "day": "31"} +{"volume": 1424, "symbol": "MDB", "ts": "2018-08-31 10:47:00", "month": "08", "high": 71.2786, "low": 71.2786, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.2786, "open": 71.2786, "day": "31"} +{"volume": 1036, "symbol": "MDB", "ts": "2018-08-31 10:49:00", "month": "08", "high": 71.215, "low": 71.215, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.215, "open": 71.215, "day": "31"} +{"volume": 2072, "symbol": "MDB", "ts": "2018-08-31 10:50:00", "month": "08", "high": 71.1332, "low": 71.11, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.11, "open": 71.1332, "day": "31"} +{"volume": 1286, "symbol": "MDB", "ts": "2018-08-31 10:52:00", "month": "08", "high": 71.02, "low": 71.02, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.02, "open": 71.02, "day": "31"} +{"volume": 2535, "symbol": "MDB", "ts": "2018-08-31 10:54:00", "month": "08", "high": 70.9001, "low": 70.9001, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.9001, "open": 70.9001, "day": "31"} +{"volume": 1353, "symbol": "MDB", "ts": "2018-08-31 10:56:00", "month": "08", "high": 70.91, "low": 70.91, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.91, "open": 70.91, "day": "31"} +{"volume": 1432, "symbol": "MDB", "ts": "2018-08-31 10:57:00", "month": "08", "high": 70.95, "low": 70.95, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.95, "open": 70.95, "day": "31"} +{"volume": 2406, "symbol": "MDB", "ts": "2018-08-31 10:58:00", "month": "08", "high": 71.01, "low": 71.01, "key": "MDB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.01, "open": 71.01, "day": "31"} +{"volume": 1600, "symbol": "LGCY", "ts": "2018-08-31 10:33:00", "month": "08", "high": 5.3713, "low": 5.35, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.35, "open": 5.3713, "day": "31"} +{"volume": 100, "symbol": "LGCY", "ts": "2018-08-31 10:38:00", "month": "08", "high": 5.34, "low": 5.34, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.34, "open": 5.34, "day": "31"} +{"volume": 173, "symbol": "LGCY", "ts": "2018-08-31 10:47:00", "month": "08", "high": 5.35, "low": 5.35, "key": "LGCY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.35, "open": 5.35, "day": "31"} +{"volume": 600, "symbol": "ATTU", "ts": "2018-08-31 10:32:00", "month": "08", "high": 20.52, "low": 20.52, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.52, "open": 20.52, "day": "31"} +{"volume": 241, "symbol": "ATTU", "ts": "2018-08-31 10:34:00", "month": "08", "high": 20.545, "low": 20.545, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.545, "open": 20.545, "day": "31"} +{"volume": 100, "symbol": "ATTU", "ts": "2018-08-31 10:35:00", "month": "08", "high": 20.545, "low": 20.545, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.545, "open": 20.545, "day": "31"} +{"volume": 401, "symbol": "ATTU", "ts": "2018-08-31 10:36:00", "month": "08", "high": 20.545, "low": 20.545, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.545, "open": 20.545, "day": "31"} +{"volume": 2100, "symbol": "ATTU", "ts": "2018-08-31 10:40:00", "month": "08", "high": 20.5954, "low": 20.5954, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.5954, "open": 20.5954, "day": "31"} +{"volume": 316, "symbol": "ATTU", "ts": "2018-08-31 10:42:00", "month": "08", "high": 20.61, "low": 20.585, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.585, "open": 20.61, "day": "31"} +{"volume": 852, "symbol": "ATTU", "ts": "2018-08-31 10:44:00", "month": "08", "high": 20.575, "low": 20.575, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.575, "open": 20.575, "day": "31"} +{"volume": 851, "symbol": "ATTU", "ts": "2018-08-31 10:47:00", "month": "08", "high": 20.575, "low": 20.575, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.575, "open": 20.575, "day": "31"} +{"volume": 500, "symbol": "ATTU", "ts": "2018-08-31 10:49:00", "month": "08", "high": 20.575, "low": 20.575, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.575, "open": 20.575, "day": "31"} +{"volume": 700, "symbol": "ATTU", "ts": "2018-08-31 10:50:00", "month": "08", "high": 20.565, "low": 20.565, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.565, "open": 20.565, "day": "31"} +{"volume": 303, "symbol": "ATTU", "ts": "2018-08-31 10:51:00", "month": "08", "high": 20.5675, "low": 20.5675, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.5675, "open": 20.5675, "day": "31"} +{"volume": 1852, "symbol": "ATTU", "ts": "2018-08-31 10:53:00", "month": "08", "high": 20.58, "low": 20.58, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.58, "open": 20.58, "day": "31"} +{"volume": 1100, "symbol": "ATTU", "ts": "2018-08-31 10:55:00", "month": "08", "high": 20.58, "low": 20.58, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.58, "open": 20.58, "day": "31"} +{"volume": 100, "symbol": "ATTU", "ts": "2018-08-31 10:57:00", "month": "08", "high": 20.5852, "low": 20.5852, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.5852, "open": 20.5852, "day": "31"} +{"volume": 244, "symbol": "ATTU", "ts": "2018-08-31 10:58:00", "month": "08", "high": 20.5868, "low": 20.5868, "key": "ATTU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 20.5868, "open": 20.5868, "day": "31"} +{"volume": 730, "symbol": "AYX", "ts": "2018-08-31 10:31:00", "month": "08", "high": 56.664, "low": 56.664, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.664, "open": 56.664, "day": "31"} +{"volume": 2358, "symbol": "AYX", "ts": "2018-08-31 10:32:00", "month": "08", "high": 56.6188, "low": 56.6188, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.6188, "open": 56.6188, "day": "31"} +{"volume": 1156, "symbol": "AYX", "ts": "2018-08-31 10:34:00", "month": "08", "high": 56.56, "low": 56.56, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.56, "open": 56.56, "day": "31"} +{"volume": 480, "symbol": "AYX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 56.51, "low": 56.51, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.51, "open": 56.51, "day": "31"} +{"volume": 1301, "symbol": "AYX", "ts": "2018-08-31 10:37:00", "month": "08", "high": 56.69, "low": 56.69, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.69, "open": 56.69, "day": "31"} +{"volume": 1391, "symbol": "AYX", "ts": "2018-08-31 10:38:00", "month": "08", "high": 56.69, "low": 56.69, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.69, "open": 56.69, "day": "31"} +{"volume": 1750, "symbol": "AYX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 56.8, "low": 56.8, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.8, "open": 56.8, "day": "31"} +{"volume": 1144, "symbol": "AYX", "ts": "2018-08-31 10:42:00", "month": "08", "high": 56.94, "low": 56.94, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.94, "open": 56.94, "day": "31"} +{"volume": 524, "symbol": "AYX", "ts": "2018-08-31 10:44:00", "month": "08", "high": 57.21, "low": 57.21, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.21, "open": 57.21, "day": "31"} +{"volume": 934, "symbol": "AYX", "ts": "2018-08-31 10:47:00", "month": "08", "high": 57.08, "low": 57.08, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.08, "open": 57.08, "day": "31"} +{"volume": 1281, "symbol": "AYX", "ts": "2018-08-31 10:49:00", "month": "08", "high": 57.025, "low": 57.025, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.025, "open": 57.025, "day": "31"} +{"volume": 1813, "symbol": "AYX", "ts": "2018-08-31 10:50:00", "month": "08", "high": 56.995, "low": 56.995, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.995, "open": 56.995, "day": "31"} +{"volume": 1621, "symbol": "AYX", "ts": "2018-08-31 10:52:00", "month": "08", "high": 57.02, "low": 57.02, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.02, "open": 57.02, "day": "31"} +{"volume": 270, "symbol": "AYX", "ts": "2018-08-31 10:53:00", "month": "08", "high": 56.98, "low": 56.98, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 56.98, "open": 56.98, "day": "31"} +{"volume": 2079, "symbol": "AYX", "ts": "2018-08-31 10:55:00", "month": "08", "high": 57.095, "low": 57.095, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.095, "open": 57.095, "day": "31"} +{"volume": 751, "symbol": "AYX", "ts": "2018-08-31 10:58:00", "month": "08", "high": 57.1132, "low": 57.1132, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.1132, "open": 57.1132, "day": "31"} +{"volume": 430, "symbol": "AYX", "ts": "2018-08-31 10:59:00", "month": "08", "high": 57.1, "low": 57.1, "key": "AYX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.1, "open": 57.1, "day": "31"} +{"volume": 1001, "symbol": "IRMD", "ts": "2018-08-31 10:32:00", "month": "08", "high": 26.8, "low": 26.8, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.8, "open": 26.8, "day": "31"} +{"volume": 1200, "symbol": "IRMD", "ts": "2018-08-31 10:34:00", "month": "08", "high": 26.875, "low": 26.875, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.875, "open": 26.875, "day": "31"} +{"volume": 800, "symbol": "IRMD", "ts": "2018-08-31 10:36:00", "month": "08", "high": 26.85, "low": 26.85, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.85, "open": 26.85, "day": "31"} +{"volume": 108, "symbol": "IRMD", "ts": "2018-08-31 10:43:00", "month": "08", "high": 26.9, "low": 26.9, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.9, "open": 26.9, "day": "31"} +{"volume": 100, "symbol": "IRMD", "ts": "2018-08-31 10:45:00", "month": "08", "high": 26.9, "low": 26.9, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.9, "open": 26.9, "day": "31"} +{"volume": 862, "symbol": "IRMD", "ts": "2018-08-31 10:47:00", "month": "08", "high": 26.9, "low": 26.9, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.9, "open": 26.9, "day": "31"} +{"volume": 832, "symbol": "IRMD", "ts": "2018-08-31 10:50:00", "month": "08", "high": 26.875, "low": 26.875, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.875, "open": 26.875, "day": "31"} +{"volume": 535, "symbol": "IRMD", "ts": "2018-08-31 10:55:00", "month": "08", "high": 27.0, "low": 27.0, "key": "IRMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.0, "open": 27.0, "day": "31"} +{"volume": 5154, "symbol": "SEAS", "ts": "2018-08-31 10:31:00", "month": "08", "high": 29.12, "low": 29.11, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.11, "open": 29.12, "day": "31"} +{"volume": 2631, "symbol": "SEAS", "ts": "2018-08-31 10:32:00", "month": "08", "high": 29.115, "low": 29.115, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.115, "open": 29.115, "day": "31"} +{"volume": 7178, "symbol": "SEAS", "ts": "2018-08-31 10:33:00", "month": "08", "high": 29.15, "low": 29.15, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.15, "open": 29.15, "day": "31"} +{"volume": 4211, "symbol": "SEAS", "ts": "2018-08-31 10:34:00", "month": "08", "high": 29.1112, "low": 29.1112, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.1112, "open": 29.1112, "day": "31"} +{"volume": 4645, "symbol": "SEAS", "ts": "2018-08-31 10:36:00", "month": "08", "high": 29.12, "low": 29.09, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.12, "open": 29.09, "day": "31"} +{"volume": 9362, "symbol": "SEAS", "ts": "2018-08-31 10:37:00", "month": "08", "high": 29.08, "low": 29.075, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.08, "open": 29.075, "day": "31"} +{"volume": 3409, "symbol": "SEAS", "ts": "2018-08-31 10:38:00", "month": "08", "high": 29.063, "low": 29.063, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.063, "open": 29.063, "day": "31"} +{"volume": 2323, "symbol": "SEAS", "ts": "2018-08-31 10:39:00", "month": "08", "high": 29.03, "low": 29.03, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.03, "open": 29.03, "day": "31"} +{"volume": 3126, "symbol": "SEAS", "ts": "2018-08-31 10:40:00", "month": "08", "high": 28.99, "low": 28.99, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.99, "open": 28.99, "day": "31"} +{"volume": 1701, "symbol": "SEAS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 29.02, "low": 29.02, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.02, "open": 29.02, "day": "31"} +{"volume": 2317, "symbol": "SEAS", "ts": "2018-08-31 10:42:00", "month": "08", "high": 29.0, "low": 29.0, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.0, "open": 29.0, "day": "31"} +{"volume": 1908, "symbol": "SEAS", "ts": "2018-08-31 10:43:00", "month": "08", "high": 28.96, "low": 28.96, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.96, "open": 28.96, "day": "31"} +{"volume": 2233, "symbol": "SEAS", "ts": "2018-08-31 10:44:00", "month": "08", "high": 28.93, "low": 28.93, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.93, "open": 28.93, "day": "31"} +{"volume": 1327, "symbol": "SEAS", "ts": "2018-08-31 10:45:00", "month": "08", "high": 28.92, "low": 28.92, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.92, "open": 28.92, "day": "31"} +{"volume": 3721, "symbol": "SEAS", "ts": "2018-08-31 10:46:00", "month": "08", "high": 28.91, "low": 28.91, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.91, "open": 28.91, "day": "31"} +{"volume": 1528, "symbol": "SEAS", "ts": "2018-08-31 10:47:00", "month": "08", "high": 28.85, "low": 28.85, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.85, "open": 28.85, "day": "31"} +{"volume": 4978, "symbol": "SEAS", "ts": "2018-08-31 10:48:00", "month": "08", "high": 28.845, "low": 28.845, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.845, "open": 28.845, "day": "31"} +{"volume": 16749, "symbol": "SEAS", "ts": "2018-08-31 10:49:00", "month": "08", "high": 28.87, "low": 28.845, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.87, "open": 28.845, "day": "31"} +{"volume": 3423, "symbol": "SEAS", "ts": "2018-08-31 10:50:00", "month": "08", "high": 28.9, "low": 28.9, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.9, "open": 28.9, "day": "31"} +{"volume": 1684, "symbol": "SEAS", "ts": "2018-08-31 10:52:00", "month": "08", "high": 28.9, "low": 28.9, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.9, "open": 28.9, "day": "31"} +{"volume": 6913, "symbol": "SEAS", "ts": "2018-08-31 10:53:00", "month": "08", "high": 28.97, "low": 28.91, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.97, "open": 28.91, "day": "31"} +{"volume": 5862, "symbol": "SEAS", "ts": "2018-08-31 10:55:00", "month": "08", "high": 28.97, "low": 28.97, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 28.97, "open": 28.97, "day": "31"} +{"volume": 7543, "symbol": "SEAS", "ts": "2018-08-31 10:56:00", "month": "08", "high": 29.02, "low": 28.99, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.02, "open": 28.99, "day": "31"} +{"volume": 2000, "symbol": "SEAS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 29.03, "low": 29.03, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.03, "open": 29.03, "day": "31"} +{"volume": 2793, "symbol": "SEAS", "ts": "2018-08-31 10:58:00", "month": "08", "high": 29.03, "low": 29.03, "key": "SEAS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 29.03, "open": 29.03, "day": "31"} +{"volume": 3151, "symbol": "RIBT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 3.2575, "low": 3.25, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.25, "open": 3.2575, "day": "31"} +{"volume": 100, "symbol": "RIBT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 3.27, "low": 3.27, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.27, "open": 3.27, "day": "31"} +{"volume": 650, "symbol": "RIBT", "ts": "2018-08-31 10:41:00", "month": "08", "high": 3.2538, "low": 3.2538, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.2538, "open": 3.2538, "day": "31"} +{"volume": 1100, "symbol": "RIBT", "ts": "2018-08-31 10:47:00", "month": "08", "high": 3.26, "low": 3.26, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.26, "open": 3.26, "day": "31"} +{"volume": 5574, "symbol": "RIBT", "ts": "2018-08-31 10:48:00", "month": "08", "high": 3.22, "low": 3.22, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.22, "open": 3.22, "day": "31"} +{"volume": 300, "symbol": "RIBT", "ts": "2018-08-31 10:55:00", "month": "08", "high": 3.235, "low": 3.235, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.235, "open": 3.235, "day": "31"} +{"volume": 500, "symbol": "RIBT", "ts": "2018-08-31 10:59:00", "month": "08", "high": 3.2329, "low": 3.2329, "key": "RIBT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.2329, "open": 3.2329, "day": "31"} +{"volume": 143, "symbol": "NSP", "ts": "2018-08-31 10:32:00", "month": "08", "high": 117.85, "low": 117.85, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 117.85, "open": 117.85, "day": "31"} +{"volume": 549, "symbol": "NSP", "ts": "2018-08-31 10:35:00", "month": "08", "high": 117.85, "low": 117.85, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 117.85, "open": 117.85, "day": "31"} +{"volume": 371, "symbol": "NSP", "ts": "2018-08-31 10:38:00", "month": "08", "high": 118.0, "low": 118.0, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.0, "open": 118.0, "day": "31"} +{"volume": 1074, "symbol": "NSP", "ts": "2018-08-31 10:41:00", "month": "08", "high": 118.1462, "low": 118.1462, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.1462, "open": 118.1462, "day": "31"} +{"volume": 583, "symbol": "NSP", "ts": "2018-08-31 10:45:00", "month": "08", "high": 118.25, "low": 118.25, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.25, "open": 118.25, "day": "31"} +{"volume": 1044, "symbol": "NSP", "ts": "2018-08-31 10:50:00", "month": "08", "high": 118.3, "low": 118.3, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.3, "open": 118.3, "day": "31"} +{"volume": 649, "symbol": "NSP", "ts": "2018-08-31 10:52:00", "month": "08", "high": 118.275, "low": 118.275, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.275, "open": 118.275, "day": "31"} +{"volume": 1217, "symbol": "NSP", "ts": "2018-08-31 10:54:00", "month": "08", "high": 118.325, "low": 118.25, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.25, "open": 118.325, "day": "31"} +{"volume": 405, "symbol": "NSP", "ts": "2018-08-31 10:56:00", "month": "08", "high": 118.45, "low": 118.45, "key": "NSP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 118.45, "open": 118.45, "day": "31"} +{"volume": 2532, "symbol": "GDS", "ts": "2018-08-31 10:31:00", "month": "08", "high": 38.18, "low": 38.18, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.18, "open": 38.18, "day": "31"} +{"volume": 1832, "symbol": "GDS", "ts": "2018-08-31 10:32:00", "month": "08", "high": 38.04, "low": 38.04, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.04, "open": 38.04, "day": "31"} +{"volume": 1340, "symbol": "GDS", "ts": "2018-08-31 10:34:00", "month": "08", "high": 38.03, "low": 38.03, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.03, "open": 38.03, "day": "31"} +{"volume": 2088, "symbol": "GDS", "ts": "2018-08-31 10:35:00", "month": "08", "high": 37.86, "low": 37.86, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.86, "open": 37.86, "day": "31"} +{"volume": 401, "symbol": "GDS", "ts": "2018-08-31 10:36:00", "month": "08", "high": 37.91, "low": 37.91, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.91, "open": 37.91, "day": "31"} +{"volume": 1536, "symbol": "GDS", "ts": "2018-08-31 10:38:00", "month": "08", "high": 38.02, "low": 38.02, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.02, "open": 38.02, "day": "31"} +{"volume": 1313, "symbol": "GDS", "ts": "2018-08-31 10:39:00", "month": "08", "high": 37.97, "low": 37.97, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.97, "open": 37.97, "day": "31"} +{"volume": 3675, "symbol": "GDS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 38.1, "low": 38.1, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.1, "open": 38.1, "day": "31"} +{"volume": 1772, "symbol": "GDS", "ts": "2018-08-31 10:42:00", "month": "08", "high": 38.09, "low": 38.09, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.09, "open": 38.09, "day": "31"} +{"volume": 432, "symbol": "GDS", "ts": "2018-08-31 10:44:00", "month": "08", "high": 38.1, "low": 38.1, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.1, "open": 38.1, "day": "31"} +{"volume": 1642, "symbol": "GDS", "ts": "2018-08-31 10:45:00", "month": "08", "high": 38.08, "low": 38.08, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.08, "open": 38.08, "day": "31"} +{"volume": 1595, "symbol": "GDS", "ts": "2018-08-31 10:46:00", "month": "08", "high": 38.03, "low": 38.03, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.03, "open": 38.03, "day": "31"} +{"volume": 2040, "symbol": "GDS", "ts": "2018-08-31 10:48:00", "month": "08", "high": 38.06, "low": 38.05, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.06, "open": 38.05, "day": "31"} +{"volume": 2087, "symbol": "GDS", "ts": "2018-08-31 10:53:00", "month": "08", "high": 37.98, "low": 37.97, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 37.97, "open": 37.98, "day": "31"} +{"volume": 510, "symbol": "GDS", "ts": "2018-08-31 10:55:00", "month": "08", "high": 38.03, "low": 38.03, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.03, "open": 38.03, "day": "31"} +{"volume": 2552, "symbol": "GDS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 38.03, "low": 38.03, "key": "GDS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 38.03, "open": 38.03, "day": "31"} +{"volume": 797, "symbol": "VICR", "ts": "2018-08-31 10:32:00", "month": "08", "high": 61.55, "low": 61.55, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 207, "symbol": "VICR", "ts": "2018-08-31 10:34:00", "month": "08", "high": 61.5, "low": 61.5, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.5, "open": 61.5, "day": "31"} +{"volume": 100, "symbol": "VICR", "ts": "2018-08-31 10:37:00", "month": "08", "high": 61.55, "low": 61.55, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 940, "symbol": "VICR", "ts": "2018-08-31 10:38:00", "month": "08", "high": 61.55, "low": 61.55, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 1505, "symbol": "VICR", "ts": "2018-08-31 10:44:00", "month": "08", "high": 61.75, "low": 61.75, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.75, "open": 61.75, "day": "31"} +{"volume": 597, "symbol": "VICR", "ts": "2018-08-31 10:48:00", "month": "08", "high": 61.7, "low": 61.7, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.7, "open": 61.7, "day": "31"} +{"volume": 800, "symbol": "VICR", "ts": "2018-08-31 10:53:00", "month": "08", "high": 61.725, "low": 61.725, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.725, "open": 61.725, "day": "31"} +{"volume": 101, "symbol": "VICR", "ts": "2018-08-31 10:57:00", "month": "08", "high": 61.8, "low": 61.8, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.8, "open": 61.8, "day": "31"} +{"volume": 1144, "symbol": "VICR", "ts": "2018-08-31 10:58:00", "month": "08", "high": 61.7, "low": 61.7, "key": "VICR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.7, "open": 61.7, "day": "31"} +{"volume": 95704, "symbol": "CRON", "ts": "2018-08-31 10:31:00", "month": "08", "high": 9.93, "low": 9.885, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.885, "open": 9.93, "day": "31"} +{"volume": 92567, "symbol": "CRON", "ts": "2018-08-31 10:32:00", "month": "08", "high": 9.93, "low": 9.8501, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.86, "day": "31"} +{"volume": 56265, "symbol": "CRON", "ts": "2018-08-31 10:33:00", "month": "08", "high": 9.9173, "low": 9.89, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9173, "open": 9.9094, "day": "31"} +{"volume": 139587, "symbol": "CRON", "ts": "2018-08-31 10:34:00", "month": "08", "high": 9.97, "low": 9.89, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.97, "open": 9.915, "day": "31"} +{"volume": 96304, "symbol": "CRON", "ts": "2018-08-31 10:35:00", "month": "08", "high": 9.99, "low": 9.95, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.99, "open": 9.955, "day": "31"} +{"volume": 215389, "symbol": "CRON", "ts": "2018-08-31 10:36:00", "month": "08", "high": 10.0699, "low": 9.9901, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0684, "open": 10.0, "day": "31"} +{"volume": 205691, "symbol": "CRON", "ts": "2018-08-31 10:37:00", "month": "08", "high": 10.12, "low": 10.03, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.03, "open": 10.0789, "day": "31"} +{"volume": 125755, "symbol": "CRON", "ts": "2018-08-31 10:38:00", "month": "08", "high": 10.089, "low": 10.0282, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.08, "open": 10.0282, "day": "31"} +{"volume": 80798, "symbol": "CRON", "ts": "2018-08-31 10:39:00", "month": "08", "high": 10.0754, "low": 10.0301, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.04, "open": 10.07, "day": "31"} +{"volume": 122857, "symbol": "CRON", "ts": "2018-08-31 10:40:00", "month": "08", "high": 10.0514, "low": 10.0201, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.03, "open": 10.0358, "day": "31"} +{"volume": 80829, "symbol": "CRON", "ts": "2018-08-31 10:41:00", "month": "08", "high": 10.04, "low": 10.0199, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 10.0389, "open": 10.02, "day": "31"} +{"volume": 210057, "symbol": "CRON", "ts": "2018-08-31 10:42:00", "month": "08", "high": 10.0049, "low": 9.89, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.985, "open": 10.0, "day": "31"} +{"volume": 59357, "symbol": "CRON", "ts": "2018-08-31 10:43:00", "month": "08", "high": 10.0, "low": 9.975, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.985, "open": 10.0, "day": "31"} +{"volume": 46859, "symbol": "CRON", "ts": "2018-08-31 10:44:00", "month": "08", "high": 9.98, "low": 9.95, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9694, "open": 9.97, "day": "31"} +{"volume": 95776, "symbol": "CRON", "ts": "2018-08-31 10:45:00", "month": "08", "high": 10.01, "low": 9.97, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9956, "open": 9.97, "day": "31"} +{"volume": 45235, "symbol": "CRON", "ts": "2018-08-31 10:46:00", "month": "08", "high": 9.99, "low": 9.9725, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.99, "open": 9.975, "day": "31"} +{"volume": 75468, "symbol": "CRON", "ts": "2018-08-31 10:47:00", "month": "08", "high": 9.99, "low": 9.93, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9601, "open": 9.9861, "day": "31"} +{"volume": 40202, "symbol": "CRON", "ts": "2018-08-31 10:48:00", "month": "08", "high": 9.9751, "low": 9.9663, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.967, "open": 9.9663, "day": "31"} +{"volume": 84927, "symbol": "CRON", "ts": "2018-08-31 10:49:00", "month": "08", "high": 10.0, "low": 9.945, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9831, "open": 9.9601, "day": "31"} +{"volume": 35931, "symbol": "CRON", "ts": "2018-08-31 10:50:00", "month": "08", "high": 9.9882, "low": 9.97, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.98, "open": 9.97, "day": "31"} +{"volume": 111523, "symbol": "CRON", "ts": "2018-08-31 10:51:00", "month": "08", "high": 9.95, "low": 9.8811, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.95, "day": "31"} +{"volume": 165709, "symbol": "CRON", "ts": "2018-08-31 10:52:00", "month": "08", "high": 9.9143, "low": 9.83, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.8699, "open": 9.91, "day": "31"} +{"volume": 70910, "symbol": "CRON", "ts": "2018-08-31 10:53:00", "month": "08", "high": 9.9, "low": 9.85, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.85, "day": "31"} +{"volume": 92793, "symbol": "CRON", "ts": "2018-08-31 10:54:00", "month": "08", "high": 9.8849, "low": 9.8501, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.8667, "open": 9.8849, "day": "31"} +{"volume": 118338, "symbol": "CRON", "ts": "2018-08-31 10:55:00", "month": "08", "high": 9.86, "low": 9.8116, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.83, "open": 9.86, "day": "31"} +{"volume": 123562, "symbol": "CRON", "ts": "2018-08-31 10:56:00", "month": "08", "high": 9.8317, "low": 9.7616, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.765, "open": 9.8317, "day": "31"} +{"volume": 326822, "symbol": "CRON", "ts": "2018-08-31 10:57:00", "month": "08", "high": 9.75, "low": 9.65, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.7211, "open": 9.75, "day": "31"} +{"volume": 154078, "symbol": "CRON", "ts": "2018-08-31 10:58:00", "month": "08", "high": 9.75, "low": 9.725, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.725, "open": 9.7272, "day": "31"} +{"volume": 149559, "symbol": "CRON", "ts": "2018-08-31 10:59:00", "month": "08", "high": 9.72, "low": 9.625, "key": "CRON_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.69, "open": 9.72, "day": "31"} +{"volume": 523, "symbol": "INS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 13.2342, "low": 13.2342, "key": "INS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.2342, "open": 13.2342, "day": "31"} +{"volume": 600, "symbol": "INS", "ts": "2018-08-31 10:52:00", "month": "08", "high": 13.3, "low": 13.3, "key": "INS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.3, "open": 13.3, "day": "31"} +{"volume": 800, "symbol": "INS", "ts": "2018-08-31 10:56:00", "month": "08", "high": 13.2349, "low": 13.2349, "key": "INS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.2349, "open": 13.2349, "day": "31"} +{"volume": 1300, "symbol": "INS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 13.2199, "low": 13.2199, "key": "INS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.2199, "open": 13.2199, "day": "31"} +{"volume": 1108, "symbol": "SRPT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 137.85, "low": 137.85, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.85, "open": 137.85, "day": "31"} +{"volume": 10135, "symbol": "SRPT", "ts": "2018-08-31 10:34:00", "month": "08", "high": 137.95, "low": 137.86, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.95, "open": 137.86, "day": "31"} +{"volume": 1788, "symbol": "SRPT", "ts": "2018-08-31 10:36:00", "month": "08", "high": 137.9629, "low": 137.9629, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 137.9629, "open": 137.9629, "day": "31"} +{"volume": 1256, "symbol": "SRPT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 138.15, "low": 138.15, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.15, "open": 138.15, "day": "31"} +{"volume": 411, "symbol": "SRPT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 138.04, "low": 138.04, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.04, "open": 138.04, "day": "31"} +{"volume": 2296, "symbol": "SRPT", "ts": "2018-08-31 10:40:00", "month": "08", "high": 138.1, "low": 138.1, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.1, "open": 138.1, "day": "31"} +{"volume": 4183, "symbol": "SRPT", "ts": "2018-08-31 10:42:00", "month": "08", "high": 138.41, "low": 138.2, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.41, "open": 138.2, "day": "31"} +{"volume": 109, "symbol": "SRPT", "ts": "2018-08-31 10:43:00", "month": "08", "high": 138.43, "low": 138.43, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.43, "open": 138.43, "day": "31"} +{"volume": 1323, "symbol": "SRPT", "ts": "2018-08-31 10:46:00", "month": "08", "high": 138.3, "low": 138.3, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.3, "open": 138.3, "day": "31"} +{"volume": 4492, "symbol": "SRPT", "ts": "2018-08-31 10:48:00", "month": "08", "high": 138.31, "low": 138.2, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.31, "open": 138.2, "day": "31"} +{"volume": 1162, "symbol": "SRPT", "ts": "2018-08-31 10:50:00", "month": "08", "high": 138.005, "low": 138.005, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.005, "open": 138.005, "day": "31"} +{"volume": 601, "symbol": "SRPT", "ts": "2018-08-31 10:52:00", "month": "08", "high": 138.29, "low": 138.29, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.29, "open": 138.29, "day": "31"} +{"volume": 140, "symbol": "SRPT", "ts": "2018-08-31 10:53:00", "month": "08", "high": 138.26, "low": 138.26, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.26, "open": 138.26, "day": "31"} +{"volume": 1232, "symbol": "SRPT", "ts": "2018-08-31 10:55:00", "month": "08", "high": 138.02, "low": 138.02, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.02, "open": 138.02, "day": "31"} +{"volume": 1156, "symbol": "SRPT", "ts": "2018-08-31 10:56:00", "month": "08", "high": 138.1, "low": 138.1, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.1, "open": 138.1, "day": "31"} +{"volume": 1791, "symbol": "SRPT", "ts": "2018-08-31 10:57:00", "month": "08", "high": 138.16, "low": 138.16, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.16, "open": 138.16, "day": "31"} +{"volume": 860, "symbol": "SRPT", "ts": "2018-08-31 10:58:00", "month": "08", "high": 138.1957, "low": 138.1957, "key": "SRPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 138.1957, "open": 138.1957, "day": "31"} +{"volume": 7982, "symbol": "HEAR", "ts": "2018-08-31 10:32:00", "month": "08", "high": 23.01, "low": 23.01, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.01, "open": 23.01, "day": "31"} +{"volume": 12944, "symbol": "HEAR", "ts": "2018-08-31 10:33:00", "month": "08", "high": 23.06, "low": 22.949, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.06, "open": 22.949, "day": "31"} +{"volume": 2771, "symbol": "HEAR", "ts": "2018-08-31 10:35:00", "month": "08", "high": 23.03, "low": 23.03, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.03, "open": 23.03, "day": "31"} +{"volume": 2686, "symbol": "HEAR", "ts": "2018-08-31 10:36:00", "month": "08", "high": 23.0, "low": 23.0, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.0, "open": 23.0, "day": "31"} +{"volume": 4567, "symbol": "HEAR", "ts": "2018-08-31 10:37:00", "month": "08", "high": 23.0, "low": 23.0, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.0, "open": 23.0, "day": "31"} +{"volume": 3001, "symbol": "HEAR", "ts": "2018-08-31 10:38:00", "month": "08", "high": 22.9773, "low": 22.9773, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 22.9773, "open": 22.9773, "day": "31"} +{"volume": 1898, "symbol": "HEAR", "ts": "2018-08-31 10:39:00", "month": "08", "high": 23.06, "low": 23.06, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.06, "open": 23.06, "day": "31"} +{"volume": 6042, "symbol": "HEAR", "ts": "2018-08-31 10:40:00", "month": "08", "high": 23.239, "low": 23.125, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.239, "open": 23.125, "day": "31"} +{"volume": 2183, "symbol": "HEAR", "ts": "2018-08-31 10:42:00", "month": "08", "high": 23.205, "low": 23.205, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.205, "open": 23.205, "day": "31"} +{"volume": 2798, "symbol": "HEAR", "ts": "2018-08-31 10:43:00", "month": "08", "high": 23.16, "low": 23.16, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.16, "open": 23.16, "day": "31"} +{"volume": 1721, "symbol": "HEAR", "ts": "2018-08-31 10:44:00", "month": "08", "high": 23.15, "low": 23.15, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.15, "open": 23.15, "day": "31"} +{"volume": 3995, "symbol": "HEAR", "ts": "2018-08-31 10:46:00", "month": "08", "high": 23.09, "low": 23.06, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.09, "open": 23.06, "day": "31"} +{"volume": 242, "symbol": "HEAR", "ts": "2018-08-31 10:47:00", "month": "08", "high": 23.14, "low": 23.14, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.14, "open": 23.14, "day": "31"} +{"volume": 982, "symbol": "HEAR", "ts": "2018-08-31 10:50:00", "month": "08", "high": 23.15, "low": 23.15, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.15, "open": 23.15, "day": "31"} +{"volume": 1805, "symbol": "HEAR", "ts": "2018-08-31 10:53:00", "month": "08", "high": 23.16, "low": 23.16, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.16, "open": 23.16, "day": "31"} +{"volume": 5015, "symbol": "HEAR", "ts": "2018-08-31 10:54:00", "month": "08", "high": 23.1504, "low": 23.11, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.1504, "open": 23.11, "day": "31"} +{"volume": 1215, "symbol": "HEAR", "ts": "2018-08-31 10:58:00", "month": "08", "high": 23.179, "low": 23.179, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.179, "open": 23.179, "day": "31"} +{"volume": 829, "symbol": "HEAR", "ts": "2018-08-31 10:59:00", "month": "08", "high": 23.135, "low": 23.135, "key": "HEAR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.135, "open": 23.135, "day": "31"} +{"volume": 333, "symbol": "BLFS", "ts": "2018-08-31 10:34:00", "month": "08", "high": 24.135, "low": 24.135, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.135, "open": 24.135, "day": "31"} +{"volume": 1600, "symbol": "BLFS", "ts": "2018-08-31 10:35:00", "month": "08", "high": 24.1467, "low": 24.1467, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.1467, "open": 24.1467, "day": "31"} +{"volume": 692, "symbol": "BLFS", "ts": "2018-08-31 10:37:00", "month": "08", "high": 24.02, "low": 24.02, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.02, "open": 24.02, "day": "31"} +{"volume": 109, "symbol": "BLFS", "ts": "2018-08-31 10:39:00", "month": "08", "high": 23.95, "low": 23.95, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 23.95, "open": 23.95, "day": "31"} +{"volume": 605, "symbol": "BLFS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 24.11, "low": 24.11, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.11, "open": 24.11, "day": "31"} +{"volume": 1850, "symbol": "BLFS", "ts": "2018-08-31 10:45:00", "month": "08", "high": 24.07, "low": 24.07, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.07, "open": 24.07, "day": "31"} +{"volume": 600, "symbol": "BLFS", "ts": "2018-08-31 10:47:00", "month": "08", "high": 24.0909, "low": 24.0909, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.0909, "open": 24.0909, "day": "31"} +{"volume": 164, "symbol": "BLFS", "ts": "2018-08-31 10:49:00", "month": "08", "high": 24.0216, "low": 24.0216, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.0216, "open": 24.0216, "day": "31"} +{"volume": 219, "symbol": "BLFS", "ts": "2018-08-31 10:51:00", "month": "08", "high": 24.08, "low": 24.08, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.08, "open": 24.08, "day": "31"} +{"volume": 210, "symbol": "BLFS", "ts": "2018-08-31 10:53:00", "month": "08", "high": 24.08, "low": 24.08, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.08, "open": 24.08, "day": "31"} +{"volume": 3386, "symbol": "BLFS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 24.17, "low": 24.17, "key": "BLFS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.17, "open": 24.17, "day": "31"} +{"volume": 200, "symbol": "LGCYO", "ts": "2018-08-31 10:36:00", "month": "08", "high": 15.42, "low": 15.42, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.42, "open": 15.42, "day": "31"} +{"volume": 372, "symbol": "LGCYO", "ts": "2018-08-31 10:46:00", "month": "08", "high": 15.4, "low": 15.4, "key": "LGCYO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.4, "open": 15.4, "day": "31"} +{"volume": 7224, "symbol": "NIHD", "ts": "2018-08-31 10:32:00", "month": "08", "high": 5.645, "low": 5.645, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.645, "open": 5.645, "day": "31"} +{"volume": 101, "symbol": "NIHD", "ts": "2018-08-31 10:33:00", "month": "08", "high": 5.64, "low": 5.64, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.64, "open": 5.64, "day": "31"} +{"volume": 403, "symbol": "NIHD", "ts": "2018-08-31 10:37:00", "month": "08", "high": 5.645, "low": 5.645, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.645, "open": 5.645, "day": "31"} +{"volume": 103, "symbol": "NIHD", "ts": "2018-08-31 10:39:00", "month": "08", "high": 5.64, "low": 5.64, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.64, "open": 5.64, "day": "31"} +{"volume": 3743, "symbol": "NIHD", "ts": "2018-08-31 10:41:00", "month": "08", "high": 5.66, "low": 5.66, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.66, "open": 5.66, "day": "31"} +{"volume": 5200, "symbol": "NIHD", "ts": "2018-08-31 10:42:00", "month": "08", "high": 5.66, "low": 5.66, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.66, "open": 5.66, "day": "31"} +{"volume": 2007, "symbol": "NIHD", "ts": "2018-08-31 10:46:00", "month": "08", "high": 5.66, "low": 5.655, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.655, "open": 5.66, "day": "31"} +{"volume": 6928, "symbol": "NIHD", "ts": "2018-08-31 10:49:00", "month": "08", "high": 5.615, "low": 5.615, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.615, "open": 5.615, "day": "31"} +{"volume": 1811, "symbol": "NIHD", "ts": "2018-08-31 10:50:00", "month": "08", "high": 5.63, "low": 5.63, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.63, "open": 5.63, "day": "31"} +{"volume": 3311, "symbol": "NIHD", "ts": "2018-08-31 10:53:00", "month": "08", "high": 5.63, "low": 5.625, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.625, "open": 5.63, "day": "31"} +{"volume": 1099, "symbol": "NIHD", "ts": "2018-08-31 10:56:00", "month": "08", "high": 5.65, "low": 5.65, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.65, "open": 5.65, "day": "31"} +{"volume": 5000, "symbol": "NIHD", "ts": "2018-08-31 10:57:00", "month": "08", "high": 5.65, "low": 5.65, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.65, "open": 5.65, "day": "31"} +{"volume": 1302, "symbol": "NIHD", "ts": "2018-08-31 10:58:00", "month": "08", "high": 5.64, "low": 5.64, "key": "NIHD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 5.64, "open": 5.64, "day": "31"} +{"volume": 2119, "symbol": "NEPT", "ts": "2018-08-31 10:31:00", "month": "08", "high": 3.969, "low": 3.969, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.969, "open": 3.969, "day": "31"} +{"volume": 11458, "symbol": "NEPT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 3.989, "low": 3.9708, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.989, "open": 3.9708, "day": "31"} +{"volume": 36506, "symbol": "NEPT", "ts": "2018-08-31 10:34:00", "month": "08", "high": 4.01, "low": 4.01, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.01, "open": 4.01, "day": "31"} +{"volume": 2829, "symbol": "NEPT", "ts": "2018-08-31 10:35:00", "month": "08", "high": 4.01, "low": 4.01, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.01, "open": 4.01, "day": "31"} +{"volume": 27830, "symbol": "NEPT", "ts": "2018-08-31 10:36:00", "month": "08", "high": 4.01, "low": 4.0, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0, "open": 4.01, "day": "31"} +{"volume": 18539, "symbol": "NEPT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 4.0199, "low": 4.0199, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0199, "open": 4.0199, "day": "31"} +{"volume": 2262, "symbol": "NEPT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 4.0188, "low": 4.0188, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0188, "open": 4.0188, "day": "31"} +{"volume": 19478, "symbol": "NEPT", "ts": "2018-08-31 10:41:00", "month": "08", "high": 4.05, "low": 4.03, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.05, "open": 4.03, "day": "31"} +{"volume": 8502, "symbol": "NEPT", "ts": "2018-08-31 10:42:00", "month": "08", "high": 4.031, "low": 4.031, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.031, "open": 4.031, "day": "31"} +{"volume": 3065, "symbol": "NEPT", "ts": "2018-08-31 10:43:00", "month": "08", "high": 4.0301, "low": 4.0301, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0301, "open": 4.0301, "day": "31"} +{"volume": 1204, "symbol": "NEPT", "ts": "2018-08-31 10:45:00", "month": "08", "high": 4.0236, "low": 4.0236, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0236, "open": 4.0236, "day": "31"} +{"volume": 3861, "symbol": "NEPT", "ts": "2018-08-31 10:47:00", "month": "08", "high": 4.035, "low": 4.035, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.035, "open": 4.035, "day": "31"} +{"volume": 1214, "symbol": "NEPT", "ts": "2018-08-31 10:48:00", "month": "08", "high": 4.035, "low": 4.035, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.035, "open": 4.035, "day": "31"} +{"volume": 823, "symbol": "NEPT", "ts": "2018-08-31 10:50:00", "month": "08", "high": 4.03, "low": 4.03, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.03, "open": 4.03, "day": "31"} +{"volume": 325, "symbol": "NEPT", "ts": "2018-08-31 10:51:00", "month": "08", "high": 4.03, "low": 4.03, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.03, "open": 4.03, "day": "31"} +{"volume": 103, "symbol": "NEPT", "ts": "2018-08-31 10:52:00", "month": "08", "high": 4.03, "low": 4.03, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.03, "open": 4.03, "day": "31"} +{"volume": 12851, "symbol": "NEPT", "ts": "2018-08-31 10:53:00", "month": "08", "high": 4.0201, "low": 4.0201, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0201, "open": 4.0201, "day": "31"} +{"volume": 30317, "symbol": "NEPT", "ts": "2018-08-31 10:54:00", "month": "08", "high": 4.0, "low": 4.0, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0, "open": 4.0, "day": "31"} +{"volume": 1409, "symbol": "NEPT", "ts": "2018-08-31 10:56:00", "month": "08", "high": 4.0016, "low": 4.0016, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0016, "open": 4.0016, "day": "31"} +{"volume": 938, "symbol": "NEPT", "ts": "2018-08-31 10:58:00", "month": "08", "high": 4.0085, "low": 4.0085, "key": "NEPT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 4.0085, "open": 4.0085, "day": "31"} +{"volume": 816, "symbol": "INGN", "ts": "2018-08-31 10:32:00", "month": "08", "high": 263.15, "low": 263.15, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.15, "open": 263.15, "day": "31"} +{"volume": 444, "symbol": "INGN", "ts": "2018-08-31 10:34:00", "month": "08", "high": 262.88, "low": 262.88, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 262.88, "open": 262.88, "day": "31"} +{"volume": 937, "symbol": "INGN", "ts": "2018-08-31 10:37:00", "month": "08", "high": 262.86, "low": 262.86, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 262.86, "open": 262.86, "day": "31"} +{"volume": 142, "symbol": "INGN", "ts": "2018-08-31 10:39:00", "month": "08", "high": 263.0658, "low": 263.0658, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.0658, "open": 263.0658, "day": "31"} +{"volume": 123, "symbol": "INGN", "ts": "2018-08-31 10:40:00", "month": "08", "high": 263.0007, "low": 263.0007, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.0007, "open": 263.0007, "day": "31"} +{"volume": 1018, "symbol": "INGN", "ts": "2018-08-31 10:43:00", "month": "08", "high": 263.27, "low": 263.02, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.02, "open": 263.27, "day": "31"} +{"volume": 353, "symbol": "INGN", "ts": "2018-08-31 10:46:00", "month": "08", "high": 263.28, "low": 263.28, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.28, "open": 263.28, "day": "31"} +{"volume": 1394, "symbol": "INGN", "ts": "2018-08-31 10:48:00", "month": "08", "high": 263.3289, "low": 263.3289, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.3289, "open": 263.3289, "day": "31"} +{"volume": 853, "symbol": "INGN", "ts": "2018-08-31 10:50:00", "month": "08", "high": 263.2418, "low": 263.2418, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 263.2418, "open": 263.2418, "day": "31"} +{"volume": 514, "symbol": "INGN", "ts": "2018-08-31 10:56:00", "month": "08", "high": 262.9269, "low": 262.9269, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 262.9269, "open": 262.9269, "day": "31"} +{"volume": 6295, "symbol": "INGN", "ts": "2018-08-31 10:58:00", "month": "08", "high": 262.665, "low": 262.61, "key": "INGN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 262.665, "open": 262.61, "day": "31"} +{"volume": 5255, "symbol": "GFN", "ts": "2018-08-31 10:37:00", "month": "08", "high": 13.4, "low": 13.4, "key": "GFN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.4, "open": 13.4, "day": "31"} +{"volume": 226, "symbol": "GFN", "ts": "2018-08-31 10:42:00", "month": "08", "high": 13.45, "low": 13.45, "key": "GFN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.45, "open": 13.45, "day": "31"} +{"volume": 100, "symbol": "GFN", "ts": "2018-08-31 10:54:00", "month": "08", "high": 13.45, "low": 13.45, "key": "GFN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.45, "open": 13.45, "day": "31"} +{"volume": 2860, "symbol": "ECYT", "ts": "2018-08-31 10:32:00", "month": "08", "high": 19.17, "low": 19.17, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.17, "open": 19.17, "day": "31"} +{"volume": 400, "symbol": "ECYT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 19.17, "low": 19.17, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.17, "open": 19.17, "day": "31"} +{"volume": 1680, "symbol": "ECYT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 19.19, "low": 19.19, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.19, "open": 19.19, "day": "31"} +{"volume": 1700, "symbol": "ECYT", "ts": "2018-08-31 10:38:00", "month": "08", "high": 19.16, "low": 19.16, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.16, "open": 19.16, "day": "31"} +{"volume": 19383, "symbol": "ECYT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 19.18, "low": 19.0, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.16, "day": "31"} +{"volume": 5723, "symbol": "ECYT", "ts": "2018-08-31 10:40:00", "month": "08", "high": 19.18, "low": 19.18, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.18, "day": "31"} +{"volume": 2227, "symbol": "ECYT", "ts": "2018-08-31 10:42:00", "month": "08", "high": 19.2, "low": 19.2, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.2, "open": 19.2, "day": "31"} +{"volume": 1392, "symbol": "ECYT", "ts": "2018-08-31 10:44:00", "month": "08", "high": 19.2, "low": 19.2, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.2, "open": 19.2, "day": "31"} +{"volume": 423, "symbol": "ECYT", "ts": "2018-08-31 10:45:00", "month": "08", "high": 19.18, "low": 19.18, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.18, "day": "31"} +{"volume": 2606, "symbol": "ECYT", "ts": "2018-08-31 10:46:00", "month": "08", "high": 19.17, "low": 19.17, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.17, "open": 19.17, "day": "31"} +{"volume": 3184, "symbol": "ECYT", "ts": "2018-08-31 10:47:00", "month": "08", "high": 19.18, "low": 19.18, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.18, "open": 19.18, "day": "31"} +{"volume": 1300, "symbol": "ECYT", "ts": "2018-08-31 10:49:00", "month": "08", "high": 19.1907, "low": 19.1907, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.1907, "open": 19.1907, "day": "31"} +{"volume": 911, "symbol": "ECYT", "ts": "2018-08-31 10:51:00", "month": "08", "high": 19.15, "low": 19.15, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.15, "open": 19.15, "day": "31"} +{"volume": 10000, "symbol": "ECYT", "ts": "2018-08-31 10:52:00", "month": "08", "high": 19.1666, "low": 19.16, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.16, "open": 19.1666, "day": "31"} +{"volume": 1300, "symbol": "ECYT", "ts": "2018-08-31 10:53:00", "month": "08", "high": 19.14, "low": 19.14, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.14, "open": 19.14, "day": "31"} +{"volume": 1257, "symbol": "ECYT", "ts": "2018-08-31 10:55:00", "month": "08", "high": 19.16, "low": 19.16, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.16, "open": 19.16, "day": "31"} +{"volume": 580, "symbol": "ECYT", "ts": "2018-08-31 10:57:00", "month": "08", "high": 19.19, "low": 19.19, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.19, "open": 19.19, "day": "31"} +{"volume": 1106, "symbol": "ECYT", "ts": "2018-08-31 10:59:00", "month": "08", "high": 19.16, "low": 19.16, "key": "ECYT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 19.16, "open": 19.16, "day": "31"} +{"volume": 167796, "symbol": "AMD", "ts": "2018-08-31 10:31:00", "month": "08", "high": 24.92, "low": 24.8898, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9, "open": 24.8898, "day": "31"} +{"volume": 281557, "symbol": "AMD", "ts": "2018-08-31 10:32:00", "month": "08", "high": 24.945, "low": 24.9, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.945, "open": 24.91, "day": "31"} +{"volume": 332679, "symbol": "AMD", "ts": "2018-08-31 10:33:00", "month": "08", "high": 24.97, "low": 24.94, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9522, "open": 24.94, "day": "31"} +{"volume": 143954, "symbol": "AMD", "ts": "2018-08-31 10:34:00", "month": "08", "high": 24.95, "low": 24.91, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.94, "open": 24.95, "day": "31"} +{"volume": 166927, "symbol": "AMD", "ts": "2018-08-31 10:35:00", "month": "08", "high": 24.95, "low": 24.895, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.895, "open": 24.94, "day": "31"} +{"volume": 93138, "symbol": "AMD", "ts": "2018-08-31 10:36:00", "month": "08", "high": 24.9147, "low": 24.89, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.91, "open": 24.89, "day": "31"} +{"volume": 85822, "symbol": "AMD", "ts": "2018-08-31 10:37:00", "month": "08", "high": 24.92, "low": 24.905, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.92, "open": 24.905, "day": "31"} +{"volume": 137367, "symbol": "AMD", "ts": "2018-08-31 10:38:00", "month": "08", "high": 24.9398, "low": 24.92, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9398, "open": 24.92, "day": "31"} +{"volume": 101112, "symbol": "AMD", "ts": "2018-08-31 10:39:00", "month": "08", "high": 24.94, "low": 24.925, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.94, "open": 24.93, "day": "31"} +{"volume": 222389, "symbol": "AMD", "ts": "2018-08-31 10:40:00", "month": "08", "high": 24.98, "low": 24.935, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.98, "open": 24.94, "day": "31"} +{"volume": 336095, "symbol": "AMD", "ts": "2018-08-31 10:41:00", "month": "08", "high": 25.01, "low": 24.975, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.98, "open": 24.975, "day": "31"} +{"volume": 97122, "symbol": "AMD", "ts": "2018-08-31 10:42:00", "month": "08", "high": 24.98, "low": 24.95, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.9699, "open": 24.98, "day": "31"} +{"volume": 183736, "symbol": "AMD", "ts": "2018-08-31 10:43:00", "month": "08", "high": 25.01, "low": 24.9725, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0, "open": 24.9725, "day": "31"} +{"volume": 247540, "symbol": "AMD", "ts": "2018-08-31 10:44:00", "month": "08", "high": 25.035, "low": 25.0001, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0199, "open": 25.0001, "day": "31"} +{"volume": 229981, "symbol": "AMD", "ts": "2018-08-31 10:45:00", "month": "08", "high": 25.04, "low": 25.0, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0378, "open": 25.02, "day": "31"} +{"volume": 192298, "symbol": "AMD", "ts": "2018-08-31 10:46:00", "month": "08", "high": 25.05, "low": 25.02, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.03, "open": 25.045, "day": "31"} +{"volume": 123688, "symbol": "AMD", "ts": "2018-08-31 10:47:00", "month": "08", "high": 25.0401, "low": 25.035, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.04, "open": 25.04, "day": "31"} +{"volume": 201197, "symbol": "AMD", "ts": "2018-08-31 10:48:00", "month": "08", "high": 25.05, "low": 25.0, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.015, "open": 25.05, "day": "31"} +{"volume": 224088, "symbol": "AMD", "ts": "2018-08-31 10:49:00", "month": "08", "high": 25.07, "low": 25.01, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.01, "open": 25.03, "day": "31"} +{"volume": 231558, "symbol": "AMD", "ts": "2018-08-31 10:50:00", "month": "08", "high": 25.07, "low": 25.02, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.065, "open": 25.02, "day": "31"} +{"volume": 252898, "symbol": "AMD", "ts": "2018-08-31 10:51:00", "month": "08", "high": 25.09, "low": 25.061, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0637, "open": 25.061, "day": "31"} +{"volume": 117932, "symbol": "AMD", "ts": "2018-08-31 10:52:00", "month": "08", "high": 25.07, "low": 25.04, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.07, "open": 25.05, "day": "31"} +{"volume": 238706, "symbol": "AMD", "ts": "2018-08-31 10:53:00", "month": "08", "high": 25.095, "low": 25.06, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.095, "open": 25.06, "day": "31"} +{"volume": 229047, "symbol": "AMD", "ts": "2018-08-31 10:54:00", "month": "08", "high": 25.1, "low": 25.06, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.1, "open": 25.095, "day": "31"} +{"volume": 215553, "symbol": "AMD", "ts": "2018-08-31 10:55:00", "month": "08", "high": 25.11, "low": 25.061, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.061, "open": 25.11, "day": "31"} +{"volume": 118549, "symbol": "AMD", "ts": "2018-08-31 10:56:00", "month": "08", "high": 25.09, "low": 25.06, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.085, "open": 25.08, "day": "31"} +{"volume": 145836, "symbol": "AMD", "ts": "2018-08-31 10:57:00", "month": "08", "high": 25.1, "low": 25.06, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0616, "open": 25.1, "day": "31"} +{"volume": 268987, "symbol": "AMD", "ts": "2018-08-31 10:58:00", "month": "08", "high": 25.06, "low": 25.01, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0345, "open": 25.06, "day": "31"} +{"volume": 155370, "symbol": "AMD", "ts": "2018-08-31 10:59:00", "month": "08", "high": 25.05, "low": 25.03, "key": "AMD_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.045, "open": 25.03, "day": "31"} +{"volume": 2189, "symbol": "CDMO", "ts": "2018-08-31 10:34:00", "month": "08", "high": 7.0154, "low": 7.0154, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0154, "open": 7.0154, "day": "31"} +{"volume": 100, "symbol": "CDMO", "ts": "2018-08-31 10:35:00", "month": "08", "high": 7.03, "low": 7.03, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.03, "open": 7.03, "day": "31"} +{"volume": 213, "symbol": "CDMO", "ts": "2018-08-31 10:36:00", "month": "08", "high": 7.015, "low": 7.015, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.015, "open": 7.015, "day": "31"} +{"volume": 2891, "symbol": "CDMO", "ts": "2018-08-31 10:38:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 8278, "symbol": "CDMO", "ts": "2018-08-31 10:41:00", "month": "08", "high": 7.015, "low": 6.99, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.015, "open": 6.99, "day": "31"} +{"volume": 1958, "symbol": "CDMO", "ts": "2018-08-31 10:44:00", "month": "08", "high": 7.0, "low": 7.0, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0, "open": 7.0, "day": "31"} +{"volume": 919, "symbol": "CDMO", "ts": "2018-08-31 10:45:00", "month": "08", "high": 7.05, "low": 7.05, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.05, "open": 7.05, "day": "31"} +{"volume": 2573, "symbol": "CDMO", "ts": "2018-08-31 10:47:00", "month": "08", "high": 7.0481, "low": 7.0481, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0481, "open": 7.0481, "day": "31"} +{"volume": 1643, "symbol": "CDMO", "ts": "2018-08-31 10:49:00", "month": "08", "high": 7.015, "low": 7.01, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.01, "open": 7.015, "day": "31"} +{"volume": 223, "symbol": "CDMO", "ts": "2018-08-31 10:54:00", "month": "08", "high": 7.0299, "low": 7.0299, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.0299, "open": 7.0299, "day": "31"} +{"volume": 2936, "symbol": "CDMO", "ts": "2018-08-31 10:56:00", "month": "08", "high": 7.03, "low": 7.03, "key": "CDMO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 7.03, "open": 7.03, "day": "31"} +{"volume": 8034, "symbol": "SFIX", "ts": "2018-08-31 10:31:00", "month": "08", "high": 40.04, "low": 40.01, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.04, "open": 40.01, "day": "31"} +{"volume": 7665, "symbol": "SFIX", "ts": "2018-08-31 10:32:00", "month": "08", "high": 40.15, "low": 40.12, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.15, "open": 40.12, "day": "31"} +{"volume": 32130, "symbol": "SFIX", "ts": "2018-08-31 10:33:00", "month": "08", "high": 40.19, "low": 40.15, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.161, "open": 40.19, "day": "31"} +{"volume": 28330, "symbol": "SFIX", "ts": "2018-08-31 10:34:00", "month": "08", "high": 40.31, "low": 40.27, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.27, "open": 40.29, "day": "31"} +{"volume": 18090, "symbol": "SFIX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 40.359, "low": 40.34, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.34, "open": 40.359, "day": "31"} +{"volume": 13242, "symbol": "SFIX", "ts": "2018-08-31 10:36:00", "month": "08", "high": 40.42, "low": 40.4056, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.4056, "open": 40.42, "day": "31"} +{"volume": 15319, "symbol": "SFIX", "ts": "2018-08-31 10:37:00", "month": "08", "high": 40.4312, "low": 40.3762, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.4312, "open": 40.3762, "day": "31"} +{"volume": 22305, "symbol": "SFIX", "ts": "2018-08-31 10:38:00", "month": "08", "high": 40.625, "low": 40.59, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.625, "open": 40.59, "day": "31"} +{"volume": 25562, "symbol": "SFIX", "ts": "2018-08-31 10:39:00", "month": "08", "high": 40.53, "low": 40.4999, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.4999, "open": 40.53, "day": "31"} +{"volume": 28621, "symbol": "SFIX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 40.334, "low": 40.3, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3049, "open": 40.3, "day": "31"} +{"volume": 17581, "symbol": "SFIX", "ts": "2018-08-31 10:41:00", "month": "08", "high": 40.29, "low": 40.1843, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.215, "open": 40.29, "day": "31"} +{"volume": 3209, "symbol": "SFIX", "ts": "2018-08-31 10:42:00", "month": "08", "high": 40.32, "low": 40.32, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.32, "open": 40.32, "day": "31"} +{"volume": 3380, "symbol": "SFIX", "ts": "2018-08-31 10:43:00", "month": "08", "high": 40.2, "low": 40.2, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.2, "open": 40.2, "day": "31"} +{"volume": 6196, "symbol": "SFIX", "ts": "2018-08-31 10:44:00", "month": "08", "high": 40.24, "low": 40.2044, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.2044, "open": 40.24, "day": "31"} +{"volume": 3068, "symbol": "SFIX", "ts": "2018-08-31 10:45:00", "month": "08", "high": 40.34, "low": 40.34, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.34, "open": 40.34, "day": "31"} +{"volume": 2121, "symbol": "SFIX", "ts": "2018-08-31 10:46:00", "month": "08", "high": 40.32, "low": 40.32, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.32, "open": 40.32, "day": "31"} +{"volume": 16707, "symbol": "SFIX", "ts": "2018-08-31 10:47:00", "month": "08", "high": 40.29, "low": 40.1, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.1, "open": 40.29, "day": "31"} +{"volume": 1692, "symbol": "SFIX", "ts": "2018-08-31 10:48:00", "month": "08", "high": 40.15, "low": 40.15, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.15, "open": 40.15, "day": "31"} +{"volume": 9119, "symbol": "SFIX", "ts": "2018-08-31 10:50:00", "month": "08", "high": 40.276, "low": 40.22, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.22, "open": 40.276, "day": "31"} +{"volume": 3734, "symbol": "SFIX", "ts": "2018-08-31 10:51:00", "month": "08", "high": 40.3175, "low": 40.3175, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3175, "open": 40.3175, "day": "31"} +{"volume": 3670, "symbol": "SFIX", "ts": "2018-08-31 10:52:00", "month": "08", "high": 40.3437, "low": 40.3437, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3437, "open": 40.3437, "day": "31"} +{"volume": 5919, "symbol": "SFIX", "ts": "2018-08-31 10:53:00", "month": "08", "high": 40.3299, "low": 40.3299, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3299, "open": 40.3299, "day": "31"} +{"volume": 2993, "symbol": "SFIX", "ts": "2018-08-31 10:54:00", "month": "08", "high": 40.3284, "low": 40.3284, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3284, "open": 40.3284, "day": "31"} +{"volume": 8489, "symbol": "SFIX", "ts": "2018-08-31 10:55:00", "month": "08", "high": 40.34, "low": 40.34, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.34, "open": 40.34, "day": "31"} +{"volume": 3094, "symbol": "SFIX", "ts": "2018-08-31 10:56:00", "month": "08", "high": 40.3103, "low": 40.3103, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3103, "open": 40.3103, "day": "31"} +{"volume": 38075, "symbol": "SFIX", "ts": "2018-08-31 10:57:00", "month": "08", "high": 40.418, "low": 40.35, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.418, "open": 40.35, "day": "31"} +{"volume": 5545, "symbol": "SFIX", "ts": "2018-08-31 10:58:00", "month": "08", "high": 40.3773, "low": 40.3773, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.3773, "open": 40.3773, "day": "31"} +{"volume": 15219, "symbol": "SFIX", "ts": "2018-08-31 10:59:00", "month": "08", "high": 40.34, "low": 40.2622, "key": "SFIX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 40.2622, "open": 40.34, "day": "31"} +{"volume": 1710, "symbol": "DRNA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 15.79, "low": 15.79, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.79, "open": 15.79, "day": "31"} +{"volume": 440, "symbol": "DRNA", "ts": "2018-08-31 10:34:00", "month": "08", "high": 15.77, "low": 15.77, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.77, "open": 15.77, "day": "31"} +{"volume": 1129, "symbol": "DRNA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 15.85, "low": 15.85, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.85, "open": 15.85, "day": "31"} +{"volume": 200, "symbol": "DRNA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 15.86, "low": 15.86, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.86, "open": 15.86, "day": "31"} +{"volume": 1801, "symbol": "DRNA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 15.86, "low": 15.86, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.86, "open": 15.86, "day": "31"} +{"volume": 4243, "symbol": "DRNA", "ts": "2018-08-31 10:46:00", "month": "08", "high": 15.87, "low": 15.81, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.81, "open": 15.87, "day": "31"} +{"volume": 250, "symbol": "DRNA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 15.8316, "low": 15.8316, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.8316, "open": 15.8316, "day": "31"} +{"volume": 2000, "symbol": "DRNA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 15.82, "low": 15.82, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.82, "open": 15.82, "day": "31"} +{"volume": 501, "symbol": "DRNA", "ts": "2018-08-31 10:51:00", "month": "08", "high": 15.8, "low": 15.8, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.8, "open": 15.8, "day": "31"} +{"volume": 301, "symbol": "DRNA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 15.87, "low": 15.87, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.87, "open": 15.87, "day": "31"} +{"volume": 375, "symbol": "DRNA", "ts": "2018-08-31 10:56:00", "month": "08", "high": 15.9, "low": 15.9, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.9, "open": 15.9, "day": "31"} +{"volume": 900, "symbol": "DRNA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 15.88, "low": 15.88, "key": "DRNA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.88, "open": 15.88, "day": "31"} +{"volume": 550, "symbol": "RFP", "ts": "2018-08-31 10:33:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 1900, "symbol": "RFP", "ts": "2018-08-31 10:38:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 100, "symbol": "RFP", "ts": "2018-08-31 10:41:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 1171, "symbol": "RFP", "ts": "2018-08-31 10:42:00", "month": "08", "high": 13.95, "low": 13.95, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.95, "open": 13.95, "day": "31"} +{"volume": 1767, "symbol": "RFP", "ts": "2018-08-31 10:46:00", "month": "08", "high": 13.9, "low": 13.9, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.9, "open": 13.9, "day": "31"} +{"volume": 1600, "symbol": "RFP", "ts": "2018-08-31 10:49:00", "month": "08", "high": 13.85, "low": 13.85, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.85, "open": 13.85, "day": "31"} +{"volume": 387, "symbol": "RFP", "ts": "2018-08-31 10:52:00", "month": "08", "high": 13.82, "low": 13.82, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.82, "open": 13.82, "day": "31"} +{"volume": 2459, "symbol": "RFP", "ts": "2018-08-31 10:54:00", "month": "08", "high": 13.8, "low": 13.8, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.8, "open": 13.8, "day": "31"} +{"volume": 2848, "symbol": "RFP", "ts": "2018-08-31 10:55:00", "month": "08", "high": 13.85, "low": 13.85, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.85, "open": 13.85, "day": "31"} +{"volume": 300, "symbol": "RFP", "ts": "2018-08-31 10:56:00", "month": "08", "high": 13.826, "low": 13.826, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.826, "open": 13.826, "day": "31"} +{"volume": 440, "symbol": "RFP", "ts": "2018-08-31 10:57:00", "month": "08", "high": 13.85, "low": 13.85, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.85, "open": 13.85, "day": "31"} +{"volume": 230, "symbol": "RFP", "ts": "2018-08-31 10:59:00", "month": "08", "high": 13.85, "low": 13.85, "key": "RFP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 13.85, "open": 13.85, "day": "31"} +{"volume": 296, "symbol": "BJRI", "ts": "2018-08-31 10:31:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 328, "symbol": "BJRI", "ts": "2018-08-31 10:36:00", "month": "08", "high": 74.525, "low": 74.525, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.525, "open": 74.525, "day": "31"} +{"volume": 1016, "symbol": "BJRI", "ts": "2018-08-31 10:37:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 222, "symbol": "BJRI", "ts": "2018-08-31 10:38:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 315, "symbol": "BJRI", "ts": "2018-08-31 10:41:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 958, "symbol": "BJRI", "ts": "2018-08-31 10:42:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 3579, "symbol": "BJRI", "ts": "2018-08-31 10:43:00", "month": "08", "high": 74.325, "low": 74.325, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.325, "open": 74.325, "day": "31"} +{"volume": 385, "symbol": "BJRI", "ts": "2018-08-31 10:45:00", "month": "08", "high": 74.3, "low": 74.3, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.3, "open": 74.3, "day": "31"} +{"volume": 950, "symbol": "BJRI", "ts": "2018-08-31 10:50:00", "month": "08", "high": 74.3, "low": 74.3, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.3, "open": 74.3, "day": "31"} +{"volume": 2717, "symbol": "BJRI", "ts": "2018-08-31 10:54:00", "month": "08", "high": 74.55, "low": 74.55, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.55, "open": 74.55, "day": "31"} +{"volume": 203, "symbol": "BJRI", "ts": "2018-08-31 10:55:00", "month": "08", "high": 74.5, "low": 74.5, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.5, "open": 74.5, "day": "31"} +{"volume": 2121, "symbol": "BJRI", "ts": "2018-08-31 10:57:00", "month": "08", "high": 74.6, "low": 74.6, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.6, "open": 74.6, "day": "31"} +{"volume": 747, "symbol": "BJRI", "ts": "2018-08-31 10:58:00", "month": "08", "high": 74.675, "low": 74.675, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.675, "open": 74.675, "day": "31"} +{"volume": 102, "symbol": "BJRI", "ts": "2018-08-31 10:59:00", "month": "08", "high": 74.675, "low": 74.675, "key": "BJRI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 74.675, "open": 74.675, "day": "31"} +{"volume": 7244, "symbol": "NVCR", "ts": "2018-08-31 10:31:00", "month": "08", "high": 44.85, "low": 44.85, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.85, "open": 44.85, "day": "31"} +{"volume": 2252, "symbol": "NVCR", "ts": "2018-08-31 10:32:00", "month": "08", "high": 44.65, "low": 44.65, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.65, "open": 44.65, "day": "31"} +{"volume": 2346, "symbol": "NVCR", "ts": "2018-08-31 10:33:00", "month": "08", "high": 44.65, "low": 44.65, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.65, "open": 44.65, "day": "31"} +{"volume": 2517, "symbol": "NVCR", "ts": "2018-08-31 10:35:00", "month": "08", "high": 44.6, "low": 44.6, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.6, "open": 44.6, "day": "31"} +{"volume": 1763, "symbol": "NVCR", "ts": "2018-08-31 10:37:00", "month": "08", "high": 44.75, "low": 44.75, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.75, "open": 44.75, "day": "31"} +{"volume": 2482, "symbol": "NVCR", "ts": "2018-08-31 10:38:00", "month": "08", "high": 44.73, "low": 44.73, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.73, "open": 44.73, "day": "31"} +{"volume": 25690, "symbol": "NVCR", "ts": "2018-08-31 10:39:00", "month": "08", "high": 44.7, "low": 44.7, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7, "open": 44.7, "day": "31"} +{"volume": 5034, "symbol": "NVCR", "ts": "2018-08-31 10:40:00", "month": "08", "high": 44.65, "low": 44.65, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.65, "open": 44.65, "day": "31"} +{"volume": 2841, "symbol": "NVCR", "ts": "2018-08-31 10:41:00", "month": "08", "high": 44.625, "low": 44.625, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.625, "open": 44.625, "day": "31"} +{"volume": 3396, "symbol": "NVCR", "ts": "2018-08-31 10:42:00", "month": "08", "high": 44.65, "low": 44.65, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.65, "open": 44.65, "day": "31"} +{"volume": 1370, "symbol": "NVCR", "ts": "2018-08-31 10:44:00", "month": "08", "high": 44.7, "low": 44.7, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7, "open": 44.7, "day": "31"} +{"volume": 2063, "symbol": "NVCR", "ts": "2018-08-31 10:45:00", "month": "08", "high": 44.7, "low": 44.7, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7, "open": 44.7, "day": "31"} +{"volume": 1253, "symbol": "NVCR", "ts": "2018-08-31 10:47:00", "month": "08", "high": 44.7, "low": 44.7, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7, "open": 44.7, "day": "31"} +{"volume": 358, "symbol": "NVCR", "ts": "2018-08-31 10:48:00", "month": "08", "high": 44.7, "low": 44.7, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.7, "open": 44.7, "day": "31"} +{"volume": 1129, "symbol": "NVCR", "ts": "2018-08-31 10:51:00", "month": "08", "high": 44.74, "low": 44.74, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.74, "open": 44.74, "day": "31"} +{"volume": 4749, "symbol": "NVCR", "ts": "2018-08-31 10:52:00", "month": "08", "high": 44.55, "low": 44.55, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.55, "open": 44.55, "day": "31"} +{"volume": 4503, "symbol": "NVCR", "ts": "2018-08-31 10:54:00", "month": "08", "high": 44.75, "low": 44.55, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.75, "open": 44.55, "day": "31"} +{"volume": 4973, "symbol": "NVCR", "ts": "2018-08-31 10:56:00", "month": "08", "high": 44.75, "low": 44.75, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.75, "open": 44.75, "day": "31"} +{"volume": 1716, "symbol": "NVCR", "ts": "2018-08-31 10:58:00", "month": "08", "high": 44.8, "low": 44.8, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.8, "open": 44.8, "day": "31"} +{"volume": 1402, "symbol": "NVCR", "ts": "2018-08-31 10:59:00", "month": "08", "high": 44.85, "low": 44.85, "key": "NVCR_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.85, "open": 44.85, "day": "31"} +{"volume": 1823, "symbol": "NOG", "ts": "2018-08-31 10:32:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 2300, "symbol": "NOG", "ts": "2018-08-31 10:33:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 3000, "symbol": "NOG", "ts": "2018-08-31 10:34:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 10200, "symbol": "NOG", "ts": "2018-08-31 10:35:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 2700, "symbol": "NOG", "ts": "2018-08-31 10:36:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 4542, "symbol": "NOG", "ts": "2018-08-31 10:37:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 1589, "symbol": "NOG", "ts": "2018-08-31 10:39:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 6890, "symbol": "NOG", "ts": "2018-08-31 10:40:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 17711, "symbol": "NOG", "ts": "2018-08-31 10:41:00", "month": "08", "high": 3.37, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.365, "day": "31"} +{"volume": 30162, "symbol": "NOG", "ts": "2018-08-31 10:42:00", "month": "08", "high": 3.37, "low": 3.37, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.37, "open": 3.37, "day": "31"} +{"volume": 10917, "symbol": "NOG", "ts": "2018-08-31 10:43:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 6213, "symbol": "NOG", "ts": "2018-08-31 10:44:00", "month": "08", "high": 3.365, "low": 3.365, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.365, "open": 3.365, "day": "31"} +{"volume": 90551, "symbol": "NOG", "ts": "2018-08-31 10:46:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 1505, "symbol": "NOG", "ts": "2018-08-31 10:47:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 700, "symbol": "NOG", "ts": "2018-08-31 10:48:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 2392, "symbol": "NOG", "ts": "2018-08-31 10:49:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 7058, "symbol": "NOG", "ts": "2018-08-31 10:50:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 2945, "symbol": "NOG", "ts": "2018-08-31 10:51:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 2912, "symbol": "NOG", "ts": "2018-08-31 10:52:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 4409, "symbol": "NOG", "ts": "2018-08-31 10:53:00", "month": "08", "high": 3.36, "low": 3.36, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.36, "open": 3.36, "day": "31"} +{"volume": 5949, "symbol": "NOG", "ts": "2018-08-31 10:54:00", "month": "08", "high": 3.3549, "low": 3.3549, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.3549, "open": 3.3549, "day": "31"} +{"volume": 1405, "symbol": "NOG", "ts": "2018-08-31 10:55:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 93753, "symbol": "NOG", "ts": "2018-08-31 10:56:00", "month": "08", "high": 3.35, "low": 3.345, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.35, "open": 3.35, "day": "31"} +{"volume": 6500, "symbol": "NOG", "ts": "2018-08-31 10:57:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 347, "symbol": "NOG", "ts": "2018-08-31 10:58:00", "month": "08", "high": 3.355, "low": 3.355, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.355, "open": 3.355, "day": "31"} +{"volume": 7601, "symbol": "NOG", "ts": "2018-08-31 10:59:00", "month": "08", "high": 3.35, "low": 3.35, "key": "NOG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 3.35, "open": 3.35, "day": "31"} +{"volume": 433, "symbol": "PTSI", "ts": "2018-08-31 10:42:00", "month": "08", "high": 57.5, "low": 57.5, "key": "PTSI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 57.5, "open": 57.5, "day": "31"} +{"volume": 1690, "symbol": "BAND", "ts": "2018-08-31 10:35:00", "month": "08", "high": 44.955, "low": 44.955, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 44.955, "open": 44.955, "day": "31"} +{"volume": 1238, "symbol": "BAND", "ts": "2018-08-31 10:36:00", "month": "08", "high": 45.08, "low": 45.08, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.08, "open": 45.08, "day": "31"} +{"volume": 528, "symbol": "BAND", "ts": "2018-08-31 10:50:00", "month": "08", "high": 45.13, "low": 45.13, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.13, "open": 45.13, "day": "31"} +{"volume": 100, "symbol": "BAND", "ts": "2018-08-31 10:54:00", "month": "08", "high": 45.16, "low": 45.16, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.16, "open": 45.16, "day": "31"} +{"volume": 591, "symbol": "BAND", "ts": "2018-08-31 10:56:00", "month": "08", "high": 45.16, "low": 45.16, "key": "BAND_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 45.16, "open": 45.16, "day": "31"} +{"volume": 11602, "symbol": "AMRS", "ts": "2018-08-31 10:31:00", "month": "08", "high": 8.8044, "low": 8.79, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.79, "open": 8.8, "day": "31"} +{"volume": 8150, "symbol": "AMRS", "ts": "2018-08-31 10:32:00", "month": "08", "high": 8.74, "low": 8.74, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.74, "open": 8.74, "day": "31"} +{"volume": 14276, "symbol": "AMRS", "ts": "2018-08-31 10:34:00", "month": "08", "high": 8.751, "low": 8.7299, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.7299, "open": 8.751, "day": "31"} +{"volume": 4193, "symbol": "AMRS", "ts": "2018-08-31 10:35:00", "month": "08", "high": 8.68, "low": 8.68, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.68, "open": 8.68, "day": "31"} +{"volume": 10479, "symbol": "AMRS", "ts": "2018-08-31 10:37:00", "month": "08", "high": 8.6645, "low": 8.6645, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6645, "open": 8.6645, "day": "31"} +{"volume": 6113, "symbol": "AMRS", "ts": "2018-08-31 10:38:00", "month": "08", "high": 8.65, "low": 8.6376, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.6376, "open": 8.65, "day": "31"} +{"volume": 12330, "symbol": "AMRS", "ts": "2018-08-31 10:39:00", "month": "08", "high": 8.66, "low": 8.65, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.66, "open": 8.65, "day": "31"} +{"volume": 2130, "symbol": "AMRS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 8.69, "low": 8.69, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.69, "open": 8.69, "day": "31"} +{"volume": 3409, "symbol": "AMRS", "ts": "2018-08-31 10:42:00", "month": "08", "high": 8.73, "low": 8.73, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.73, "open": 8.73, "day": "31"} +{"volume": 5785, "symbol": "AMRS", "ts": "2018-08-31 10:43:00", "month": "08", "high": 8.71, "low": 8.71, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.71, "open": 8.71, "day": "31"} +{"volume": 2653, "symbol": "AMRS", "ts": "2018-08-31 10:44:00", "month": "08", "high": 8.74, "low": 8.74, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.74, "open": 8.74, "day": "31"} +{"volume": 2588, "symbol": "AMRS", "ts": "2018-08-31 10:45:00", "month": "08", "high": 8.7599, "low": 8.7599, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.7599, "open": 8.7599, "day": "31"} +{"volume": 4762, "symbol": "AMRS", "ts": "2018-08-31 10:47:00", "month": "08", "high": 8.77, "low": 8.77, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.77, "open": 8.77, "day": "31"} +{"volume": 4220, "symbol": "AMRS", "ts": "2018-08-31 10:48:00", "month": "08", "high": 8.765, "low": 8.765, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.765, "open": 8.765, "day": "31"} +{"volume": 1918, "symbol": "AMRS", "ts": "2018-08-31 10:49:00", "month": "08", "high": 8.8, "low": 8.8, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.8, "open": 8.8, "day": "31"} +{"volume": 13586, "symbol": "AMRS", "ts": "2018-08-31 10:50:00", "month": "08", "high": 8.82, "low": 8.81, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.81, "open": 8.82, "day": "31"} +{"volume": 17358, "symbol": "AMRS", "ts": "2018-08-31 10:51:00", "month": "08", "high": 8.86, "low": 8.8, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.86, "open": 8.8, "day": "31"} +{"volume": 9370, "symbol": "AMRS", "ts": "2018-08-31 10:52:00", "month": "08", "high": 8.8718, "low": 8.87, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.8718, "open": 8.87, "day": "31"} +{"volume": 4761, "symbol": "AMRS", "ts": "2018-08-31 10:53:00", "month": "08", "high": 8.86, "low": 8.86, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.86, "open": 8.86, "day": "31"} +{"volume": 10179, "symbol": "AMRS", "ts": "2018-08-31 10:54:00", "month": "08", "high": 8.8948, "low": 8.8948, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.8948, "open": 8.8948, "day": "31"} +{"volume": 25982, "symbol": "AMRS", "ts": "2018-08-31 10:55:00", "month": "08", "high": 8.9127, "low": 8.9067, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.9127, "open": 8.9067, "day": "31"} +{"volume": 5250, "symbol": "AMRS", "ts": "2018-08-31 10:56:00", "month": "08", "high": 8.9188, "low": 8.9188, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.9188, "open": 8.9188, "day": "31"} +{"volume": 3617, "symbol": "AMRS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 8.9, "low": 8.9, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.9, "open": 8.9, "day": "31"} +{"volume": 3591, "symbol": "AMRS", "ts": "2018-08-31 10:58:00", "month": "08", "high": 8.92, "low": 8.92, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.92, "open": 8.92, "day": "31"} +{"volume": 8955, "symbol": "AMRS", "ts": "2018-08-31 10:59:00", "month": "08", "high": 8.9301, "low": 8.92, "key": "AMRS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 8.92, "open": 8.9301, "day": "31"} +{"volume": 613, "symbol": "COUP", "ts": "2018-08-31 10:33:00", "month": "08", "high": 70.94, "low": 70.94, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.94, "open": 70.94, "day": "31"} +{"volume": 5461, "symbol": "COUP", "ts": "2018-08-31 10:35:00", "month": "08", "high": 70.785, "low": 70.785, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.785, "open": 70.785, "day": "31"} +{"volume": 2576, "symbol": "COUP", "ts": "2018-08-31 10:36:00", "month": "08", "high": 70.59, "low": 70.59, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.59, "open": 70.59, "day": "31"} +{"volume": 653, "symbol": "COUP", "ts": "2018-08-31 10:37:00", "month": "08", "high": 70.585, "low": 70.585, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.585, "open": 70.585, "day": "31"} +{"volume": 1741, "symbol": "COUP", "ts": "2018-08-31 10:39:00", "month": "08", "high": 70.53, "low": 70.53, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.53, "open": 70.53, "day": "31"} +{"volume": 782, "symbol": "COUP", "ts": "2018-08-31 10:40:00", "month": "08", "high": 70.61, "low": 70.61, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.61, "open": 70.61, "day": "31"} +{"volume": 1504, "symbol": "COUP", "ts": "2018-08-31 10:42:00", "month": "08", "high": 70.64, "low": 70.64, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.64, "open": 70.64, "day": "31"} +{"volume": 516, "symbol": "COUP", "ts": "2018-08-31 10:45:00", "month": "08", "high": 70.66, "low": 70.66, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.66, "open": 70.66, "day": "31"} +{"volume": 2906, "symbol": "COUP", "ts": "2018-08-31 10:46:00", "month": "08", "high": 70.75, "low": 70.73, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.73, "open": 70.75, "day": "31"} +{"volume": 1398, "symbol": "COUP", "ts": "2018-08-31 10:49:00", "month": "08", "high": 70.6893, "low": 70.6893, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.6893, "open": 70.6893, "day": "31"} +{"volume": 846, "symbol": "COUP", "ts": "2018-08-31 10:51:00", "month": "08", "high": 70.72, "low": 70.72, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.72, "open": 70.72, "day": "31"} +{"volume": 473, "symbol": "COUP", "ts": "2018-08-31 10:52:00", "month": "08", "high": 70.71, "low": 70.71, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.71, "open": 70.71, "day": "31"} +{"volume": 2372, "symbol": "COUP", "ts": "2018-08-31 10:54:00", "month": "08", "high": 70.726, "low": 70.726, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.726, "open": 70.726, "day": "31"} +{"volume": 2145, "symbol": "COUP", "ts": "2018-08-31 10:57:00", "month": "08", "high": 70.82, "low": 70.82, "key": "COUP_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.82, "open": 70.82, "day": "31"} +{"volume": 1579, "symbol": "AAXN", "ts": "2018-08-31 10:31:00", "month": "08", "high": 66.83, "low": 66.83, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.83, "open": 66.83, "day": "31"} +{"volume": 514, "symbol": "AAXN", "ts": "2018-08-31 10:32:00", "month": "08", "high": 66.66, "low": 66.66, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.66, "open": 66.66, "day": "31"} +{"volume": 790, "symbol": "AAXN", "ts": "2018-08-31 10:35:00", "month": "08", "high": 66.6699, "low": 66.6699, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.6699, "open": 66.6699, "day": "31"} +{"volume": 207, "symbol": "AAXN", "ts": "2018-08-31 10:38:00", "month": "08", "high": 66.68, "low": 66.68, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.68, "open": 66.68, "day": "31"} +{"volume": 883, "symbol": "AAXN", "ts": "2018-08-31 10:40:00", "month": "08", "high": 66.6245, "low": 66.6245, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.6245, "open": 66.6245, "day": "31"} +{"volume": 340, "symbol": "AAXN", "ts": "2018-08-31 10:42:00", "month": "08", "high": 66.62, "low": 66.62, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.62, "open": 66.62, "day": "31"} +{"volume": 440, "symbol": "AAXN", "ts": "2018-08-31 10:44:00", "month": "08", "high": 66.64, "low": 66.64, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.64, "open": 66.64, "day": "31"} +{"volume": 2005, "symbol": "AAXN", "ts": "2018-08-31 10:45:00", "month": "08", "high": 66.6201, "low": 66.6201, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.6201, "open": 66.6201, "day": "31"} +{"volume": 328, "symbol": "AAXN", "ts": "2018-08-31 10:47:00", "month": "08", "high": 66.88, "low": 66.88, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.88, "open": 66.88, "day": "31"} +{"volume": 324, "symbol": "AAXN", "ts": "2018-08-31 10:50:00", "month": "08", "high": 66.909, "low": 66.909, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.909, "open": 66.909, "day": "31"} +{"volume": 669, "symbol": "AAXN", "ts": "2018-08-31 10:54:00", "month": "08", "high": 66.895, "low": 66.895, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.895, "open": 66.895, "day": "31"} +{"volume": 420, "symbol": "AAXN", "ts": "2018-08-31 10:57:00", "month": "08", "high": 66.915, "low": 66.915, "key": "AAXN_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 66.915, "open": 66.915, "day": "31"} +{"volume": 4658, "symbol": "OKTA", "ts": "2018-08-31 10:33:00", "month": "08", "high": 61.35, "low": 61.265, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.35, "open": 61.265, "day": "31"} +{"volume": 13812, "symbol": "OKTA", "ts": "2018-08-31 10:34:00", "month": "08", "high": 61.33, "low": 61.26, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.33, "open": 61.26, "day": "31"} +{"volume": 2593, "symbol": "OKTA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 61.46, "low": 61.4, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.46, "open": 61.4, "day": "31"} +{"volume": 14504, "symbol": "OKTA", "ts": "2018-08-31 10:38:00", "month": "08", "high": 61.69, "low": 61.59, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.59, "open": 61.69, "day": "31"} +{"volume": 3130, "symbol": "OKTA", "ts": "2018-08-31 10:39:00", "month": "08", "high": 61.525, "low": 61.525, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.525, "open": 61.525, "day": "31"} +{"volume": 1306, "symbol": "OKTA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 61.55, "low": 61.55, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 1325, "symbol": "OKTA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 61.55, "low": 61.55, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.55, "open": 61.55, "day": "31"} +{"volume": 985, "symbol": "OKTA", "ts": "2018-08-31 10:43:00", "month": "08", "high": 61.5269, "low": 61.5269, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.5269, "open": 61.5269, "day": "31"} +{"volume": 626, "symbol": "OKTA", "ts": "2018-08-31 10:45:00", "month": "08", "high": 61.56, "low": 61.56, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.56, "open": 61.56, "day": "31"} +{"volume": 4670, "symbol": "OKTA", "ts": "2018-08-31 10:46:00", "month": "08", "high": 61.61, "low": 61.61, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.61, "open": 61.61, "day": "31"} +{"volume": 11260, "symbol": "OKTA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 61.545, "low": 61.545, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.545, "open": 61.545, "day": "31"} +{"volume": 598, "symbol": "OKTA", "ts": "2018-08-31 10:48:00", "month": "08", "high": 61.535, "low": 61.535, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.535, "open": 61.535, "day": "31"} +{"volume": 2984, "symbol": "OKTA", "ts": "2018-08-31 10:49:00", "month": "08", "high": 61.5301, "low": 61.5301, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.5301, "open": 61.5301, "day": "31"} +{"volume": 1593, "symbol": "OKTA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 61.535, "low": 61.535, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.535, "open": 61.535, "day": "31"} +{"volume": 1917, "symbol": "OKTA", "ts": "2018-08-31 10:53:00", "month": "08", "high": 61.59, "low": 61.59, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.59, "open": 61.59, "day": "31"} +{"volume": 6014, "symbol": "OKTA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 61.73, "low": 61.63, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.73, "open": 61.63, "day": "31"} +{"volume": 6971, "symbol": "OKTA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 61.798, "low": 61.798, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.798, "open": 61.798, "day": "31"} +{"volume": 3825, "symbol": "OKTA", "ts": "2018-08-31 10:57:00", "month": "08", "high": 61.78, "low": 61.78, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.78, "open": 61.78, "day": "31"} +{"volume": 856, "symbol": "OKTA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 61.76, "low": 61.76, "key": "OKTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 61.76, "open": 61.76, "day": "31"} +{"volume": 300, "symbol": "EVBG", "ts": "2018-08-31 10:32:00", "month": "08", "high": 59.95, "low": 59.95, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.95, "open": 59.95, "day": "31"} +{"volume": 310, "symbol": "EVBG", "ts": "2018-08-31 10:33:00", "month": "08", "high": 59.785, "low": 59.785, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.785, "open": 59.785, "day": "31"} +{"volume": 355, "symbol": "EVBG", "ts": "2018-08-31 10:35:00", "month": "08", "high": 59.845, "low": 59.845, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.845, "open": 59.845, "day": "31"} +{"volume": 610, "symbol": "EVBG", "ts": "2018-08-31 10:38:00", "month": "08", "high": 59.91, "low": 59.91, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.91, "open": 59.91, "day": "31"} +{"volume": 705, "symbol": "EVBG", "ts": "2018-08-31 10:40:00", "month": "08", "high": 59.98, "low": 59.98, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.98, "open": 59.98, "day": "31"} +{"volume": 3125, "symbol": "EVBG", "ts": "2018-08-31 10:41:00", "month": "08", "high": 59.78, "low": 59.78, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.78, "open": 59.78, "day": "31"} +{"volume": 1805, "symbol": "EVBG", "ts": "2018-08-31 10:42:00", "month": "08", "high": 59.81, "low": 59.81, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.81, "open": 59.81, "day": "31"} +{"volume": 163, "symbol": "EVBG", "ts": "2018-08-31 10:44:00", "month": "08", "high": 59.89, "low": 59.89, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.89, "open": 59.89, "day": "31"} +{"volume": 203, "symbol": "EVBG", "ts": "2018-08-31 10:46:00", "month": "08", "high": 59.78, "low": 59.78, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.78, "open": 59.78, "day": "31"} +{"volume": 2081, "symbol": "EVBG", "ts": "2018-08-31 10:49:00", "month": "08", "high": 59.75, "low": 59.75, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.75, "open": 59.75, "day": "31"} +{"volume": 229, "symbol": "EVBG", "ts": "2018-08-31 10:52:00", "month": "08", "high": 59.75, "low": 59.75, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.75, "open": 59.75, "day": "31"} +{"volume": 4639, "symbol": "EVBG", "ts": "2018-08-31 10:55:00", "month": "08", "high": 59.81, "low": 59.81, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.81, "open": 59.81, "day": "31"} +{"volume": 1100, "symbol": "EVBG", "ts": "2018-08-31 10:56:00", "month": "08", "high": 59.81, "low": 59.81, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.81, "open": 59.81, "day": "31"} +{"volume": 1946, "symbol": "EVBG", "ts": "2018-08-31 10:58:00", "month": "08", "high": 59.7301, "low": 59.7301, "key": "EVBG_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 59.7301, "open": 59.7301, "day": "31"} +{"volume": 3212, "symbol": "RETA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 85.0, "low": 85.0, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.0, "open": 85.0, "day": "31"} +{"volume": 1554, "symbol": "RETA", "ts": "2018-08-31 10:33:00", "month": "08", "high": 84.9859, "low": 84.9859, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 84.9859, "open": 84.9859, "day": "31"} +{"volume": 364, "symbol": "RETA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 85.1184, "low": 85.1184, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.1184, "open": 85.1184, "day": "31"} +{"volume": 2506, "symbol": "RETA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 85.01, "low": 85.01, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.01, "open": 85.01, "day": "31"} +{"volume": 573, "symbol": "RETA", "ts": "2018-08-31 10:38:00", "month": "08", "high": 85.13, "low": 85.13, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.13, "open": 85.13, "day": "31"} +{"volume": 964, "symbol": "RETA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 85.19, "low": 85.19, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.19, "open": 85.19, "day": "31"} +{"volume": 1212, "symbol": "RETA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 85.03, "low": 85.03, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.03, "open": 85.03, "day": "31"} +{"volume": 411, "symbol": "RETA", "ts": "2018-08-31 10:44:00", "month": "08", "high": 85.3, "low": 85.3, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.3, "open": 85.3, "day": "31"} +{"volume": 255, "symbol": "RETA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 85.41, "low": 85.41, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.41, "open": 85.41, "day": "31"} +{"volume": 808, "symbol": "RETA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 85.38, "low": 85.38, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.38, "open": 85.38, "day": "31"} +{"volume": 216, "symbol": "RETA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 85.3987, "low": 85.3987, "key": "RETA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 85.3987, "open": 85.3987, "day": "31"} +{"volume": 1608, "symbol": "UIS", "ts": "2018-08-31 10:31:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 1701, "symbol": "UIS", "ts": "2018-08-31 10:32:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 3308, "symbol": "UIS", "ts": "2018-08-31 10:33:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 7766, "symbol": "UIS", "ts": "2018-08-31 10:34:00", "month": "08", "high": 18.6, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.6, "day": "31"} +{"volume": 1906, "symbol": "UIS", "ts": "2018-08-31 10:35:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 134945, "symbol": "UIS", "ts": "2018-08-31 10:37:00", "month": "08", "high": 18.65, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.6, "day": "31"} +{"volume": 6178, "symbol": "UIS", "ts": "2018-08-31 10:38:00", "month": "08", "high": 18.625, "low": 18.625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.625, "day": "31"} +{"volume": 5612, "symbol": "UIS", "ts": "2018-08-31 10:39:00", "month": "08", "high": 18.65, "low": 18.65, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.65, "open": 18.65, "day": "31"} +{"volume": 11608, "symbol": "UIS", "ts": "2018-08-31 10:40:00", "month": "08", "high": 18.625, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.6, "day": "31"} +{"volume": 1101, "symbol": "UIS", "ts": "2018-08-31 10:41:00", "month": "08", "high": 18.625, "low": 18.625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.625, "day": "31"} +{"volume": 1863, "symbol": "UIS", "ts": "2018-08-31 10:42:00", "month": "08", "high": 18.625, "low": 18.625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.625, "day": "31"} +{"volume": 3001, "symbol": "UIS", "ts": "2018-08-31 10:43:00", "month": "08", "high": 18.625, "low": 18.625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.625, "day": "31"} +{"volume": 4345, "symbol": "UIS", "ts": "2018-08-31 10:44:00", "month": "08", "high": 18.65, "low": 18.625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.65, "day": "31"} +{"volume": 1211, "symbol": "UIS", "ts": "2018-08-31 10:45:00", "month": "08", "high": 18.65, "low": 18.65, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.65, "open": 18.65, "day": "31"} +{"volume": 1696, "symbol": "UIS", "ts": "2018-08-31 10:46:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 25710, "symbol": "UIS", "ts": "2018-08-31 10:48:00", "month": "08", "high": 18.625, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.625, "day": "31"} +{"volume": 2110, "symbol": "UIS", "ts": "2018-08-31 10:49:00", "month": "08", "high": 18.5722, "low": 18.5722, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.5722, "open": 18.5722, "day": "31"} +{"volume": 35236, "symbol": "UIS", "ts": "2018-08-31 10:50:00", "month": "08", "high": 18.575, "low": 18.55, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.55, "day": "31"} +{"volume": 8253, "symbol": "UIS", "ts": "2018-08-31 10:51:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 2393, "symbol": "UIS", "ts": "2018-08-31 10:53:00", "month": "08", "high": 18.55, "low": 18.55, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.55, "open": 18.55, "day": "31"} +{"volume": 20354, "symbol": "UIS", "ts": "2018-08-31 10:54:00", "month": "08", "high": 18.6, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 2008, "symbol": "UIS", "ts": "2018-08-31 10:55:00", "month": "08", "high": 18.575, "low": 18.575, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.575, "open": 18.575, "day": "31"} +{"volume": 1320, "symbol": "UIS", "ts": "2018-08-31 10:57:00", "month": "08", "high": 18.6, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.6, "open": 18.6, "day": "31"} +{"volume": 27730, "symbol": "UIS", "ts": "2018-08-31 10:58:00", "month": "08", "high": 18.625, "low": 18.6, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.6, "day": "31"} +{"volume": 6794, "symbol": "UIS", "ts": "2018-08-31 10:59:00", "month": "08", "high": 18.625, "low": 18.625, "key": "UIS_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 18.625, "open": 18.625, "day": "31"} +{"volume": 3694, "symbol": "ETSY", "ts": "2018-08-31 10:31:00", "month": "08", "high": 48.7, "low": 48.6478, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.7, "open": 48.6478, "day": "31"} +{"volume": 507, "symbol": "ETSY", "ts": "2018-08-31 10:32:00", "month": "08", "high": 48.66, "low": 48.66, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.66, "open": 48.66, "day": "31"} +{"volume": 964, "symbol": "ETSY", "ts": "2018-08-31 10:35:00", "month": "08", "high": 48.63, "low": 48.63, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.63, "open": 48.63, "day": "31"} +{"volume": 1392, "symbol": "ETSY", "ts": "2018-08-31 10:36:00", "month": "08", "high": 48.64, "low": 48.64, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.64, "open": 48.64, "day": "31"} +{"volume": 1543, "symbol": "ETSY", "ts": "2018-08-31 10:37:00", "month": "08", "high": 48.63, "low": 48.63, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.63, "open": 48.63, "day": "31"} +{"volume": 2119, "symbol": "ETSY", "ts": "2018-08-31 10:38:00", "month": "08", "high": 48.685, "low": 48.685, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.685, "open": 48.685, "day": "31"} +{"volume": 1151, "symbol": "ETSY", "ts": "2018-08-31 10:40:00", "month": "08", "high": 48.68, "low": 48.68, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.68, "open": 48.68, "day": "31"} +{"volume": 2376, "symbol": "ETSY", "ts": "2018-08-31 10:42:00", "month": "08", "high": 48.7667, "low": 48.7667, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.7667, "open": 48.7667, "day": "31"} +{"volume": 7316, "symbol": "ETSY", "ts": "2018-08-31 10:43:00", "month": "08", "high": 48.91, "low": 48.79, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.91, "open": 48.79, "day": "31"} +{"volume": 2125, "symbol": "ETSY", "ts": "2018-08-31 10:44:00", "month": "08", "high": 48.9, "low": 48.9, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.9, "open": 48.9, "day": "31"} +{"volume": 534, "symbol": "ETSY", "ts": "2018-08-31 10:45:00", "month": "08", "high": 48.875, "low": 48.875, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.875, "open": 48.875, "day": "31"} +{"volume": 1943, "symbol": "ETSY", "ts": "2018-08-31 10:47:00", "month": "08", "high": 48.89, "low": 48.89, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.89, "open": 48.89, "day": "31"} +{"volume": 2031, "symbol": "ETSY", "ts": "2018-08-31 10:48:00", "month": "08", "high": 48.88, "low": 48.88, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.88, "open": 48.88, "day": "31"} +{"volume": 1578, "symbol": "ETSY", "ts": "2018-08-31 10:49:00", "month": "08", "high": 48.92, "low": 48.92, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.92, "open": 48.92, "day": "31"} +{"volume": 3288, "symbol": "ETSY", "ts": "2018-08-31 10:50:00", "month": "08", "high": 48.91, "low": 48.91, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.91, "open": 48.91, "day": "31"} +{"volume": 6712, "symbol": "ETSY", "ts": "2018-08-31 10:51:00", "month": "08", "high": 48.89, "low": 48.89, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.89, "open": 48.89, "day": "31"} +{"volume": 3856, "symbol": "ETSY", "ts": "2018-08-31 10:52:00", "month": "08", "high": 48.77, "low": 48.77, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.77, "open": 48.77, "day": "31"} +{"volume": 3697, "symbol": "ETSY", "ts": "2018-08-31 10:54:00", "month": "08", "high": 48.82, "low": 48.82, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.82, "open": 48.82, "day": "31"} +{"volume": 1558, "symbol": "ETSY", "ts": "2018-08-31 10:56:00", "month": "08", "high": 48.84, "low": 48.84, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.84, "open": 48.84, "day": "31"} +{"volume": 383, "symbol": "ETSY", "ts": "2018-08-31 10:57:00", "month": "08", "high": 48.8098, "low": 48.8098, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.8098, "open": 48.8098, "day": "31"} +{"volume": 2372, "symbol": "ETSY", "ts": "2018-08-31 10:58:00", "month": "08", "high": 48.83, "low": 48.83, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.83, "open": 48.83, "day": "31"} +{"volume": 1607, "symbol": "ETSY", "ts": "2018-08-31 10:59:00", "month": "08", "high": 48.87, "low": 48.87, "key": "ETSY_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 48.87, "open": 48.87, "day": "31"} +{"volume": 6690, "symbol": "TWLO", "ts": "2018-08-31 10:31:00", "month": "08", "high": 80.22, "low": 80.04, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.04, "open": 80.22, "day": "31"} +{"volume": 8338, "symbol": "TWLO", "ts": "2018-08-31 10:32:00", "month": "08", "high": 79.96, "low": 79.86, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 79.86, "open": 79.96, "day": "31"} +{"volume": 1982, "symbol": "TWLO", "ts": "2018-08-31 10:34:00", "month": "08", "high": 79.87, "low": 79.87, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 79.87, "open": 79.87, "day": "31"} +{"volume": 3162, "symbol": "TWLO", "ts": "2018-08-31 10:35:00", "month": "08", "high": 79.8005, "low": 79.8005, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 79.8005, "open": 79.8005, "day": "31"} +{"volume": 1116, "symbol": "TWLO", "ts": "2018-08-31 10:36:00", "month": "08", "high": 79.84, "low": 79.84, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 79.84, "open": 79.84, "day": "31"} +{"volume": 2097, "symbol": "TWLO", "ts": "2018-08-31 10:38:00", "month": "08", "high": 79.8647, "low": 79.8647, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 79.8647, "open": 79.8647, "day": "31"} +{"volume": 2960, "symbol": "TWLO", "ts": "2018-08-31 10:39:00", "month": "08", "high": 79.78, "low": 79.78, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 79.78, "open": 79.78, "day": "31"} +{"volume": 13049, "symbol": "TWLO", "ts": "2018-08-31 10:40:00", "month": "08", "high": 80.11, "low": 80.02, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.11, "open": 80.02, "day": "31"} +{"volume": 2820, "symbol": "TWLO", "ts": "2018-08-31 10:42:00", "month": "08", "high": 80.14, "low": 80.075, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.075, "open": 80.14, "day": "31"} +{"volume": 1966, "symbol": "TWLO", "ts": "2018-08-31 10:45:00", "month": "08", "high": 80.2, "low": 80.2, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.2, "open": 80.2, "day": "31"} +{"volume": 2209, "symbol": "TWLO", "ts": "2018-08-31 10:46:00", "month": "08", "high": 80.28, "low": 80.28, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.28, "open": 80.28, "day": "31"} +{"volume": 3548, "symbol": "TWLO", "ts": "2018-08-31 10:47:00", "month": "08", "high": 80.15, "low": 80.15, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.15, "open": 80.15, "day": "31"} +{"volume": 2332, "symbol": "TWLO", "ts": "2018-08-31 10:48:00", "month": "08", "high": 80.021, "low": 80.021, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.021, "open": 80.021, "day": "31"} +{"volume": 3740, "symbol": "TWLO", "ts": "2018-08-31 10:49:00", "month": "08", "high": 80.05, "low": 80.05, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.05, "open": 80.05, "day": "31"} +{"volume": 2436, "symbol": "TWLO", "ts": "2018-08-31 10:50:00", "month": "08", "high": 80.13, "low": 80.13, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.13, "open": 80.13, "day": "31"} +{"volume": 1985, "symbol": "TWLO", "ts": "2018-08-31 10:51:00", "month": "08", "high": 80.16, "low": 80.16, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.16, "open": 80.16, "day": "31"} +{"volume": 1257, "symbol": "TWLO", "ts": "2018-08-31 10:52:00", "month": "08", "high": 80.24, "low": 80.24, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.24, "open": 80.24, "day": "31"} +{"volume": 1704, "symbol": "TWLO", "ts": "2018-08-31 10:54:00", "month": "08", "high": 80.259, "low": 80.259, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.259, "open": 80.259, "day": "31"} +{"volume": 2661, "symbol": "TWLO", "ts": "2018-08-31 10:57:00", "month": "08", "high": 80.21, "low": 80.21, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.21, "open": 80.21, "day": "31"} +{"volume": 687, "symbol": "TWLO", "ts": "2018-08-31 10:58:00", "month": "08", "high": 80.22, "low": 80.22, "key": "TWLO_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 80.22, "open": 80.22, "day": "31"} +{"volume": 2924, "symbol": "AMED", "ts": "2018-08-31 10:31:00", "month": "08", "high": 121.485, "low": 121.485, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.485, "open": 121.485, "day": "31"} +{"volume": 1372, "symbol": "AMED", "ts": "2018-08-31 10:32:00", "month": "08", "high": 121.38, "low": 121.38, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.38, "open": 121.38, "day": "31"} +{"volume": 244, "symbol": "AMED", "ts": "2018-08-31 10:36:00", "month": "08", "high": 121.23, "low": 121.23, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.23, "open": 121.23, "day": "31"} +{"volume": 3518, "symbol": "AMED", "ts": "2018-08-31 10:38:00", "month": "08", "high": 121.5833, "low": 121.5833, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.5833, "open": 121.5833, "day": "31"} +{"volume": 8575, "symbol": "AMED", "ts": "2018-08-31 10:39:00", "month": "08", "high": 121.55, "low": 121.456, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.55, "open": 121.456, "day": "31"} +{"volume": 2363, "symbol": "AMED", "ts": "2018-08-31 10:40:00", "month": "08", "high": 121.53, "low": 121.53, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.53, "open": 121.53, "day": "31"} +{"volume": 1965, "symbol": "AMED", "ts": "2018-08-31 10:41:00", "month": "08", "high": 121.845, "low": 121.845, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.845, "open": 121.845, "day": "31"} +{"volume": 2231, "symbol": "AMED", "ts": "2018-08-31 10:43:00", "month": "08", "high": 121.7, "low": 121.7, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.7, "open": 121.7, "day": "31"} +{"volume": 1002, "symbol": "AMED", "ts": "2018-08-31 10:45:00", "month": "08", "high": 121.76, "low": 121.68, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.68, "open": 121.76, "day": "31"} +{"volume": 685, "symbol": "AMED", "ts": "2018-08-31 10:48:00", "month": "08", "high": 121.75, "low": 121.75, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.75, "open": 121.75, "day": "31"} +{"volume": 487, "symbol": "AMED", "ts": "2018-08-31 10:51:00", "month": "08", "high": 121.76, "low": 121.76, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.76, "open": 121.76, "day": "31"} +{"volume": 1356, "symbol": "AMED", "ts": "2018-08-31 10:53:00", "month": "08", "high": 121.9472, "low": 121.865, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.9472, "open": 121.865, "day": "31"} +{"volume": 1860, "symbol": "AMED", "ts": "2018-08-31 10:56:00", "month": "08", "high": 121.945, "low": 121.945, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.945, "open": 121.945, "day": "31"} +{"volume": 4572, "symbol": "AMED", "ts": "2018-08-31 10:57:00", "month": "08", "high": 122.1, "low": 122.015, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 122.015, "open": 122.1, "day": "31"} +{"volume": 1427, "symbol": "AMED", "ts": "2018-08-31 10:58:00", "month": "08", "high": 121.915, "low": 121.915, "key": "AMED_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 121.915, "open": 121.915, "day": "31"} +{"volume": 406, "symbol": "USAT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 16.1, "low": 16.1, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.1, "open": 16.1, "day": "31"} +{"volume": 3020, "symbol": "USAT", "ts": "2018-08-31 10:35:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 1150, "symbol": "USAT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 16.025, "low": 16.025, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.025, "open": 16.025, "day": "31"} +{"volume": 205, "symbol": "USAT", "ts": "2018-08-31 10:41:00", "month": "08", "high": 16.025, "low": 16.025, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.025, "open": 16.025, "day": "31"} +{"volume": 5495, "symbol": "USAT", "ts": "2018-08-31 10:44:00", "month": "08", "high": 16.095, "low": 16.025, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.095, "open": 16.025, "day": "31"} +{"volume": 450, "symbol": "USAT", "ts": "2018-08-31 10:46:00", "month": "08", "high": 16.0817, "low": 16.05, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.0817, "open": 16.05, "day": "31"} +{"volume": 1217, "symbol": "USAT", "ts": "2018-08-31 10:49:00", "month": "08", "high": 16.01, "low": 16.01, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.01, "open": 16.01, "day": "31"} +{"volume": 500, "symbol": "USAT", "ts": "2018-08-31 10:51:00", "month": "08", "high": 16.0, "low": 16.0, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.0, "open": 16.0, "day": "31"} +{"volume": 200, "symbol": "USAT", "ts": "2018-08-31 10:54:00", "month": "08", "high": 16.025, "low": 16.025, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.025, "open": 16.025, "day": "31"} +{"volume": 3610, "symbol": "USAT", "ts": "2018-08-31 10:56:00", "month": "08", "high": 16.091, "low": 16.05, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.091, "day": "31"} +{"volume": 517, "symbol": "USAT", "ts": "2018-08-31 10:58:00", "month": "08", "high": 16.05, "low": 16.05, "key": "USAT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 16.05, "open": 16.05, "day": "31"} +{"volume": 532, "symbol": "RGNX", "ts": "2018-08-31 10:31:00", "month": "08", "high": 71.15, "low": 71.15, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.15, "open": 71.15, "day": "31"} +{"volume": 1601, "symbol": "RGNX", "ts": "2018-08-31 10:33:00", "month": "08", "high": 71.1, "low": 71.1, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.1, "open": 71.1, "day": "31"} +{"volume": 3360, "symbol": "RGNX", "ts": "2018-08-31 10:35:00", "month": "08", "high": 71.0, "low": 71.0, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.0, "open": 71.0, "day": "31"} +{"volume": 276, "symbol": "RGNX", "ts": "2018-08-31 10:37:00", "month": "08", "high": 71.15, "low": 71.15, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.15, "open": 71.15, "day": "31"} +{"volume": 2095, "symbol": "RGNX", "ts": "2018-08-31 10:40:00", "month": "08", "high": 71.165, "low": 71.165, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.165, "open": 71.165, "day": "31"} +{"volume": 1605, "symbol": "RGNX", "ts": "2018-08-31 10:41:00", "month": "08", "high": 71.35, "low": 71.35, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.35, "open": 71.35, "day": "31"} +{"volume": 480, "symbol": "RGNX", "ts": "2018-08-31 10:43:00", "month": "08", "high": 71.35, "low": 71.35, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.35, "open": 71.35, "day": "31"} +{"volume": 850, "symbol": "RGNX", "ts": "2018-08-31 10:46:00", "month": "08", "high": 71.4, "low": 71.4, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.4, "open": 71.4, "day": "31"} +{"volume": 1213, "symbol": "RGNX", "ts": "2018-08-31 10:47:00", "month": "08", "high": 71.15, "low": 71.15, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.15, "open": 71.15, "day": "31"} +{"volume": 2590, "symbol": "RGNX", "ts": "2018-08-31 10:48:00", "month": "08", "high": 70.929, "low": 70.929, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.929, "open": 70.929, "day": "31"} +{"volume": 855, "symbol": "RGNX", "ts": "2018-08-31 10:50:00", "month": "08", "high": 70.85, "low": 70.85, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.85, "open": 70.85, "day": "31"} +{"volume": 1008, "symbol": "RGNX", "ts": "2018-08-31 10:51:00", "month": "08", "high": 70.925, "low": 70.925, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 70.925, "open": 70.925, "day": "31"} +{"volume": 2349, "symbol": "RGNX", "ts": "2018-08-31 10:52:00", "month": "08", "high": 71.0, "low": 71.0, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.0, "open": 71.0, "day": "31"} +{"volume": 283, "symbol": "RGNX", "ts": "2018-08-31 10:54:00", "month": "08", "high": 71.15, "low": 71.15, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.15, "open": 71.15, "day": "31"} +{"volume": 651, "symbol": "RGNX", "ts": "2018-08-31 10:56:00", "month": "08", "high": 71.1, "low": 71.1, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.1, "open": 71.1, "day": "31"} +{"volume": 2077, "symbol": "RGNX", "ts": "2018-08-31 10:58:00", "month": "08", "high": 71.05, "low": 71.05, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.05, "open": 71.05, "day": "31"} +{"volume": 667, "symbol": "RGNX", "ts": "2018-08-31 10:59:00", "month": "08", "high": 71.0, "low": 71.0, "key": "RGNX_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 71.0, "open": 71.0, "day": "31"} +{"volume": 507, "symbol": "HAE", "ts": "2018-08-31 10:31:00", "month": "08", "high": 109.47, "low": 109.47, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.47, "open": 109.47, "day": "31"} +{"volume": 1211, "symbol": "HAE", "ts": "2018-08-31 10:35:00", "month": "08", "high": 109.53, "low": 109.53, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.53, "open": 109.53, "day": "31"} +{"volume": 1647, "symbol": "HAE", "ts": "2018-08-31 10:41:00", "month": "08", "high": 109.949, "low": 109.949, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 109.949, "open": 109.949, "day": "31"} +{"volume": 1028, "symbol": "HAE", "ts": "2018-08-31 10:43:00", "month": "08", "high": 110.0, "low": 110.0, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.0, "open": 110.0, "day": "31"} +{"volume": 296, "symbol": "HAE", "ts": "2018-08-31 10:44:00", "month": "08", "high": 110.01, "low": 110.01, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.01, "open": 110.01, "day": "31"} +{"volume": 1131, "symbol": "HAE", "ts": "2018-08-31 10:45:00", "month": "08", "high": 110.2, "low": 110.2, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.2, "open": 110.2, "day": "31"} +{"volume": 127, "symbol": "HAE", "ts": "2018-08-31 10:47:00", "month": "08", "high": 110.18, "low": 110.18, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.18, "open": 110.18, "day": "31"} +{"volume": 894, "symbol": "HAE", "ts": "2018-08-31 10:50:00", "month": "08", "high": 110.14, "low": 110.14, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.14, "open": 110.14, "day": "31"} +{"volume": 818, "symbol": "HAE", "ts": "2018-08-31 10:52:00", "month": "08", "high": 110.11, "low": 110.11, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.11, "open": 110.11, "day": "31"} +{"volume": 469, "symbol": "HAE", "ts": "2018-08-31 10:55:00", "month": "08", "high": 110.1091, "low": 110.1091, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.1091, "open": 110.1091, "day": "31"} +{"volume": 363, "symbol": "HAE", "ts": "2018-08-31 10:58:00", "month": "08", "high": 110.16, "low": 110.16, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.16, "open": 110.16, "day": "31"} +{"volume": 759, "symbol": "HAE", "ts": "2018-08-31 10:59:00", "month": "08", "high": 110.24, "low": 110.24, "key": "HAE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 110.24, "open": 110.24, "day": "31"} +{"volume": 4356, "symbol": "NVTA", "ts": "2018-08-31 10:31:00", "month": "08", "high": 14.9797, "low": 14.9797, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9797, "open": 14.9797, "day": "31"} +{"volume": 2743, "symbol": "NVTA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 14.92, "low": 14.92, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.92, "open": 14.92, "day": "31"} +{"volume": 3956, "symbol": "NVTA", "ts": "2018-08-31 10:33:00", "month": "08", "high": 14.89, "low": 14.89, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.89, "open": 14.89, "day": "31"} +{"volume": 2832, "symbol": "NVTA", "ts": "2018-08-31 10:35:00", "month": "08", "high": 14.89, "low": 14.89, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.89, "open": 14.89, "day": "31"} +{"volume": 2625, "symbol": "NVTA", "ts": "2018-08-31 10:36:00", "month": "08", "high": 14.94, "low": 14.94, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.94, "day": "31"} +{"volume": 194, "symbol": "NVTA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 14.9441, "low": 14.9441, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9441, "open": 14.9441, "day": "31"} +{"volume": 1017, "symbol": "NVTA", "ts": "2018-08-31 10:39:00", "month": "08", "high": 14.955, "low": 14.955, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.955, "open": 14.955, "day": "31"} +{"volume": 3638, "symbol": "NVTA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 14.94, "low": 14.94, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.94, "day": "31"} +{"volume": 1624, "symbol": "NVTA", "ts": "2018-08-31 10:41:00", "month": "08", "high": 15.0, "low": 15.0, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 15.0, "open": 15.0, "day": "31"} +{"volume": 2401, "symbol": "NVTA", "ts": "2018-08-31 10:42:00", "month": "08", "high": 14.955, "low": 14.955, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.955, "open": 14.955, "day": "31"} +{"volume": 681, "symbol": "NVTA", "ts": "2018-08-31 10:44:00", "month": "08", "high": 14.94, "low": 14.94, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.94, "day": "31"} +{"volume": 1387, "symbol": "NVTA", "ts": "2018-08-31 10:46:00", "month": "08", "high": 14.94, "low": 14.94, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.94, "day": "31"} +{"volume": 3612, "symbol": "NVTA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 14.98, "low": 14.98, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.98, "open": 14.98, "day": "31"} +{"volume": 14569, "symbol": "NVTA", "ts": "2018-08-31 10:48:00", "month": "08", "high": 15.0, "low": 14.98, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.98, "open": 14.99, "day": "31"} +{"volume": 4730, "symbol": "NVTA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 14.95, "low": 14.95, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.95, "open": 14.95, "day": "31"} +{"volume": 2523, "symbol": "NVTA", "ts": "2018-08-31 10:51:00", "month": "08", "high": 14.9628, "low": 14.9628, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9628, "open": 14.9628, "day": "31"} +{"volume": 3211, "symbol": "NVTA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 14.985, "low": 14.985, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.985, "open": 14.985, "day": "31"} +{"volume": 6040, "symbol": "NVTA", "ts": "2018-08-31 10:53:00", "month": "08", "high": 14.9601, "low": 14.9601, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9601, "open": 14.9601, "day": "31"} +{"volume": 2637, "symbol": "NVTA", "ts": "2018-08-31 10:54:00", "month": "08", "high": 14.94, "low": 14.94, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.94, "open": 14.94, "day": "31"} +{"volume": 2768, "symbol": "NVTA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 14.89, "low": 14.89, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.89, "open": 14.89, "day": "31"} +{"volume": 8710, "symbol": "NVTA", "ts": "2018-08-31 10:56:00", "month": "08", "high": 14.9, "low": 14.8762, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.9, "open": 14.8762, "day": "31"} +{"volume": 1560, "symbol": "NVTA", "ts": "2018-08-31 10:58:00", "month": "08", "high": 14.92, "low": 14.92, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.92, "open": 14.92, "day": "31"} +{"volume": 8034, "symbol": "NVTA", "ts": "2018-08-31 10:59:00", "month": "08", "high": 14.88, "low": 14.85, "key": "NVTA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 14.85, "open": 14.88, "day": "31"} +{"volume": 422, "symbol": "TPB", "ts": "2018-08-31 10:35:00", "month": "08", "high": 33.96, "low": 33.84, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.84, "open": 33.96, "day": "31"} +{"volume": 131, "symbol": "TPB", "ts": "2018-08-31 10:39:00", "month": "08", "high": 33.89, "low": 33.89, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.89, "open": 33.89, "day": "31"} +{"volume": 483, "symbol": "TPB", "ts": "2018-08-31 10:44:00", "month": "08", "high": 33.7, "low": 33.7, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.7, "open": 33.7, "day": "31"} +{"volume": 445, "symbol": "TPB", "ts": "2018-08-31 10:48:00", "month": "08", "high": 33.82, "low": 33.73, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.73, "open": 33.82, "day": "31"} +{"volume": 801, "symbol": "TPB", "ts": "2018-08-31 10:51:00", "month": "08", "high": 33.79, "low": 33.79, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.79, "open": 33.79, "day": "31"} +{"volume": 603, "symbol": "TPB", "ts": "2018-08-31 10:53:00", "month": "08", "high": 33.75, "low": 33.75, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.75, "open": 33.75, "day": "31"} +{"volume": 260, "symbol": "TPB", "ts": "2018-08-31 10:54:00", "month": "08", "high": 33.75, "low": 33.75, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.75, "open": 33.75, "day": "31"} +{"volume": 200, "symbol": "TPB", "ts": "2018-08-31 10:55:00", "month": "08", "high": 33.76, "low": 33.76, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.76, "open": 33.76, "day": "31"} +{"volume": 1788, "symbol": "TPB", "ts": "2018-08-31 10:56:00", "month": "08", "high": 33.74, "low": 33.74, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.74, "open": 33.74, "day": "31"} +{"volume": 664, "symbol": "TPB", "ts": "2018-08-31 10:57:00", "month": "08", "high": 33.65, "low": 33.65, "key": "TPB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 33.65, "open": 33.65, "day": "31"} +{"volume": 166, "symbol": "NTRA", "ts": "2018-08-31 10:31:00", "month": "08", "high": 27.435, "low": 27.435, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.435, "open": 27.435, "day": "31"} +{"volume": 450, "symbol": "NTRA", "ts": "2018-08-31 10:32:00", "month": "08", "high": 27.46, "low": 27.46, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.46, "open": 27.46, "day": "31"} +{"volume": 1112, "symbol": "NTRA", "ts": "2018-08-31 10:37:00", "month": "08", "high": 27.465, "low": 27.465, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.465, "open": 27.465, "day": "31"} +{"volume": 815, "symbol": "NTRA", "ts": "2018-08-31 10:40:00", "month": "08", "high": 27.51, "low": 27.51, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.51, "open": 27.51, "day": "31"} +{"volume": 1734, "symbol": "NTRA", "ts": "2018-08-31 10:43:00", "month": "08", "high": 27.51, "low": 27.51, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.51, "open": 27.51, "day": "31"} +{"volume": 3222, "symbol": "NTRA", "ts": "2018-08-31 10:46:00", "month": "08", "high": 27.54, "low": 27.54, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.54, "open": 27.54, "day": "31"} +{"volume": 1810, "symbol": "NTRA", "ts": "2018-08-31 10:47:00", "month": "08", "high": 27.51, "low": 27.51, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.51, "open": 27.51, "day": "31"} +{"volume": 1172, "symbol": "NTRA", "ts": "2018-08-31 10:50:00", "month": "08", "high": 27.55, "low": 27.55, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.55, "open": 27.55, "day": "31"} +{"volume": 1303, "symbol": "NTRA", "ts": "2018-08-31 10:52:00", "month": "08", "high": 27.58, "low": 27.58, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.58, "open": 27.58, "day": "31"} +{"volume": 1716, "symbol": "NTRA", "ts": "2018-08-31 10:53:00", "month": "08", "high": 27.58, "low": 27.55, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.55, "open": 27.58, "day": "31"} +{"volume": 1010, "symbol": "NTRA", "ts": "2018-08-31 10:55:00", "month": "08", "high": 27.56, "low": 27.56, "key": "NTRA_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 27.56, "open": 27.56, "day": "31"} +{"volume": 2821, "symbol": "WTI", "ts": "2018-08-31 10:32:00", "month": "08", "high": 6.81, "low": 6.81, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.81, "open": 6.81, "day": "31"} +{"volume": 702, "symbol": "WTI", "ts": "2018-08-31 10:33:00", "month": "08", "high": 6.815, "low": 6.815, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.815, "open": 6.815, "day": "31"} +{"volume": 1478, "symbol": "WTI", "ts": "2018-08-31 10:35:00", "month": "08", "high": 6.805, "low": 6.805, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.805, "open": 6.805, "day": "31"} +{"volume": 4920, "symbol": "WTI", "ts": "2018-08-31 10:37:00", "month": "08", "high": 6.815, "low": 6.815, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.815, "open": 6.815, "day": "31"} +{"volume": 11318, "symbol": "WTI", "ts": "2018-08-31 10:38:00", "month": "08", "high": 6.8211, "low": 6.8211, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.8211, "open": 6.8211, "day": "31"} +{"volume": 1700, "symbol": "WTI", "ts": "2018-08-31 10:39:00", "month": "08", "high": 6.83, "low": 6.83, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.83, "open": 6.83, "day": "31"} +{"volume": 3434, "symbol": "WTI", "ts": "2018-08-31 10:40:00", "month": "08", "high": 6.84, "low": 6.84, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.84, "open": 6.84, "day": "31"} +{"volume": 3705, "symbol": "WTI", "ts": "2018-08-31 10:41:00", "month": "08", "high": 6.84, "low": 6.84, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.84, "open": 6.84, "day": "31"} +{"volume": 6258, "symbol": "WTI", "ts": "2018-08-31 10:42:00", "month": "08", "high": 6.84, "low": 6.84, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.84, "open": 6.84, "day": "31"} +{"volume": 4497, "symbol": "WTI", "ts": "2018-08-31 10:43:00", "month": "08", "high": 6.84, "low": 6.84, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.84, "open": 6.84, "day": "31"} +{"volume": 11028, "symbol": "WTI", "ts": "2018-08-31 10:45:00", "month": "08", "high": 6.85, "low": 6.84, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.85, "open": 6.84, "day": "31"} +{"volume": 5540, "symbol": "WTI", "ts": "2018-08-31 10:47:00", "month": "08", "high": 6.83, "low": 6.83, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.83, "open": 6.83, "day": "31"} +{"volume": 5024, "symbol": "WTI", "ts": "2018-08-31 10:48:00", "month": "08", "high": 6.815, "low": 6.815, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.815, "open": 6.815, "day": "31"} +{"volume": 6479, "symbol": "WTI", "ts": "2018-08-31 10:50:00", "month": "08", "high": 6.82, "low": 6.82, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.82, "open": 6.82, "day": "31"} +{"volume": 1900, "symbol": "WTI", "ts": "2018-08-31 10:52:00", "month": "08", "high": 6.815, "low": 6.815, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.815, "open": 6.815, "day": "31"} +{"volume": 5554, "symbol": "WTI", "ts": "2018-08-31 10:53:00", "month": "08", "high": 6.8, "low": 6.8, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.8, "open": 6.8, "day": "31"} +{"volume": 7268, "symbol": "WTI", "ts": "2018-08-31 10:54:00", "month": "08", "high": 6.79, "low": 6.79, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.79, "open": 6.79, "day": "31"} +{"volume": 503, "symbol": "WTI", "ts": "2018-08-31 10:55:00", "month": "08", "high": 6.795, "low": 6.795, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.795, "open": 6.795, "day": "31"} +{"volume": 9320, "symbol": "WTI", "ts": "2018-08-31 10:57:00", "month": "08", "high": 6.805, "low": 6.8, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.805, "open": 6.8, "day": "31"} +{"volume": 800, "symbol": "WTI", "ts": "2018-08-31 10:59:00", "month": "08", "high": 6.8, "low": 6.8, "key": "WTI_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 6.8, "open": 6.8, "day": "31"} +{"volume": 1934, "symbol": "RCM", "ts": "2018-08-31 10:31:00", "month": "08", "high": 9.96, "low": 9.96, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.96, "open": 9.96, "day": "31"} +{"volume": 803, "symbol": "RCM", "ts": "2018-08-31 10:32:00", "month": "08", "high": 9.94, "low": 9.94, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.94, "open": 9.94, "day": "31"} +{"volume": 3032, "symbol": "RCM", "ts": "2018-08-31 10:33:00", "month": "08", "high": 9.88, "low": 9.88, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.88, "open": 9.88, "day": "31"} +{"volume": 2519, "symbol": "RCM", "ts": "2018-08-31 10:34:00", "month": "08", "high": 9.9, "low": 9.9, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.9, "day": "31"} +{"volume": 100, "symbol": "RCM", "ts": "2018-08-31 10:37:00", "month": "08", "high": 9.9, "low": 9.9, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.9, "day": "31"} +{"volume": 100, "symbol": "RCM", "ts": "2018-08-31 10:38:00", "month": "08", "high": 9.91, "low": 9.91, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.91, "open": 9.91, "day": "31"} +{"volume": 100, "symbol": "RCM", "ts": "2018-08-31 10:39:00", "month": "08", "high": 9.905, "low": 9.905, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.905, "open": 9.905, "day": "31"} +{"volume": 2424, "symbol": "RCM", "ts": "2018-08-31 10:41:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 300, "symbol": "RCM", "ts": "2018-08-31 10:43:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 970, "symbol": "RCM", "ts": "2018-08-31 10:45:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 500, "symbol": "RCM", "ts": "2018-08-31 10:46:00", "month": "08", "high": 9.93, "low": 9.93, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.93, "open": 9.93, "day": "31"} +{"volume": 1301, "symbol": "RCM", "ts": "2018-08-31 10:47:00", "month": "08", "high": 9.9, "low": 9.9, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.9, "day": "31"} +{"volume": 700, "symbol": "RCM", "ts": "2018-08-31 10:50:00", "month": "08", "high": 9.91, "low": 9.91, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.91, "open": 9.91, "day": "31"} +{"volume": 545, "symbol": "RCM", "ts": "2018-08-31 10:53:00", "month": "08", "high": 9.905, "low": 9.905, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.905, "open": 9.905, "day": "31"} +{"volume": 2445, "symbol": "RCM", "ts": "2018-08-31 10:54:00", "month": "08", "high": 9.9, "low": 9.9, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.9, "day": "31"} +{"volume": 102, "symbol": "RCM", "ts": "2018-08-31 10:55:00", "month": "08", "high": 9.89, "low": 9.89, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.89, "open": 9.89, "day": "31"} +{"volume": 400, "symbol": "RCM", "ts": "2018-08-31 10:58:00", "month": "08", "high": 9.9, "low": 9.9, "key": "RCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 9.9, "open": 9.9, "day": "31"} +{"volume": 33730, "symbol": "LULU", "ts": "2018-08-31 10:31:00", "month": "08", "high": 156.3732, "low": 156.072, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.072, "open": 156.26, "day": "31"} +{"volume": 57524, "symbol": "LULU", "ts": "2018-08-31 10:32:00", "month": "08", "high": 156.0, "low": 155.5101, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 155.86, "open": 156.0, "day": "31"} +{"volume": 25573, "symbol": "LULU", "ts": "2018-08-31 10:33:00", "month": "08", "high": 156.09, "low": 155.83, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 155.85, "open": 155.84, "day": "31"} +{"volume": 35599, "symbol": "LULU", "ts": "2018-08-31 10:34:00", "month": "08", "high": 156.3, "low": 155.8692, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.3, "open": 155.8692, "day": "31"} +{"volume": 42417, "symbol": "LULU", "ts": "2018-08-31 10:35:00", "month": "08", "high": 156.25, "low": 156.04, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.2, "open": 156.25, "day": "31"} +{"volume": 28511, "symbol": "LULU", "ts": "2018-08-31 10:36:00", "month": "08", "high": 156.23, "low": 156.036, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.036, "open": 156.23, "day": "31"} +{"volume": 54486, "symbol": "LULU", "ts": "2018-08-31 10:37:00", "month": "08", "high": 156.25, "low": 155.781, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.25, "open": 156.02, "day": "31"} +{"volume": 18273, "symbol": "LULU", "ts": "2018-08-31 10:38:00", "month": "08", "high": 156.1002, "low": 155.96, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.1002, "open": 156.031, "day": "31"} +{"volume": 61601, "symbol": "LULU", "ts": "2018-08-31 10:39:00", "month": "08", "high": 156.7204, "low": 156.17, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.6501, "open": 156.1732, "day": "31"} +{"volume": 69703, "symbol": "LULU", "ts": "2018-08-31 10:40:00", "month": "08", "high": 157.0599, "low": 156.65, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.94, "open": 156.65, "day": "31"} +{"volume": 63964, "symbol": "LULU", "ts": "2018-08-31 10:41:00", "month": "08", "high": 157.25, "low": 156.92, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.166, "open": 157.036, "day": "31"} +{"volume": 30686, "symbol": "LULU", "ts": "2018-08-31 10:42:00", "month": "08", "high": 157.1602, "low": 156.9, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.96, "open": 157.1602, "day": "31"} +{"volume": 22665, "symbol": "LULU", "ts": "2018-08-31 10:43:00", "month": "08", "high": 156.89, "low": 156.6, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.6, "open": 156.89, "day": "31"} +{"volume": 25078, "symbol": "LULU", "ts": "2018-08-31 10:44:00", "month": "08", "high": 156.669, "low": 156.4, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.669, "open": 156.58, "day": "31"} +{"volume": 22641, "symbol": "LULU", "ts": "2018-08-31 10:45:00", "month": "08", "high": 157.01, "low": 156.69, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.92, "open": 156.69, "day": "31"} +{"volume": 19390, "symbol": "LULU", "ts": "2018-08-31 10:46:00", "month": "08", "high": 157.0546, "low": 156.78, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.9201, "open": 156.78, "day": "31"} +{"volume": 14016, "symbol": "LULU", "ts": "2018-08-31 10:47:00", "month": "08", "high": 157.14, "low": 157.03, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.03, "open": 157.14, "day": "31"} +{"volume": 19261, "symbol": "LULU", "ts": "2018-08-31 10:48:00", "month": "08", "high": 157.02, "low": 156.9616, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.0, "open": 156.965, "day": "31"} +{"volume": 10711, "symbol": "LULU", "ts": "2018-08-31 10:49:00", "month": "08", "high": 157.01, "low": 156.97, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 156.97, "open": 157.01, "day": "31"} +{"volume": 29480, "symbol": "LULU", "ts": "2018-08-31 10:50:00", "month": "08", "high": 157.2099, "low": 157.03, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.1534, "open": 157.03, "day": "31"} +{"volume": 53113, "symbol": "LULU", "ts": "2018-08-31 10:51:00", "month": "08", "high": 157.41, "low": 157.14, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.3981, "open": 157.14, "day": "31"} +{"volume": 30904, "symbol": "LULU", "ts": "2018-08-31 10:52:00", "month": "08", "high": 157.45, "low": 157.1678, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.31, "open": 157.38, "day": "31"} +{"volume": 36689, "symbol": "LULU", "ts": "2018-08-31 10:53:00", "month": "08", "high": 157.43, "low": 157.12, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.23, "open": 157.37, "day": "31"} +{"volume": 14666, "symbol": "LULU", "ts": "2018-08-31 10:54:00", "month": "08", "high": 157.21, "low": 157.1, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.13, "open": 157.21, "day": "31"} +{"volume": 12125, "symbol": "LULU", "ts": "2018-08-31 10:55:00", "month": "08", "high": 157.18, "low": 157.12, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.13, "open": 157.12, "day": "31"} +{"volume": 11068, "symbol": "LULU", "ts": "2018-08-31 10:56:00", "month": "08", "high": 157.21, "low": 157.175, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.21, "open": 157.175, "day": "31"} +{"volume": 15854, "symbol": "LULU", "ts": "2018-08-31 10:57:00", "month": "08", "high": 157.3032, "low": 157.17, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.3032, "open": 157.19, "day": "31"} +{"volume": 26887, "symbol": "LULU", "ts": "2018-08-31 10:58:00", "month": "08", "high": 157.46, "low": 157.255, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.41, "open": 157.255, "day": "31"} +{"volume": 26023, "symbol": "LULU", "ts": "2018-08-31 10:59:00", "month": "08", "high": 157.53, "low": 157.4, "key": "LULU_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 157.484, "open": 157.53, "day": "31"} +{"volume": 6444, "symbol": "YEXT", "ts": "2018-08-31 10:31:00", "month": "08", "high": 25.425, "low": 25.4, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.4, "open": 25.425, "day": "31"} +{"volume": 15792, "symbol": "YEXT", "ts": "2018-08-31 10:32:00", "month": "08", "high": 25.25, "low": 25.1, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.1, "open": 25.25, "day": "31"} +{"volume": 9172, "symbol": "YEXT", "ts": "2018-08-31 10:33:00", "month": "08", "high": 25.07, "low": 25.001, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.07, "open": 25.001, "day": "31"} +{"volume": 11509, "symbol": "YEXT", "ts": "2018-08-31 10:34:00", "month": "08", "high": 25.185, "low": 25.0406, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.0406, "open": 25.185, "day": "31"} +{"volume": 2168, "symbol": "YEXT", "ts": "2018-08-31 10:35:00", "month": "08", "high": 25.12, "low": 25.12, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.12, "open": 25.12, "day": "31"} +{"volume": 11298, "symbol": "YEXT", "ts": "2018-08-31 10:36:00", "month": "08", "high": 25.18, "low": 25.18, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.18, "open": 25.18, "day": "31"} +{"volume": 7050, "symbol": "YEXT", "ts": "2018-08-31 10:37:00", "month": "08", "high": 25.21, "low": 25.11, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.11, "open": 25.21, "day": "31"} +{"volume": 2941, "symbol": "YEXT", "ts": "2018-08-31 10:38:00", "month": "08", "high": 25.036, "low": 25.036, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.036, "open": 25.036, "day": "31"} +{"volume": 2477, "symbol": "YEXT", "ts": "2018-08-31 10:39:00", "month": "08", "high": 24.99, "low": 24.99, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.99, "open": 24.99, "day": "31"} +{"volume": 3331, "symbol": "YEXT", "ts": "2018-08-31 10:40:00", "month": "08", "high": 24.959, "low": 24.959, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 24.959, "open": 24.959, "day": "31"} +{"volume": 6609, "symbol": "YEXT", "ts": "2018-08-31 10:41:00", "month": "08", "high": 25.025, "low": 25.025, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.025, "open": 25.025, "day": "31"} +{"volume": 14175, "symbol": "YEXT", "ts": "2018-08-31 10:42:00", "month": "08", "high": 25.095, "low": 25.095, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.095, "open": 25.095, "day": "31"} +{"volume": 2524, "symbol": "YEXT", "ts": "2018-08-31 10:43:00", "month": "08", "high": 25.21, "low": 25.21, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.21, "open": 25.21, "day": "31"} +{"volume": 4085, "symbol": "YEXT", "ts": "2018-08-31 10:44:00", "month": "08", "high": 25.29, "low": 25.29, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.29, "open": 25.29, "day": "31"} +{"volume": 11598, "symbol": "YEXT", "ts": "2018-08-31 10:45:00", "month": "08", "high": 25.24, "low": 25.23, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.24, "open": 25.23, "day": "31"} +{"volume": 1509, "symbol": "YEXT", "ts": "2018-08-31 10:47:00", "month": "08", "high": 25.3, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.3, "day": "31"} +{"volume": 6976, "symbol": "YEXT", "ts": "2018-08-31 10:49:00", "month": "08", "high": 25.3008, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.3008, "day": "31"} +{"volume": 9534, "symbol": "YEXT", "ts": "2018-08-31 10:50:00", "month": "08", "high": 25.31, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.3, "open": 25.31, "day": "31"} +{"volume": 7966, "symbol": "YEXT", "ts": "2018-08-31 10:51:00", "month": "08", "high": 25.355, "low": 25.3, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.355, "open": 25.3, "day": "31"} +{"volume": 4100, "symbol": "YEXT", "ts": "2018-08-31 10:53:00", "month": "08", "high": 25.39, "low": 25.39, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.39, "open": 25.39, "day": "31"} +{"volume": 5012, "symbol": "YEXT", "ts": "2018-08-31 10:54:00", "month": "08", "high": 25.42, "low": 25.39, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.39, "open": 25.42, "day": "31"} +{"volume": 1568, "symbol": "YEXT", "ts": "2018-08-31 10:55:00", "month": "08", "high": 25.44, "low": 25.44, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.44, "open": 25.44, "day": "31"} +{"volume": 2521, "symbol": "YEXT", "ts": "2018-08-31 10:57:00", "month": "08", "high": 25.42, "low": 25.42, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.42, "open": 25.42, "day": "31"} +{"volume": 44045, "symbol": "YEXT", "ts": "2018-08-31 10:58:00", "month": "08", "high": 26.07, "low": 25.5, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 25.96, "open": 25.5, "day": "31"} +{"volume": 19597, "symbol": "YEXT", "ts": "2018-08-31 10:59:00", "month": "08", "high": 26.1, "low": 25.91, "key": "YEXT_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 26.1, "open": 25.91, "day": "31"} +{"volume": 1030, "symbol": "GRUB", "ts": "2018-08-31 10:33:00", "month": "08", "high": 142.7214, "low": 142.7214, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.7214, "open": 142.7214, "day": "31"} +{"volume": 1985, "symbol": "GRUB", "ts": "2018-08-31 10:34:00", "month": "08", "high": 142.49, "low": 142.49, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.49, "open": 142.49, "day": "31"} +{"volume": 1501, "symbol": "GRUB", "ts": "2018-08-31 10:35:00", "month": "08", "high": 142.72, "low": 142.72, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.72, "open": 142.72, "day": "31"} +{"volume": 3954, "symbol": "GRUB", "ts": "2018-08-31 10:36:00", "month": "08", "high": 142.65, "low": 142.65, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.65, "open": 142.65, "day": "31"} +{"volume": 2126, "symbol": "GRUB", "ts": "2018-08-31 10:37:00", "month": "08", "high": 142.825, "low": 142.825, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.825, "open": 142.825, "day": "31"} +{"volume": 1664, "symbol": "GRUB", "ts": "2018-08-31 10:39:00", "month": "08", "high": 142.76, "low": 142.76, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 142.76, "open": 142.76, "day": "31"} +{"volume": 3338, "symbol": "GRUB", "ts": "2018-08-31 10:40:00", "month": "08", "high": 143.02, "low": 143.02, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.02, "open": 143.02, "day": "31"} +{"volume": 1010, "symbol": "GRUB", "ts": "2018-08-31 10:41:00", "month": "08", "high": 143.21, "low": 143.21, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.21, "open": 143.21, "day": "31"} +{"volume": 1620, "symbol": "GRUB", "ts": "2018-08-31 10:42:00", "month": "08", "high": 143.19, "low": 143.19, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.19, "open": 143.19, "day": "31"} +{"volume": 1657, "symbol": "GRUB", "ts": "2018-08-31 10:44:00", "month": "08", "high": 143.25, "low": 143.25, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.25, "open": 143.25, "day": "31"} +{"volume": 1436, "symbol": "GRUB", "ts": "2018-08-31 10:46:00", "month": "08", "high": 143.37, "low": 143.37, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.37, "open": 143.37, "day": "31"} +{"volume": 1771, "symbol": "GRUB", "ts": "2018-08-31 10:47:00", "month": "08", "high": 143.585, "low": 143.585, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.585, "open": 143.585, "day": "31"} +{"volume": 1871, "symbol": "GRUB", "ts": "2018-08-31 10:48:00", "month": "08", "high": 143.41, "low": 143.41, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.41, "open": 143.41, "day": "31"} +{"volume": 2496, "symbol": "GRUB", "ts": "2018-08-31 10:49:00", "month": "08", "high": 143.4894, "low": 143.38, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.4894, "open": 143.38, "day": "31"} +{"volume": 884, "symbol": "GRUB", "ts": "2018-08-31 10:52:00", "month": "08", "high": 143.69, "low": 143.69, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.69, "open": 143.69, "day": "31"} +{"volume": 1820, "symbol": "GRUB", "ts": "2018-08-31 10:53:00", "month": "08", "high": 143.29, "low": 143.29, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.29, "open": 143.29, "day": "31"} +{"volume": 636, "symbol": "GRUB", "ts": "2018-08-31 10:54:00", "month": "08", "high": 143.4436, "low": 143.4436, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.4436, "open": 143.4436, "day": "31"} +{"volume": 1098, "symbol": "GRUB", "ts": "2018-08-31 10:56:00", "month": "08", "high": 143.4, "low": 143.4, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.4, "open": 143.4, "day": "31"} +{"volume": 1070, "symbol": "GRUB", "ts": "2018-08-31 10:57:00", "month": "08", "high": 143.42, "low": 143.42, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.42, "open": 143.42, "day": "31"} +{"volume": 290, "symbol": "GRUB", "ts": "2018-08-31 10:58:00", "month": "08", "high": 143.57, "low": 143.57, "key": "GRUB_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 143.57, "open": 143.57, "day": "31"} +{"volume": 6619, "symbol": "DXCM", "ts": "2018-08-31 10:31:00", "month": "08", "high": 144.6, "low": 144.56, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.56, "open": 144.6, "day": "31"} +{"volume": 1275, "symbol": "DXCM", "ts": "2018-08-31 10:33:00", "month": "08", "high": 144.41, "low": 144.41, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.41, "open": 144.41, "day": "31"} +{"volume": 455, "symbol": "DXCM", "ts": "2018-08-31 10:34:00", "month": "08", "high": 144.4238, "low": 144.4238, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.4238, "open": 144.4238, "day": "31"} +{"volume": 2195, "symbol": "DXCM", "ts": "2018-08-31 10:35:00", "month": "08", "high": 144.355, "low": 144.355, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.355, "open": 144.355, "day": "31"} +{"volume": 407, "symbol": "DXCM", "ts": "2018-08-31 10:37:00", "month": "08", "high": 144.01, "low": 144.01, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.01, "open": 144.01, "day": "31"} +{"volume": 557, "symbol": "DXCM", "ts": "2018-08-31 10:38:00", "month": "08", "high": 144.05, "low": 144.05, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.05, "open": 144.05, "day": "31"} +{"volume": 1551, "symbol": "DXCM", "ts": "2018-08-31 10:41:00", "month": "08", "high": 144.22, "low": 144.22, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.22, "open": 144.22, "day": "31"} +{"volume": 646, "symbol": "DXCM", "ts": "2018-08-31 10:44:00", "month": "08", "high": 144.54, "low": 144.54, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.54, "open": 144.54, "day": "31"} +{"volume": 1405, "symbol": "DXCM", "ts": "2018-08-31 10:45:00", "month": "08", "high": 144.4901, "low": 144.4901, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.4901, "open": 144.4901, "day": "31"} +{"volume": 210, "symbol": "DXCM", "ts": "2018-08-31 10:47:00", "month": "08", "high": 144.5574, "low": 144.5574, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.5574, "open": 144.5574, "day": "31"} +{"volume": 301, "symbol": "DXCM", "ts": "2018-08-31 10:48:00", "month": "08", "high": 144.69, "low": 144.69, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.69, "open": 144.69, "day": "31"} +{"volume": 501, "symbol": "DXCM", "ts": "2018-08-31 10:50:00", "month": "08", "high": 144.52, "low": 144.52, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.52, "open": 144.52, "day": "31"} +{"volume": 557, "symbol": "DXCM", "ts": "2018-08-31 10:51:00", "month": "08", "high": 144.71, "low": 144.71, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.71, "open": 144.71, "day": "31"} +{"volume": 1355, "symbol": "DXCM", "ts": "2018-08-31 10:53:00", "month": "08", "high": 144.66, "low": 144.66, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.66, "open": 144.66, "day": "31"} +{"volume": 527, "symbol": "DXCM", "ts": "2018-08-31 10:54:00", "month": "08", "high": 144.53, "low": 144.53, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.53, "open": 144.53, "day": "31"} +{"volume": 1298, "symbol": "DXCM", "ts": "2018-08-31 10:56:00", "month": "08", "high": 144.63, "low": 144.63, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.63, "open": 144.63, "day": "31"} +{"volume": 2357, "symbol": "DXCM", "ts": "2018-08-31 10:57:00", "month": "08", "high": 144.31, "low": 144.31, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.31, "open": 144.31, "day": "31"} +{"volume": 623, "symbol": "DXCM", "ts": "2018-08-31 10:58:00", "month": "08", "high": 144.4, "low": 144.4, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.4, "open": 144.4, "day": "31"} +{"volume": 2055, "symbol": "DXCM", "ts": "2018-08-31 10:59:00", "month": "08", "high": 144.325, "low": 144.325, "key": "DXCM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 144.325, "open": 144.325, "day": "31"} +{"volume": 545, "symbol": "QURE", "ts": "2018-08-31 10:33:00", "month": "08", "high": 42.005, "low": 42.005, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 42.005, "open": 42.005, "day": "31"} +{"volume": 841, "symbol": "QURE", "ts": "2018-08-31 10:35:00", "month": "08", "high": 41.96, "low": 41.96, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.96, "open": 41.96, "day": "31"} +{"volume": 1415, "symbol": "QURE", "ts": "2018-08-31 10:39:00", "month": "08", "high": 42.0, "low": 41.91, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.91, "open": 42.0, "day": "31"} +{"volume": 863, "symbol": "QURE", "ts": "2018-08-31 10:43:00", "month": "08", "high": 41.88, "low": 41.88, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.88, "open": 41.88, "day": "31"} +{"volume": 701, "symbol": "QURE", "ts": "2018-08-31 10:46:00", "month": "08", "high": 42.0376, "low": 41.995, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.995, "open": 42.0376, "day": "31"} +{"volume": 227, "symbol": "QURE", "ts": "2018-08-31 10:48:00", "month": "08", "high": 41.88, "low": 41.88, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.88, "open": 41.88, "day": "31"} +{"volume": 301, "symbol": "QURE", "ts": "2018-08-31 10:53:00", "month": "08", "high": 41.925, "low": 41.925, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.925, "open": 41.925, "day": "31"} +{"volume": 216, "symbol": "QURE", "ts": "2018-08-31 10:54:00", "month": "08", "high": 41.89, "low": 41.89, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.89, "open": 41.89, "day": "31"} +{"volume": 150, "symbol": "QURE", "ts": "2018-08-31 10:56:00", "month": "08", "high": 41.96, "low": 41.96, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.96, "open": 41.96, "day": "31"} +{"volume": 834, "symbol": "QURE", "ts": "2018-08-31 10:59:00", "month": "08", "high": 41.91, "low": 41.91, "key": "QURE_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 41.91, "open": 41.91, "day": "31"} +{"volume": 5212, "symbol": "CRM", "ts": "2018-08-31 10:31:00", "month": "08", "high": 152.91, "low": 152.88, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.88, "open": 152.91, "day": "31"} +{"volume": 11219, "symbol": "CRM", "ts": "2018-08-31 10:32:00", "month": "08", "high": 152.91, "low": 152.85, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.91, "open": 152.88, "day": "31"} +{"volume": 28613, "symbol": "CRM", "ts": "2018-08-31 10:33:00", "month": "08", "high": 153.07, "low": 152.9606, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.07, "open": 152.9606, "day": "31"} +{"volume": 24815, "symbol": "CRM", "ts": "2018-08-31 10:34:00", "month": "08", "high": 153.0283, "low": 153.0, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.0283, "open": 153.02, "day": "31"} +{"volume": 8731, "symbol": "CRM", "ts": "2018-08-31 10:35:00", "month": "08", "high": 153.01, "low": 153.0, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.0, "open": 153.01, "day": "31"} +{"volume": 24306, "symbol": "CRM", "ts": "2018-08-31 10:36:00", "month": "08", "high": 153.0867, "low": 153.0, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.04, "open": 153.0, "day": "31"} +{"volume": 7243, "symbol": "CRM", "ts": "2018-08-31 10:37:00", "month": "08", "high": 153.11, "low": 153.07, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.11, "open": 153.07, "day": "31"} +{"volume": 6047, "symbol": "CRM", "ts": "2018-08-31 10:38:00", "month": "08", "high": 153.06, "low": 153.06, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.06, "open": 153.06, "day": "31"} +{"volume": 9952, "symbol": "CRM", "ts": "2018-08-31 10:39:00", "month": "08", "high": 153.01, "low": 152.97, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 152.995, "open": 152.97, "day": "31"} +{"volume": 8131, "symbol": "CRM", "ts": "2018-08-31 10:40:00", "month": "08", "high": 153.14, "low": 153.1, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.14, "open": 153.1, "day": "31"} +{"volume": 17436, "symbol": "CRM", "ts": "2018-08-31 10:41:00", "month": "08", "high": 153.31, "low": 153.16, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.31, "open": 153.17, "day": "31"} +{"volume": 7480, "symbol": "CRM", "ts": "2018-08-31 10:42:00", "month": "08", "high": 153.24, "low": 153.21, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.21, "open": 153.24, "day": "31"} +{"volume": 6793, "symbol": "CRM", "ts": "2018-08-31 10:43:00", "month": "08", "high": 153.22, "low": 153.2, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.22, "open": 153.2, "day": "31"} +{"volume": 9152, "symbol": "CRM", "ts": "2018-08-31 10:44:00", "month": "08", "high": 153.2, "low": 153.17, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.17, "open": 153.2, "day": "31"} +{"volume": 4888, "symbol": "CRM", "ts": "2018-08-31 10:45:00", "month": "08", "high": 153.12, "low": 153.12, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.12, "open": 153.12, "day": "31"} +{"volume": 11282, "symbol": "CRM", "ts": "2018-08-31 10:46:00", "month": "08", "high": 153.1907, "low": 153.14, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.15, "open": 153.14, "day": "31"} +{"volume": 13468, "symbol": "CRM", "ts": "2018-08-31 10:47:00", "month": "08", "high": 153.276, "low": 153.2, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.276, "open": 153.2, "day": "31"} +{"volume": 15931, "symbol": "CRM", "ts": "2018-08-31 10:48:00", "month": "08", "high": 153.2, "low": 153.1, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.2, "open": 153.18, "day": "31"} +{"volume": 12094, "symbol": "CRM", "ts": "2018-08-31 10:49:00", "month": "08", "high": 153.33, "low": 153.26, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.33, "open": 153.26, "day": "31"} +{"volume": 7452, "symbol": "CRM", "ts": "2018-08-31 10:50:00", "month": "08", "high": 153.32, "low": 153.2976, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.2976, "open": 153.32, "day": "31"} +{"volume": 12239, "symbol": "CRM", "ts": "2018-08-31 10:51:00", "month": "08", "high": 153.28, "low": 153.28, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.28, "open": 153.28, "day": "31"} +{"volume": 58853, "symbol": "CRM", "ts": "2018-08-31 10:52:00", "month": "08", "high": 153.37, "low": 153.35, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.37, "open": 153.35, "day": "31"} +{"volume": 9130, "symbol": "CRM", "ts": "2018-08-31 10:53:00", "month": "08", "high": 153.33, "low": 153.33, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.33, "open": 153.33, "day": "31"} +{"volume": 16212, "symbol": "CRM", "ts": "2018-08-31 10:54:00", "month": "08", "high": 153.49, "low": 153.37, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.49, "open": 153.37, "day": "31"} +{"volume": 26523, "symbol": "CRM", "ts": "2018-08-31 10:55:00", "month": "08", "high": 153.56, "low": 153.49, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.54, "open": 153.49, "day": "31"} +{"volume": 10653, "symbol": "CRM", "ts": "2018-08-31 10:56:00", "month": "08", "high": 153.535, "low": 153.48, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.48, "open": 153.535, "day": "31"} +{"volume": 7307, "symbol": "CRM", "ts": "2018-08-31 10:57:00", "month": "08", "high": 153.4254, "low": 153.38, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.4254, "open": 153.38, "day": "31"} +{"volume": 13292, "symbol": "CRM", "ts": "2018-08-31 10:58:00", "month": "08", "high": 153.4572, "low": 153.37, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.37, "open": 153.38, "day": "31"} +{"volume": 9205, "symbol": "CRM", "ts": "2018-08-31 10:59:00", "month": "08", "high": 153.48, "low": 153.36, "key": "CRM_2018-08-31 10", "year": 2018, "date": "2018/08/31", "close": 153.48, "open": 153.36, "day": "31"} diff --git a/docker/demo/setup_demo_container.sh b/docker/demo/setup_demo_container.sh new file mode 100755 index 0000000000000..d130ffd70fd51 --- /dev/null +++ b/docker/demo/setup_demo_container.sh @@ -0,0 +1,6 @@ +echo "Copying spark default config and setting up configs" +cp /var/hoodie/ws/docker/demo/config/spark-defaults.conf $SPARK_CONF_DIR/. +hadoop fs -mkdir -p /var/demo/ +hadoop fs -mkdir -p /tmp/spark-events +hadoop fs -copyFromLocal -f /var/hoodie/ws/docker/demo/config /var/demo/. +chmod +x /var/hoodie/ws/hoodie-hive/run_sync_tool.sh diff --git a/docker/hoodie/hadoop/base/Dockerfile b/docker/hoodie/hadoop/base/Dockerfile new file mode 100644 index 0000000000000..4f78ffd8c3d45 --- /dev/null +++ b/docker/hoodie/hadoop/base/Dockerfile @@ -0,0 +1,45 @@ +FROM frolvlad/alpine-oraclejdk8 +MAINTAINER Hoodie +USER root + +# Default to UTF-8 file.encoding +ENV LANG C.UTF-8 + +# Updating & Installing packages +RUN apk add net-tools curl bash perl procps + +ARG HADOOP_VERSION=2.8.4 +ARG HADOOP_URL=https://www.apache.org/dist/hadoop/common/hadoop-${HADOOP_VERSION}/hadoop-${HADOOP_VERSION}.tar.gz +ENV HADOOP_VERSION ${HADOOP_VERSION} +ENV HADOOP_URL ${HADOOP_URL} + +RUN set -x \ + && echo "Fetch URL2 is : ${HADOOP_URL}" \ + && curl -fSL "${HADOOP_URL}" -o /tmp/hadoop.tar.gz \ + && curl -fSL "${HADOOP_URL}.asc" -o /tmp/hadoop.tar.gz.asc \ + && mkdir -p /opt/hadoop-$HADOOP_VERSION/logs \ + && tar -xvf /tmp/hadoop.tar.gz -C /opt/ \ + && rm /tmp/hadoop.tar.gz* \ + && ln -s /opt/hadoop-$HADOOP_VERSION/etc/hadoop /etc/hadoop \ + && cp /etc/hadoop/mapred-site.xml.template /etc/hadoop/mapred-site.xml \ + && mkdir /hadoop-data + +ENV HADOOP_PREFIX=/opt/hadoop-$HADOOP_VERSION +ENV HADOOP_CONF_DIR=/etc/hadoop +ENV MULTIHOMED_NETWORK=1 +ENV HADOOP_HOME=${HADOOP_PREFIX} +ENV HADOOP_INSTALL=${HADOOP_HOME} +ENV USER=root +ENV PATH /usr/bin:/bin:$HADOOP_PREFIX/bin/:$PATH + +# Exposing a union of ports across hadoop versions +# Well known ports including ssh +EXPOSE 0-1024 4040 7000-10100 5000-5100 50000-50200 58188 58088 58042 + +ADD entrypoint.sh /entrypoint.sh +ADD export_container_ip.sh /usr/bin/ +RUN chmod a+x /usr/bin/export_container_ip.sh \ + && chmod a+x /entrypoint.sh + +ENTRYPOINT ["/bin/bash", "/entrypoint.sh"] + diff --git a/docker/hoodie/hadoop/base/entrypoint.sh b/docker/hoodie/hadoop/base/entrypoint.sh new file mode 100644 index 0000000000000..1495d590278b1 --- /dev/null +++ b/docker/hoodie/hadoop/base/entrypoint.sh @@ -0,0 +1,91 @@ +#!/bin/bash + + +####################################################################################### +## COPIED FROM ## +## https://github.com/big-data-europe/docker-hadoop/blob/master/base/entrypoint.sh ## +# ## +####################################################################################### + +# Set some sensible defaults +export CORE_CONF_fs_defaultFS=${CORE_CONF_fs_defaultFS:-hdfs://`hostname -f`:8020} + +function addProperty() { + local path=$1 + local name=$2 + local value=$3 + + local entry="$name${value}" + local escapedEntry=$(echo $entry | sed 's/\//\\\//g') + sed -i "/<\/configuration>/ s/.*/${escapedEntry}\n&/" $path +} + +function configure() { + local path=$1 + local module=$2 + local envPrefix=$3 + + local var + local value + + echo "Configuring $module" + for c in `printenv | perl -sne 'print "$1 " if m/^${envPrefix}_(.+?)=.*/' -- -envPrefix=$envPrefix`; do + name=`echo ${c} | perl -pe 's/___/-/g; s/__/@/g; s/_/./g; s/@/_/g;'` + var="${envPrefix}_${c}" + value=${!var} + echo " - Setting $name=$value" + addProperty /etc/hadoop/$module-site.xml $name "$value" + done +} + +configure /etc/hadoop/core-site.xml core CORE_CONF +configure /etc/hadoop/hdfs-site.xml hdfs HDFS_CONF +configure /etc/hadoop/yarn-site.xml yarn YARN_CONF +configure /etc/hadoop/httpfs-site.xml httpfs HTTPFS_CONF +configure /etc/hadoop/kms-site.xml kms KMS_CONF + +if [ "$MULTIHOMED_NETWORK" = "1" ]; then + echo "Configuring for multihomed network" + + # HDFS + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.rpc-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.servicerpc-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.http-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.https-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.client.use.datanode.hostname true + addProperty /etc/hadoop/hdfs-site.xml dfs.datanode.use.datanode.hostname true + + # YARN + addProperty /etc/hadoop/yarn-site.xml yarn.resourcemanager.bind-host 0.0.0.0 + addProperty /etc/hadoop/yarn-site.xml yarn.nodemanager.bind-host 0.0.0.0 + addProperty /etc/hadoop/yarn-site.xml yarn.nodemanager.bind-host 0.0.0.0 + addProperty /etc/hadoop/yarn-site.xml yarn.timeline-service.bind-host 0.0.0.0 + + # MAPRED + addProperty /etc/hadoop/mapred-site.xml yarn.nodemanager.bind-host 0.0.0.0 +fi + +if [ -n "$GANGLIA_HOST" ]; then + mv /etc/hadoop/hadoop-metrics.properties /etc/hadoop/hadoop-metrics.properties.orig + mv /etc/hadoop/hadoop-metrics2.properties /etc/hadoop/hadoop-metrics2.properties.orig + + for module in mapred jvm rpc ugi; do + echo "$module.class=org.apache.hadoop.metrics.ganglia.GangliaContext31" + echo "$module.period=10" + echo "$module.servers=$GANGLIA_HOST:8649" + done > /etc/hadoop/hadoop-metrics.properties + + for module in namenode datanode resourcemanager nodemanager mrappmaster jobhistoryserver; do + echo "$module.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31" + echo "$module.sink.ganglia.period=10" + echo "$module.sink.ganglia.supportsparse=true" + echo "$module.sink.ganglia.slope=jvm.metrics.gcCount=zero,jvm.metrics.memHeapUsedM=both" + echo "$module.sink.ganglia.dmax=jvm.metrics.threadsBlocked=70,jvm.metrics.memHeapUsedM=40" + echo "$module.sink.ganglia.servers=$GANGLIA_HOST:8649" + done > /etc/hadoop/hadoop-metrics2.properties +fi + +# Save Container IP in ENV variable +/usr/bin/export_container_ip.sh + +exec "$@" diff --git a/docker/hoodie/hadoop/base/export_container_ip.sh b/docker/hoodie/hadoop/base/export_container_ip.sh new file mode 100755 index 0000000000000..bd8e1e61c3393 --- /dev/null +++ b/docker/hoodie/hadoop/base/export_container_ip.sh @@ -0,0 +1,13 @@ +interfaces=( "en0" "eth0" ) + +ipAddr="" +for interface in "${interfaces[@]}" +do + ipAddr=`ifconfig $interface | grep -Eo 'inet (addr:)?([0-9]+\.){3}[0-9]+' | grep -Eo '([0-9]+\.){3}[0-9]+' | grep -v '127.0.0.1' | head` + if [ -n "$ipAddr" ]; then + break + fi +done + +echo "Container IP is set to : $ipAddr" +export MY_CONTAINER_IP=$ipAddr diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml new file mode 100644 index 0000000000000..c975319e371d5 --- /dev/null +++ b/docker/hoodie/hadoop/base/pom.xml @@ -0,0 +1,90 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-base-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-docker + ${project.version} + pom + import + + + + + hoodie + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-base + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-base + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/datanode/Dockerfile b/docker/hoodie/hadoop/datanode/Dockerfile new file mode 100644 index 0000000000000..703a622cf8ed0 --- /dev/null +++ b/docker/hoodie/hadoop/datanode/Dockerfile @@ -0,0 +1,14 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HADOOP_DN_PORT=50075 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-base:latest + +ENV HADOOP_DN_PORT ${HADOOP_DN_PORT} + +ENV HDFS_CONF_dfs_datanode_data_dir=file:///hadoop/dfs/data +RUN mkdir -p /hadoop/dfs/data +VOLUME /hadoop/dfs/data + +ADD run_dn.sh /run_dn.sh +RUN chmod a+x /run_dn.sh + +CMD ["/run_dn.sh"] diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml new file mode 100644 index 0000000000000..94f5a4892ef24 --- /dev/null +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-datanode-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-base-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-datanode + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-datanode + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/datanode/run_dn.sh b/docker/hoodie/hadoop/datanode/run_dn.sh new file mode 100644 index 0000000000000..9f57ee254834f --- /dev/null +++ b/docker/hoodie/hadoop/datanode/run_dn.sh @@ -0,0 +1,9 @@ +#!/bin/bash + +datadir=`echo $HDFS_CONF_dfs_datanode_data_dir | perl -pe 's#file://##'` +if [ ! -d $datadir ]; then + echo "Datanode data directory not found: $datadir" + exit 2 +fi + +$HADOOP_PREFIX/bin/hdfs --config $HADOOP_CONF_DIR datanode diff --git a/docker/hoodie/hadoop/historyserver/Dockerfile b/docker/hoodie/hadoop/historyserver/Dockerfile new file mode 100644 index 0000000000000..d2e05e6da82be --- /dev/null +++ b/docker/hoodie/hadoop/historyserver/Dockerfile @@ -0,0 +1,14 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HADOOP_HISTORY_PORT=8188 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-base:latest + +ENV HADOOP_HISTORY_PORT ${HADOOP_HISTORY_PORT} + +ENV YARN_CONF_yarn_timeline___service_leveldb___timeline___store_path=/hadoop/yarn/timeline +RUN mkdir -p /hadoop/yarn/timeline +VOLUME /hadoop/yarn/timeline + +ADD run_history.sh /run_history.sh +RUN chmod a+x /run_history.sh + +CMD ["/run_history.sh"] diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml new file mode 100644 index 0000000000000..2e7e002545d14 --- /dev/null +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-history-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-base-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-history + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-history + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/historyserver/run_history.sh b/docker/hoodie/hadoop/historyserver/run_history.sh new file mode 100644 index 0000000000000..1ce663321e8f9 --- /dev/null +++ b/docker/hoodie/hadoop/historyserver/run_history.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +$HADOOP_PREFIX/bin/yarn --config $HADOOP_CONF_DIR historyserver diff --git a/docker/hoodie/hadoop/hive_base/Dockerfile b/docker/hoodie/hadoop/hive_base/Dockerfile new file mode 100644 index 0000000000000..40ab1d10c72af --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/Dockerfile @@ -0,0 +1,51 @@ +ARG HADOOP_VERSION=2.8.4 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-base:latest + +ENV HIVE_HOME /opt/hive +ENV PATH $HIVE_HOME/bin:$PATH +ENV HADOOP_HOME /opt/hadoop-$HADOOP_VERSION + +WORKDIR /opt + +ARG HIVE_VERSION=2.3.3 +ARG HIVE_URL=https://archive.apache.org/dist/hive/hive-$HIVE_VERSION/apache-hive-$HIVE_VERSION-bin.tar.gz +ENV HIVE_VERSION ${HIVE_VERSION} +ENV HIVE_URL ${HIVE_URL} + +#Install Hive MySQL, PostgreSQL JDBC +RUN echo "Hive URL is :${HIVE_URL}" && wget ${HIVE_URL} -O hive.tar.gz && \ + tar -xzvf hive.tar.gz && mv *hive*-bin hive && \ + ln -s /usr/share/java/mysql-connector-java.jar $HIVE_HOME/lib/mysql-connector-java.jar && \ + wget https://jdbc.postgresql.org/download/postgresql-9.4.1212.jar -O $HIVE_HOME/lib/postgresql-jdbc.jar && \ + rm hive.tar.gz && mkdir -p /var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/ + +#Spark should be compiled with Hive to be able to use it +#hive-site.xml should be copied to $SPARK_HOME/conf folder + +#Custom configuration goes here +ADD conf/hive-site.xml $HADOOP_CONF_DIR +ADD conf/beeline-log4j2.properties $HIVE_HOME/conf +ADD conf/hive-env.sh $HIVE_HOME/conf +ADD conf/hive-exec-log4j2.properties $HIVE_HOME/conf +ADD conf/hive-log4j2.properties $HIVE_HOME/conf +ADD conf/ivysettings.xml $HIVE_HOME/conf +ADD conf/llap-daemon-log4j2.properties $HIVE_HOME/conf + +# Setup Hoodie Library jars +ADD target/ /var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/ + +ENV HUDI_HADOOP_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-hadoop-mr-bundle.jar +ENV HUDI_HIVE_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-hive-bundle.jar +ENV HUDI_SPARK_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-spark-bundle.jar +ENV HUDI_UTILITIES_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-utilities.jar + +COPY startup.sh /usr/local/bin/ +RUN chmod +x /usr/local/bin/startup.sh + +COPY entrypoint.sh /usr/local/bin/ +RUN chmod +x /usr/local/bin/entrypoint.sh + +ENV PATH $HIVE_HOME/bin/:$PATH + +ENTRYPOINT ["entrypoint.sh"] +CMD startup.sh diff --git a/docker/hoodie/hadoop/hive_base/conf/beeline-log4j2.properties b/docker/hoodie/hadoop/hive_base/conf/beeline-log4j2.properties new file mode 100644 index 0000000000000..103d72253f4e0 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/beeline-log4j2.properties @@ -0,0 +1,45 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = INFO +name = BeelineLog4j2 +packages = org.apache.hadoop.hive.ql.log + +# list of properties +property.hive.log.level = WARN +property.hive.root.logger = console + +# list of all appenders +appenders = console + +# console appender +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n + +# list of all loggers +loggers = HiveConnection + +# HiveConnection logs useful info for dynamic service discovery +logger.HiveConnection.name = org.apache.hive.jdbc.HiveConnection +logger.HiveConnection.level = INFO + +# root logger +rootLogger.level = ${sys:hive.log.level} +rootLogger.appenderRefs = root +rootLogger.appenderRef.root.ref = ${sys:hive.root.logger} diff --git a/docker/hoodie/hadoop/hive_base/conf/hive-env.sh b/docker/hoodie/hadoop/hive_base/conf/hive-env.sh new file mode 100644 index 0000000000000..91a2905d08cc3 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/hive-env.sh @@ -0,0 +1,54 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Set Hive and Hadoop environment variables here. These variables can be used +# to control the execution of Hive. It should be used by admins to configure +# the Hive installation (so that users do not have to set environment variables +# or set command line parameters to get correct behavior). +# +# The hive service being invoked (CLI/HWI etc.) is available via the environment +# variable SERVICE + + +# Hive Client memory usage can be an issue if a large number of clients +# are running at the same time. The flags below have been useful in +# reducing memory usage: +# +# if [ "$SERVICE" = "cli" ]; then +# if [ -z "$DEBUG" ]; then +# export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -Xms10m -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:+UseParNewGC -XX:-UseGCOverheadLimit" +# else +# export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -Xms10m -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:-UseGCOverheadLimit" +# fi +# fi + +# The heap size of the jvm stared by hive shell script can be controlled via: +# +# export HADOOP_HEAPSIZE=1024 +# +# Larger heap size may be required when running queries over large number of files or partitions. +# By default hive shell scripts use a heap size of 256 (MB). Larger heap size would also be +# appropriate for hive server (hwi etc). + + +# Set HADOOP_HOME to point to a specific hadoop install directory +# HADOOP_HOME=${bin}/../../hadoop + +# Hive Configuration Directory can be controlled by: +# export HIVE_CONF_DIR= + +# Folder containing extra ibraries required for hive compilation/execution can be controlled by: +# export HIVE_AUX_JARS_PATH= diff --git a/docker/hoodie/hadoop/hive_base/conf/hive-exec-log4j2.properties b/docker/hoodie/hadoop/hive_base/conf/hive-exec-log4j2.properties new file mode 100644 index 0000000000000..4fba04ca37d1c --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/hive-exec-log4j2.properties @@ -0,0 +1,66 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = INFO +name = HiveExecLog4j2 +packages = org.apache.hadoop.hive.ql.log + +# list of properties +property.hive.log.level = INFO +property.hive.root.logger = FA +property.hive.query.id = hadoop +property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name} +property.hive.log.file = ${sys:hive.query.id}.log + +# list of all appenders +appenders = console, FA + +# console appender +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n + +# simple file appender +appender.FA.type = File +appender.FA.name = FA +appender.FA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file} +appender.FA.layout.type = PatternLayout +appender.FA.layout.pattern = %d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n + +# list of all loggers +loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX + +logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn +logger.NIOServerCnxn.level = WARN + +logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO +logger.ClientCnxnSocketNIO.level = WARN + +logger.DataNucleus.name = DataNucleus +logger.DataNucleus.level = ERROR + +logger.Datastore.name = Datastore +logger.Datastore.level = ERROR + +logger.JPOX.name = JPOX +logger.JPOX.level = ERROR + +# root logger +rootLogger.level = ${sys:hive.log.level} +rootLogger.appenderRefs = root +rootLogger.appenderRef.root.ref = ${sys:hive.root.logger} diff --git a/docker/hoodie/hadoop/hive_base/conf/hive-log4j2.properties b/docker/hoodie/hadoop/hive_base/conf/hive-log4j2.properties new file mode 100644 index 0000000000000..12cd9acbc9b96 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/hive-log4j2.properties @@ -0,0 +1,73 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = INFO +name = HiveLog4j2 +packages = org.apache.hadoop.hive.ql.log + +# list of properties +property.hive.log.level = INFO +property.hive.root.logger = DRFA +property.hive.log.dir = ${sys:java.io.tmpdir}/${sys:user.name} +property.hive.log.file = hive.log + +# list of all appenders +appenders = console, DRFA + +# console appender +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n + +# daily rolling file appender +appender.DRFA.type = RollingFile +appender.DRFA.name = DRFA +appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file} +# Use %pid in the filePattern to append @ to the filename if you want separate log files for different CLI session +appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd} +appender.DRFA.layout.type = PatternLayout +appender.DRFA.layout.pattern = %d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n +appender.DRFA.policies.type = Policies +appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy +appender.DRFA.policies.time.interval = 1 +appender.DRFA.policies.time.modulate = true +appender.DRFA.strategy.type = DefaultRolloverStrategy +appender.DRFA.strategy.max = 30 + +# list of all loggers +loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX + +logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn +logger.NIOServerCnxn.level = WARN + +logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO +logger.ClientCnxnSocketNIO.level = WARN + +logger.DataNucleus.name = DataNucleus +logger.DataNucleus.level = ERROR + +logger.Datastore.name = Datastore +logger.Datastore.level = ERROR + +logger.JPOX.name = JPOX +logger.JPOX.level = ERROR + +# root logger +rootLogger.level = ${sys:hive.log.level} +rootLogger.appenderRefs = root +rootLogger.appenderRef.root.ref = ${sys:hive.root.logger} diff --git a/docker/hoodie/hadoop/hive_base/conf/hive-site.xml b/docker/hoodie/hadoop/hive_base/conf/hive-site.xml new file mode 100644 index 0000000000000..60f393591bab5 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/hive-site.xml @@ -0,0 +1,18 @@ + + + diff --git a/docker/hoodie/hadoop/hive_base/conf/ivysettings.xml b/docker/hoodie/hadoop/hive_base/conf/ivysettings.xml new file mode 100644 index 0000000000000..aa10f438a781c --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/ivysettings.xml @@ -0,0 +1,45 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docker/hoodie/hadoop/hive_base/conf/llap-daemon-log4j2.properties b/docker/hoodie/hadoop/hive_base/conf/llap-daemon-log4j2.properties new file mode 100644 index 0000000000000..5051ca505cb0f --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/conf/llap-daemon-log4j2.properties @@ -0,0 +1,93 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = INFO +name = LlapDaemonLog4j2 +packages = org.apache.hadoop.hive.ql.log + +# list of properties +property.llap.daemon.log.level = INFO +property.llap.daemon.root.logger = console +property.llap.daemon.log.dir = . +property.llap.daemon.log.file = llapdaemon.log +property.llap.daemon.historylog.file = llapdaemon_history.log +property.llap.daemon.log.maxfilesize = 256MB +property.llap.daemon.log.maxbackupindex = 20 + +# list of all appenders +appenders = console, RFA, HISTORYAPPENDER + +# console appender +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t%x] %p %c{2} : %m%n + +# rolling file appender +appender.RFA.type = RollingFile +appender.RFA.name = RFA +appender.RFA.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file} +appender.RFA.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}_%i +appender.RFA.layout.type = PatternLayout +appender.RFA.layout.pattern = %d{ISO8601} %-5p [%t%x]: %c{2} (%F:%M(%L)) - %m%n +appender.RFA.policies.type = Policies +appender.RFA.policies.size.type = SizeBasedTriggeringPolicy +appender.RFA.policies.size.size = ${sys:llap.daemon.log.maxfilesize} +appender.RFA.strategy.type = DefaultRolloverStrategy +appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex} + +# history file appender +appender.HISTORYAPPENDER.type = RollingFile +appender.HISTORYAPPENDER.name = HISTORYAPPENDER +appender.HISTORYAPPENDER.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file} +appender.HISTORYAPPENDER.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}_%i +appender.HISTORYAPPENDER.layout.type = PatternLayout +appender.HISTORYAPPENDER.layout.pattern = %m%n +appender.HISTORYAPPENDER.policies.type = Policies +appender.HISTORYAPPENDER.policies.size.type = SizeBasedTriggeringPolicy +appender.HISTORYAPPENDER.policies.size.size = ${sys:llap.daemon.log.maxfilesize} +appender.HISTORYAPPENDER.strategy.type = DefaultRolloverStrategy +appender.HISTORYAPPENDER.strategy.max = ${sys:llap.daemon.log.maxbackupindex} + +# list of all loggers +loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger + +logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn +logger.NIOServerCnxn.level = WARN + +logger.ClientCnxnSocketNIO.name = org.apache.zookeeper.ClientCnxnSocketNIO +logger.ClientCnxnSocketNIO.level = WARN + +logger.DataNucleus.name = DataNucleus +logger.DataNucleus.level = ERROR + +logger.Datastore.name = Datastore +logger.Datastore.level = ERROR + +logger.JPOX.name = JPOX +logger.JPOX.level = ERROR + +logger.HistoryLogger.name = org.apache.hadoop.hive.llap.daemon.HistoryLogger +logger.HistoryLogger.level = INFO +logger.HistoryLogger.additivity = false +logger.HistoryLogger.appenderRefs = HistoryAppender +logger.HistoryLogger.appenderRef.HistoryAppender.ref = HISTORYAPPENDER + +# root logger +rootLogger.level = ${sys:llap.daemon.log.level} +rootLogger.appenderRefs = root +rootLogger.appenderRef.root.ref = ${sys:llap.daemon.root.logger} diff --git a/docker/hoodie/hadoop/hive_base/entrypoint.sh b/docker/hoodie/hadoop/hive_base/entrypoint.sh new file mode 100644 index 0000000000000..da4c502b793e5 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/entrypoint.sh @@ -0,0 +1,118 @@ +#!/bin/bash + +# Set some sensible defaults +export CORE_CONF_fs_defaultFS=${CORE_CONF_fs_defaultFS:-hdfs://`hostname -f`:8020} + +function addProperty() { + local path=$1 + local name=$2 + local value=$3 + + local entry="$name${value}" + local escapedEntry=$(echo $entry | sed 's/\//\\\//g') + sed -i "/<\/configuration>/ s/.*/${escapedEntry}\n&/" $path +} + +function configure() { + local path=$1 + local module=$2 + local envPrefix=$3 + + local var + local value + + echo "Configuring $module" + for c in `printenv | perl -sne 'print "$1 " if m/^${envPrefix}_(.+?)=.*/' -- -envPrefix=$envPrefix`; do + name=`echo ${c} | perl -pe 's/___/-/g; s/__/_/g; s/_/./g'` + var="${envPrefix}_${c}" + value=${!var} + echo " - Setting $name=$value" + addProperty $path $name "$value" + done +} + +configure /etc/hadoop/core-site.xml core CORE_CONF +configure /etc/hadoop/hdfs-site.xml hdfs HDFS_CONF +configure /etc/hadoop/yarn-site.xml yarn YARN_CONF +configure /etc/hadoop/httpfs-site.xml httpfs HTTPFS_CONF +configure /etc/hadoop/kms-site.xml kms KMS_CONF +configure /etc/hadoop/mapred-site.xml mapred MAPRED_CONF +configure /etc/hadoop/hive-site.xml hive HIVE_SITE_CONF + +if [ "$MULTIHOMED_NETWORK" = "1" ]; then + echo "Configuring for multihomed network" + + # HDFS + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.rpc-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.servicerpc-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.http-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.namenode.https-bind-host 0.0.0.0 + addProperty /etc/hadoop/hdfs-site.xml dfs.client.use.datanode.hostname true + addProperty /etc/hadoop/hdfs-site.xml dfs.datanode.use.datanode.hostname true + + # YARN + addProperty /etc/hadoop/yarn-site.xml yarn.resourcemanager.bind-host 0.0.0.0 + addProperty /etc/hadoop/yarn-site.xml yarn.nodemanager.bind-host 0.0.0.0 + addProperty /etc/hadoop/yarn-site.xml yarn.nodemanager.bind-host 0.0.0.0 + addProperty /etc/hadoop/yarn-site.xml yarn.timeline-service.bind-host 0.0.0.0 + + # MAPRED + addProperty /etc/hadoop/mapred-site.xml yarn.nodemanager.bind-host 0.0.0.0 +fi + +if [ -n "$GANGLIA_HOST" ]; then + mv /etc/hadoop/hadoop-metrics.properties /etc/hadoop/hadoop-metrics.properties.orig + mv /etc/hadoop/hadoop-metrics2.properties /etc/hadoop/hadoop-metrics2.properties.orig + + for module in mapred jvm rpc ugi; do + echo "$module.class=org.apache.hadoop.metrics.ganglia.GangliaContext31" + echo "$module.period=10" + echo "$module.servers=$GANGLIA_HOST:8649" + done > /etc/hadoop/hadoop-metrics.properties + + for module in namenode datanode resourcemanager nodemanager mrappmaster jobhistoryserver; do + echo "$module.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31" + echo "$module.sink.ganglia.period=10" + echo "$module.sink.ganglia.supportsparse=true" + echo "$module.sink.ganglia.slope=jvm.metrics.gcCount=zero,jvm.metrics.memHeapUsedM=both" + echo "$module.sink.ganglia.dmax=jvm.metrics.threadsBlocked=70,jvm.metrics.memHeapUsedM=40" + echo "$module.sink.ganglia.servers=$GANGLIA_HOST:8649" + done > /etc/hadoop/hadoop-metrics2.properties +fi + +function wait_for_it() +{ + local serviceport=$1 + local service=${serviceport%%:*} + local port=${serviceport#*:} + local retry_seconds=5 + local max_try=100 + let i=1 + + nc -z $service $port + result=$? + + until [ $result -eq 0 ]; do + echo "[$i/$max_try] check for ${service}:${port}..." + echo "[$i/$max_try] ${service}:${port} is not available yet" + if (( $i == $max_try )); then + echo "[$i/$max_try] ${service}:${port} is still not available; giving up after ${max_try} tries. :/" + exit 1 + fi + + echo "[$i/$max_try] try in ${retry_seconds}s once again ..." + let "i++" + sleep $retry_seconds + + nc -z $service $port + result=$? + done + echo "[$i/$max_try] $service:${port} is available." +} + +for i in ${SERVICE_PRECONDITION[@]} +do + wait_for_it ${i} +done + +exec $@ diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml new file mode 100644 index 0000000000000..ec6b4fa6a56a2 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -0,0 +1,113 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-hive-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-base-docker + ${project.version} + pom + import + + + + + + + maven-antrun-plugin + 1.7 + + + package + + + + + + + + + + run + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version} + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version} + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/hive_base/startup.sh b/docker/hoodie/hadoop/hive_base/startup.sh new file mode 100644 index 0000000000000..d243e5609af06 --- /dev/null +++ b/docker/hoodie/hadoop/hive_base/startup.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +hadoop fs -mkdir /tmp +hadoop fs -mkdir -p /user/hive/warehouse +hadoop fs -chmod g+w /tmp +hadoop fs -chmod g+w /user/hive/warehouse + +cd $HIVE_HOME/bin +export AUX_CLASSPATH=file://${HUDI_HADOOP_BUNDLE} +./hiveserver2 --hiveconf hive.server2.enable.doAs=false --hiveconf hive.aux.jars.path=file://${HUDI_HADOOP_BUNDLE} diff --git a/docker/hoodie/hadoop/namenode/Dockerfile b/docker/hoodie/hadoop/namenode/Dockerfile new file mode 100644 index 0000000000000..648032977d611 --- /dev/null +++ b/docker/hoodie/hadoop/namenode/Dockerfile @@ -0,0 +1,14 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HADOOP_WEBHDFS_PORT=50070 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-base:latest + +ENV HADOOP_WEBHDFS_PORT ${HADOOP_WEBHDFS_PORT} + +ENV HDFS_CONF_dfs_namenode_name_dir=file:///hadoop/dfs/name +RUN mkdir -p /hadoop/dfs/name +VOLUME /hadoop/dfs/name + +ADD run_nn.sh /run_nn.sh +RUN chmod a+x /run_nn.sh + +CMD ["/run_nn.sh"] diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml new file mode 100644 index 0000000000000..2604a99a6a716 --- /dev/null +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-namenode-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-base-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-namenode + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-namenode + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/namenode/run_nn.sh b/docker/hoodie/hadoop/namenode/run_nn.sh new file mode 100644 index 0000000000000..8ef0797025800 --- /dev/null +++ b/docker/hoodie/hadoop/namenode/run_nn.sh @@ -0,0 +1,19 @@ +#!/bin/bash + +namedir=`echo $HDFS_CONF_dfs_namenode_name_dir | perl -pe 's#file://##'` +if [ ! -d $namedir ]; then + echo "Namenode name directory not found: $namedir" + exit 2 +fi + +if [ -z "$CLUSTER_NAME" ]; then + echo "Cluster name not specified" + exit 2 +fi + +if [ "`ls -A $namedir`" == "" ]; then + echo "Formatting namenode name directory: $namedir" + $HADOOP_PREFIX/bin/hdfs --config $HADOOP_CONF_DIR namenode -format $CLUSTER_NAME +fi + +$HADOOP_PREFIX/bin/hdfs --config $HADOOP_CONF_DIR namenode diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml new file mode 100644 index 0000000000000..7ef639c7f33de --- /dev/null +++ b/docker/hoodie/hadoop/pom.xml @@ -0,0 +1,78 @@ + + + + + + hoodie + com.uber.hoodie + 0.4.5-SNAPSHOT + ../../../pom.xml + + 4.0.0 + + hoodie-hadoop-docker + pom + + base + namenode + datanode + historyserver + hive_base + spark_base + sparkmaster + sparkworker + sparkadhoc + + + + + com.uber.hoodie + hoodie-spark-bundle + ${project.version} + + + + + false + true + 2.3.1 + 2.3.3 + 2.8.4 + 1.4.3 + true + + + + + + com.spotify + dockerfile-maven-extension + ${dockerfile.maven.version} + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + true + + + + + diff --git a/docker/hoodie/hadoop/spark_base/Dockerfile b/docker/hoodie/hadoop/spark_base/Dockerfile new file mode 100644 index 0000000000000..badaaa6902689 --- /dev/null +++ b/docker/hoodie/hadoop/spark_base/Dockerfile @@ -0,0 +1,46 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HIVE_VERSION=2.3.3 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-hive_${HIVE_VERSION} + +ENV ENABLE_INIT_DAEMON true +ENV INIT_DAEMON_BASE_URI http://identifier/init-daemon +ENV INIT_DAEMON_STEP spark_master_init + +ARG SPARK_VERSION=2.3.1 +ARG SPARK_HADOOP_VERSION=2.7 + +ENV SPARK_VERSION ${SPARK_VERSION} +ENV HADOOP_VERSION ${SPARK_HADOOP_VERSION} + +COPY wait-for-step.sh / +COPY execute-step.sh / +COPY finish-step.sh / + +RUN echo "Installing Spark-version (${SPARK_VERSION})" \ + && wget http://apache.mirror.iphh.net/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz \ + && tar -xvzf spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz \ + && mv spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION} /opt/spark \ + && rm spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz \ + && cd / + +#Give permission to execute scripts +RUN chmod +x /wait-for-step.sh && chmod +x /execute-step.sh && chmod +x /finish-step.sh + +# Fix the value of PYTHONHASHSEED +# Note: this is needed when you use Python 3.3 or greater +ENV PYTHONHASHSEED 1 + +ENV SPARK_HOME /opt/spark +ENV SPARK_INSTALL ${SPARK_HOME} +ENV SPARK_CONF_DIR ${SPARK_HOME}/conf +ENV PATH $SPARK_INSTALL/bin:$PATH + +ENV SPARK_DRIVER_PORT 5001 +ENV SPARK_UI_PORT 5002 +ENV SPARK_BLOCKMGR_PORT 5003 + +EXPOSE $SPARK_DRIVER_PORT $SPARK_UI_PORT $SPARK_BLOCKMGR_PORT + +# Without this spark-shell fails - Download if it is not already there in $SPARK_INSTALL +RUN wget -nc -q -O "${SPARK_INSTALL}/jars/jersey-bundle-1.19.4.jar" "http://repo1.maven.org/maven2/com/sun/jersey/jersey-bundle/1.19.4/jersey-bundle-1.19.4.jar" + diff --git a/docker/hoodie/hadoop/spark_base/execute-step.sh b/docker/hoodie/hadoop/spark_base/execute-step.sh new file mode 100644 index 0000000000000..24131b6e2ae59 --- /dev/null +++ b/docker/hoodie/hadoop/spark_base/execute-step.sh @@ -0,0 +1,14 @@ +#!/bin/bash + +if [ $ENABLE_INIT_DAEMON = "true" ] + then + echo "Execute step ${INIT_DAEMON_STEP} in pipeline" + while true; do + sleep 5 + echo -n '.' + string=$(curl -sL -w "%{http_code}" -X PUT $INIT_DAEMON_BASE_URI/execute?step=$INIT_DAEMON_STEP -o /dev/null) + [ "$string" = "204" ] && break + done + echo "Notified execution of step ${INIT_DAEMON_STEP}" +fi + diff --git a/docker/hoodie/hadoop/spark_base/finish-step.sh b/docker/hoodie/hadoop/spark_base/finish-step.sh new file mode 100644 index 0000000000000..7403bd942f9bd --- /dev/null +++ b/docker/hoodie/hadoop/spark_base/finish-step.sh @@ -0,0 +1,16 @@ +#!/bin/bash + +if [ $ENABLE_INIT_DAEMON = "true" ] + then + echo "Finish step ${INIT_DAEMON_STEP} in pipeline" + while true; do + sleep 5 + echo -n '.' + string=$(curl -sL -w "%{http_code}" -X PUT $INIT_DAEMON_BASE_URI/finish?step=$INIT_DAEMON_STEP -o /dev/null) + [ "$string" = "204" ] && break + done + echo "Notified finish of step ${INIT_DAEMON_STEP}" +fi + + + diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml new file mode 100644 index 0000000000000..fbd34e9909da9 --- /dev/null +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-sparkbase-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-hive-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkbase_${docker.spark.version} + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkbase_${docker.spark.version} + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/spark_base/wait-for-step.sh b/docker/hoodie/hadoop/spark_base/wait-for-step.sh new file mode 100644 index 0000000000000..75e6996d3a560 --- /dev/null +++ b/docker/hoodie/hadoop/spark_base/wait-for-step.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +if [ $ENABLE_INIT_DAEMON = "true" ] + then + echo "Validating if step ${INIT_DAEMON_STEP} can start in pipeline" + while true; do + sleep 5 + echo -n '.' + string=$(curl -s $INIT_DAEMON_BASE_URI/canStart?step=$INIT_DAEMON_STEP) + [ "$string" = "true" ] && break + done + echo "Can start step ${INIT_DAEMON_STEP}" +fi diff --git a/docker/hoodie/hadoop/sparkadhoc/Dockerfile b/docker/hoodie/hadoop/sparkadhoc/Dockerfile new file mode 100644 index 0000000000000..8a731305d5227 --- /dev/null +++ b/docker/hoodie/hadoop/sparkadhoc/Dockerfile @@ -0,0 +1,12 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HIVE_VERSION=2.3.3 +ARG SPARK_VERSION=2.3.1 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-hive_${HIVE_VERSION}-sparkbase_${SPARK_VERSION} + +COPY adhoc.sh /opt/spark + +ENV SPARK_WORKER_WEBUI_PORT 8081 +ENV SPARK_WORKER_LOG /spark/logs +ENV SPARK_MASTER "spark://spark-master:7077" + +CMD ["/bin/bash", "/opt/spark/adhoc.sh"] diff --git a/docker/hoodie/hadoop/sparkadhoc/adhoc.sh b/docker/hoodie/hadoop/sparkadhoc/adhoc.sh new file mode 100644 index 0000000000000..2e46d339e9b4c --- /dev/null +++ b/docker/hoodie/hadoop/sparkadhoc/adhoc.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +. "/spark/sbin/spark-config.sh" + +. "/spark/bin/load-spark-env.sh" + + +export SPARK_HOME=/opt/spark + +date +echo "SPARK HOME is : $SPARK_HOME" + +tail -f /dev/null diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml new file mode 100644 index 0000000000000..b77a4c0a771aa --- /dev/null +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-sparkadhoc-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-sparkbase-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkadhoc_${docker.spark.version} + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkadhoc_${docker.spark.version} + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/sparkmaster/Dockerfile b/docker/hoodie/hadoop/sparkmaster/Dockerfile new file mode 100644 index 0000000000000..acfba8c691a59 --- /dev/null +++ b/docker/hoodie/hadoop/sparkmaster/Dockerfile @@ -0,0 +1,14 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HIVE_VERSION=2.3.3 +ARG SPARK_VERSION=2.3.1 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-hive_${HIVE_VERSION}-sparkbase_${SPARK_VERSION} + +COPY master.sh /opt/spark + +ENV SPARK_MASTER_PORT 7077 +ENV SPARK_MASTER_WEBUI_PORT 8080 +ENV SPARK_MASTER_LOG /opt/spark/logs + +EXPOSE 8080 7077 6066 + +CMD ["/bin/bash", "/opt/spark/master.sh"] diff --git a/docker/hoodie/hadoop/sparkmaster/master.sh b/docker/hoodie/hadoop/sparkmaster/master.sh new file mode 100644 index 0000000000000..64fc289069748 --- /dev/null +++ b/docker/hoodie/hadoop/sparkmaster/master.sh @@ -0,0 +1,16 @@ +#!/bin/bash + +export SPARK_MASTER_HOST=`hostname` + +. "/opt/spark/sbin/spark-config.sh" + +. "/opt/spark/bin/load-spark-env.sh" + +mkdir -p $SPARK_MASTER_LOG + +export SPARK_HOME=/opt/spark + +ln -sf /dev/stdout $SPARK_MASTER_LOG/spark-master.out + +cd /opt/spark/bin && /opt/spark/sbin/../bin/spark-class org.apache.spark.deploy.master.Master \ + --ip $SPARK_MASTER_HOST --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT >> $SPARK_MASTER_LOG/spark-master.out diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml new file mode 100644 index 0000000000000..f988ed9ef2516 --- /dev/null +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-sparkmaster-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-sparkbase-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkmaster_${docker.spark.version} + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkmaster_${docker.spark.version} + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/sparkworker/Dockerfile b/docker/hoodie/hadoop/sparkworker/Dockerfile new file mode 100644 index 0000000000000..acbddc0773a82 --- /dev/null +++ b/docker/hoodie/hadoop/sparkworker/Dockerfile @@ -0,0 +1,14 @@ +ARG HADOOP_VERSION=2.8.4 +ARG HIVE_VERSION=2.3.3 +ARG SPARK_VERSION=2.3.1 +FROM varadarb/hudi-hadoop_${HADOOP_VERSION}-hive_${HIVE_VERSION}-sparkbase_${SPARK_VERSION} + +COPY worker.sh /opt/spark + +ENV SPARK_WORKER_WEBUI_PORT 8081 +ENV SPARK_WORKER_LOG /spark/logs +ENV SPARK_MASTER "spark://spark-master:7077" + +EXPOSE 8081 + +CMD ["/bin/bash", "/opt/spark/worker.sh"] diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml new file mode 100644 index 0000000000000..93064bb11c3f4 --- /dev/null +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -0,0 +1,89 @@ + + + + + + hoodie-hadoop-docker + com.uber.hoodie + 0.4.5-SNAPSHOT + + 4.0.0 + pom + hoodie-hadoop-sparkworker-docker + + Base Docker Image with Hoodie + + + UTF-8 + true + + + + + com.uber.hoodie + hoodie-hadoop-sparkbase-docker + ${project.version} + pom + import + + + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile.maven.version} + + + tag-latest + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkworker_${docker.spark.version} + true + latest + + + + tag-version + pre-integration-test + + build + tag + + + + ${docker.build.skip} + false + varadarb/hudi-hadoop_${docker.hadoop.version}-hive_${docker.hive.version}-sparkworker_${docker.spark.version} + true + ${project.version} + + + + + + + diff --git a/docker/hoodie/hadoop/sparkworker/worker.sh b/docker/hoodie/hadoop/sparkworker/worker.sh new file mode 100644 index 0000000000000..1387411135484 --- /dev/null +++ b/docker/hoodie/hadoop/sparkworker/worker.sh @@ -0,0 +1,16 @@ +#!/bin/bash + +. "/spark/sbin/spark-config.sh" + +. "/spark/bin/load-spark-env.sh" + +mkdir -p $SPARK_WORKER_LOG + +export SPARK_HOME=/opt/spark + +ln -sf /dev/stdout $SPARK_WORKER_LOG/spark-worker.out + +date +echo "SPARK HOME is : $SPARK_HOME" +/opt/spark/sbin/../bin/spark-class org.apache.spark.deploy.worker.Worker \ + --webui-port $SPARK_WORKER_WEBUI_PORT $SPARK_MASTER >> $SPARK_WORKER_LOG/spark-worker.out diff --git a/docker/setup_demo.sh b/docker/setup_demo.sh new file mode 100755 index 0000000000000..efbae8472e277 --- /dev/null +++ b/docker/setup_demo.sh @@ -0,0 +1,16 @@ +# Create host mount directory and copy +mkdir -p /tmp/hadoop_name +mkdir -p /tmp/hadoop_data + +WS_ROOT=`dirname $PWD` +# restart cluster +HUDI_WS=${WS_ROOT} docker-compose -f compose/docker-compose_hadoop284_hive233_spark231.yml down +HUDI_WS=${WS_ROOT} docker-compose -f compose/docker-compose_hadoop284_hive233_spark231.yml pull +rm -rf /tmp/hadoop_data/* +rm -rf /tmp/hadoop_name/* +sleep 5 +HUDI_WS=${WS_ROOT} docker-compose -f compose/docker-compose_hadoop284_hive233_spark231.yml up -d +sleep 15 + +docker exec -it adhoc-1 /bin/bash /var/hoodie/ws/docker/demo/setup_demo_container.sh +docker exec -it adhoc-2 /bin/bash /var/hoodie/ws/docker/demo/setup_demo_container.sh diff --git a/docs/quickstart.md b/docs/quickstart.md index ea18c752fefdc..d025f3e0d670d 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -14,11 +14,11 @@ Check out code and pull it into Intellij as a normal maven project. Normally build the maven project, from command line ``` -$ mvn clean install -DskipTests +$ mvn clean install -DskipTests -DskipITs To work with older version of Hive (pre Hive-1.2.1), use -$ mvn clean install -DskipTests -Dhive11 +$ mvn clean install -DskipTests -DskipITs -Dhive11 ``` @@ -293,6 +293,947 @@ hive> {% include note.html content="This is only supported for Read-optimized tables for now." %} +## A Demo using docker containers +Lets use a real world example to see how hudi works end to end. For this purpose, a self contained +data infrastructure is brought up in a local docker cluster within your computer. +The steps assume you are using Mac laptop +### Prerequisites + + * Docker Setup : For Mac, Please follow the steps as defined in [https://docs.docker.com/v17.12/docker-for-mac/install/]. For running Spark-SQL queries, please ensure atleast 6 GB and 4 CPUs are allocated to Docker (See Docker -> Preferences -> Advanced). Otherwise, spark-SQL queries could be killed because of memory issues. + * kafkacat : A command-line utility to publish/consume from kafka topics. Use `brew install kafkacat` to install kafkacat + * /etc/hosts : The demo references many services running in container by the hostname. Add the following settings to /etc/hosts + + ``` + 127.0.0.1 adhoc-1 + 127.0.0.1 adhoc-2 + 127.0.0.1 namenode + 127.0.0.1 datanode1 + 127.0.0.1 hiveserver + 127.0.0.1 hivemetastore + 127.0.0.1 kafkabroker + 127.0.0.1 sparkmaster + 127.0.0.1 zookeeper + ``` + +### Setting up Docker Cluster + + +#### Build Hoodie + +The first step is to build hoodie +``` +cd +mvn package -DskipTests +``` + +#### Bringing up Demo Cluster + +The next step is to run the docker compose script and setup configs for bringing up the cluster. +This should pull the docker images from docker hub and setup docker cluster. + +``` +cd docker +./setup_demo.sh +.... +.... +.... +Stopping spark-worker-1 ... done +Stopping hiveserver ... done +Stopping hivemetastore ... done +Stopping historyserver ... done +....... +...... +Creating network "hudi_demo" with the default driver +Creating hive-metastore-postgresql ... done +Creating namenode ... done +Creating zookeeper ... done +Creating kafkabroker ... done +Creating hivemetastore ... done +Creating historyserver ... done +Creating hiveserver ... done +Creating datanode1 ... done +Creating sparkmaster ... done +Creating adhoc-1 ... done +Creating adhoc-2 ... done +Creating spark-worker-1 ... done +Copying spark default config and setting up configs +Copying spark default config and setting up configs +Copying spark default config and setting up configs +varadarb-C02SG7Q3G8WP:docker varadarb$ docker ps +``` + +At this point, the docker cluster will be up and running. The demo cluster brings up the following services + + * HDFS Services (NameNode, DataNode) + * Spark Master and Worker + * Hive Services (Metastore, HiveServer2 along with PostgresDB) + * Kafka Broker and a Zookeeper Node (Kakfa will be used as upstream source for the demo) + * Adhoc containers to run Hudi/Hive CLI commands + +### Demo + +Stock Tracker data will be used to showcase both different Hudi Views and the effects of Compaction. + +Take a look at the directory `docker/demo/data`. There are 2 batches of stock data - each at 1 minute granularity. +The first batch contains stocker tracker data for some stock symbols during the first hour of trading window +(9:30 a.m to 10:30 a.m). The second batch contains tracker data for next 30 mins (10:30 - 11 a.m). Hudi will +be used to ingest these batches to a dataset which will contain the latest stock tracker data at hour level granularity. +The batches are windowed intentionally so that the second batch contains updates to some of the rows in the first batch. + +#### Step 1 : Publish the first batch to Kafka + +Upload the first batch to Kafka topic 'stock ticks' + +``` +cat docker/demo/data/batch_1.json | kafkacat -b kafkabroker -t stock_ticks -P + +To check if the new topic shows up, use +kafkacat -b kafkabroker -L -J | jq . +{ + "originating_broker": { + "id": 1001, + "name": "kafkabroker:9092/1001" + }, + "query": { + "topic": "*" + }, + "brokers": [ + { + "id": 1001, + "name": "kafkabroker:9092" + } + ], + "topics": [ + { + "topic": "stock_ticks", + "partitions": [ + { + "partition": 0, + "leader": 1001, + "replicas": [ + { + "id": 1001 + } + ], + "isrs": [ + { + "id": 1001 + } + ] + } + ] + } + ] +} + +``` + +#### Step 2: Incrementally ingest data from Kafka topic + +Hudi comes with a tool named DeltaStreamer. This tool can connect to variety of data sources (including Kafka) to +pull changes and apply to Hudi dataset using upsert/insert primitives. Here, we will use the tool to download +json data from kafka topic and ingest to both COW and MOR tables we initialized in the previous step. This tool +automatically initializes the datasets in the file-system if they do not exist yet. + +``` +docker exec -it adhoc-2 /bin/bash + +# Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_cow dataset in HDFS +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type COPY_ON_WRITE --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_cow --target-table stock_ticks_cow --props /var/demo/config/kafka-source.properties +.... +.... +2018-09-24 22:20:00 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint:54 - OutputCommitCoordinator stopped! +2018-09-24 22:20:00 INFO SparkContext:54 - Successfully stopped SparkContext +# Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_mor dataset in HDFS +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type MERGE_ON_READ --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_mor --target-table stock_ticks_mor --props /var/demo/config/kafka-source.properties +.... +2018-09-24 22:22:01 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint:54 - OutputCommitCoordinator stopped! +2018-09-24 22:22:01 INFO SparkContext:54 - Successfully stopped SparkContext +.... + +# As part of the setup (Look at setup_demo.sh), the configs needed for DeltaStreamer is uploaded to HDFS. The configs +# contain mostly Kafa connectivity settings, the avro-schema to be used for ingesting along with key and partitioning fields. + +exit +``` + +You can use HDFS web-browser to look at the datasets +`http://namenode:50070/explorer.html#/user/hive/warehouse/stock_ticks_cow`. + +You can explore the new partition folder created in the dataset along with a "deltacommit" +file under .hoodie which signals a successful commit. + +There will be a similar setup when you browse the MOR dataset +`http://namenode:50070/explorer.html#/user/hive/warehouse/stock_ticks_mor` + + +#### Step 3: Sync with Hive + +At this step, the datasets are available in HDFS. We need to sync with Hive to create new Hive tables and add partitions +inorder to run Hive queries against those datasets. + +``` +docker exec -it adhoc-2 /bin/bash + +# THis command takes in HIveServer URL and COW Hudi Dataset location in HDFS and sync the HDFS state to Hive +/var/hoodie/ws/hoodie-hive/run_sync_tool.sh --jdbc-url jdbc:hive2://hiveserver:10000 --user hive --pass hive --partitioned-by dt --base-path /user/hive/warehouse/stock_ticks_cow --database default --table stock_ticks_cow +..... +2018-09-24 22:22:45,568 INFO [main] hive.HiveSyncTool (HiveSyncTool.java:syncHoodieTable(112)) - Sync complete for stock_ticks_cow +..... + +# Now run hive-sync for the second data-set in HDFS using Merge-On-Read (MOR storage) +/var/hoodie/ws/hoodie-hive/run_sync_tool.sh --jdbc-url jdbc:hive2://hiveserver:10000 --user hive --pass hive --partitioned-by dt --base-path /user/hive/warehouse/stock_ticks_mor --database default --table stock_ticks_mor +... +2018-09-24 22:23:09,171 INFO [main] hive.HiveSyncTool (HiveSyncTool.java:syncHoodieTable(112)) - Sync complete for stock_ticks_mor +... +2018-09-24 22:23:09,559 INFO [main] hive.HiveSyncTool (HiveSyncTool.java:syncHoodieTable(112)) - Sync complete for stock_ticks_mor_rt +.... +exit +``` +After executing the above command, you will notice + +1. A hive table named `stock_ticks_cow` created which provides Read-Optimized view for the Copy On Write dataset. +2. Two new tables `stock_ticks_mor` and `stock_ticks_mor_rt` created for the Merge On Read dataset. The former +provides the ReadOptimized view for the Hudi dataset and the later provides the realtime-view for the dataset. + + +#### Step 4 (a): Run Hive Queries + +Run a hive query to find the latest timestamp ingested for stock symbol 'GOOG'. You will notice that both read-optimized +(for both COW and MOR dataset)and realtime views (for MOR dataset)give the same value "10:29 a.m" as Hudi create a +parquet file for the first batch of data. + +``` +docker exec -it adhoc-2 /bin/bash +beeline -u jdbc:hive2://hiveserver:10000 --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat --hiveconf hive.stats.autogather=false +# List Tables +0: jdbc:hive2://hiveserver:10000> show tables; ++---------------------+--+ +| tab_name | ++---------------------+--+ +| stock_ticks_cow | +| stock_ticks_mor | +| stock_ticks_mor_rt | ++---------------------+--+ +2 rows selected (0.801 seconds) +0: jdbc:hive2://hiveserver:10000> + + +# Look at partitions that were added +0: jdbc:hive2://hiveserver:10000> show partitions stock_ticks_mor_rt; ++----------------+--+ +| partition | ++----------------+--+ +| dt=2018-08-31 | ++----------------+--+ +1 row selected (0.24 seconds) + + +# COPY-ON-WRITE Queries: +========================= + + +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'; ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:29:00 | ++---------+----------------------+--+ + +Now, run a projection query: + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924221953 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924221953 | GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + + +# Merge-On-Read Queries: +========================== + +Lets run similar queries against M-O-R dataset. Lets look at both +ReadOptimized and Realtime views supported by M-O-R dataset + +# Run against ReadOptimized View. Notice that the latest timestamp is 10:29 +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:29:00 | ++---------+----------------------+--+ +1 row selected (6.326 seconds) + + +# Run against Realtime View. Notice that the latest timestamp is again 10:29 + +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:29:00 | ++---------+----------------------+--+ +1 row selected (1.606 seconds) + + +# Run projection query against Read Optimized and Realtime tables + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924222155 | GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924222155 | GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +exit +exit +``` + +#### Step 4 (b): Run Spark-SQL Queries +Hudi support Spark as query processor just like Hive. Here are the same hive queries +running in spark-sql + +``` +docker exec -it adhoc-1 /bin/bash +$SPARK_INSTALL/bin/spark-shell --jars $HUDI_SPARK_BUNDLE --master local[2] --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false --deploy-mode client --driver-memory 1G --executor-memory 3G --num-executors 1 --packages com.databricks:spark-avro_2.11:4.0.0 +... + +Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version 2.3.1 + /_/ + +Using Scala version 2.11.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_181) +Type in expressions to have them evaluated. +Type :help for more information. + +scala> +scala> spark.sql("show tables").show(100, false) ++--------+------------------+-----------+ +|database|tableName |isTemporary| ++--------+------------------+-----------+ +|default |stock_ticks_cow |false | +|default |stock_ticks_mor |false | +|default |stock_ticks_mor_rt|false | ++--------+------------------+-----------+ + +# Copy-On-Write Table + +## Run max timestamp query against COW table + +scala> spark.sql("select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'").show(100, false) +[Stage 0:> (0 + 1) / 1]SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder". +SLF4J: Defaulting to no-operation (NOP) logger implementation +SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. ++------+-------------------+ +|symbol|max(ts) | ++------+-------------------+ +|GOOG |2018-08-31 10:29:00| ++------+-------------------+ + +## Projection Query + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'").show(100, false) ++-------------------+------+-------------------+------+---------+--------+ +|_hoodie_commit_time|symbol|ts |volume|open |close | ++-------------------+------+-------------------+------+---------+--------+ +|20180924221953 |GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 | +|20180924221953 |GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085| ++-------------------+------+-------------------+------+---------+--------+ + +# Merge-On-Read Queries: +========================== + +Lets run similar queries against M-O-R dataset. Lets look at both +ReadOptimized and Realtime views supported by M-O-R dataset + +# Run against ReadOptimized View. Notice that the latest timestamp is 10:29 +scala> spark.sql("select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'").show(100, false) ++------+-------------------+ +|symbol|max(ts) | ++------+-------------------+ +|GOOG |2018-08-31 10:29:00| ++------+-------------------+ + + +# Run against Realtime View. Notice that the latest timestamp is again 10:29 + +scala> spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'").show(100, false) ++------+-------------------+ +|symbol|max(ts) | ++------+-------------------+ +|GOOG |2018-08-31 10:29:00| ++------+-------------------+ + +# Run projection query against Read Optimized and Realtime tables + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'").show(100, false) ++-------------------+------+-------------------+------+---------+--------+ +|_hoodie_commit_time|symbol|ts |volume|open |close | ++-------------------+------+-------------------+------+---------+--------+ +|20180924222155 |GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 | +|20180924222155 |GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085| ++-------------------+------+-------------------+------+---------+--------+ + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'").show(100, false) ++-------------------+------+-------------------+------+---------+--------+ +|_hoodie_commit_time|symbol|ts |volume|open |close | ++-------------------+------+-------------------+------+---------+--------+ +|20180924222155 |GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 | +|20180924222155 |GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085| ++-------------------+------+-------------------+------+---------+--------+ + +``` + + +#### Step 5: Upload second batch to Kafka and run DeltaStreamer to ingest + +Upload the second batch of data and ingest this batch using delta-streamer. As this batch does not bring in any new +partitions, there is no need to run hive-sync + +``` +cat docker/demo/data/batch_2.json | kafkacat -b kafkabroker -t stock_ticks -P + +# Within Docker container, run the ingestion command +docker exec -it adhoc-2 /bin/bash + +# Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_cow dataset in HDFS +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type COPY_ON_WRITE --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_cow --target-table stock_ticks_cow --props /var/demo/config/kafka-source.properties + +# Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_mor dataset in HDFS +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type MERGE_ON_READ --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_mor --target-table stock_ticks_mor --props /var/demo/config/kafka-source.properties + +exit +``` +With Copy-On-Write table, the second ingestion by DeltaStreamer resulted in a new version of Parquet file getting created. +See `http://namenode:50070/explorer.html#/user/hive/warehouse/stock_ticks_cow/2018/08/31` + +With Merge-On-Read table, the second ingestion merely appended the batch to an unmerged delta (log) file. +Take a look at the HDFS filesystem to get an idea: `http://namenode:50070/explorer.html#/user/hive/warehouse/stock_ticks_mor/2018/08/31` + +#### Step 6(a): Run Hive Queries + +With Copy-On-Write table, the read-optimized view immediately sees the changes as part of second batch once the batch +got committed as each ingestion creates newer versions of parquet files. + +With Merge-On-Read table, the second ingestion merely appended the batch to an unmerged delta (log) file. +This is the time, when ReadOptimized and Realtime views will provide different results. ReadOptimized view will still +return "10:29 am" as it will only read from the Parquet file. Realtime View will do on-the-fly merge and return +latest committed data which is "10:59 a.m". + +``` +docker exec -it adhoc-2 /bin/bash +beeline -u jdbc:hive2://hiveserver:10000 --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat --hiveconf hive.stats.autogather=false + +# Copy On Write Table: + +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ +1 row selected (1.932 seconds) + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924221953 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924224524 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +As you can notice, the above queries now reflect the changes that came as part of ingesting second batch. + + +# Merge On Read Table: + +# Read Optimized View +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:29:00 | ++---------+----------------------+--+ +1 row selected (1.6 seconds) + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924222155 | GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +# Realtime View +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924224537 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +exit +exit +``` + +#### Step 6(b): Run Spark SQL Queries + +Running the same queries in Spark-SQL: + +``` +docker exec -it adhoc-1 /bin/bash +bash-4.4# $SPARK_INSTALL/bin/spark-shell --jars $HUDI_SPARK_BUNDLE --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false --deploy-mode client --driver-memory 1G --master local[2] --executor-memory 3G --num-executors 1 --packages com.databricks:spark-avro_2.11:4.0.0 + +# Copy On Write Table: + +scala> spark.sql("select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'").show(100, false) ++------+-------------------+ +|symbol|max(ts) | ++------+-------------------+ +|GOOG |2018-08-31 10:59:00| ++------+-------------------+ + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'").show(100, false) + ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924221953 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924224524 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +As you can notice, the above queries now reflect the changes that came as part of ingesting second batch. + + +# Merge On Read Table: + +# Read Optimized View +scala> spark.sql("select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'").show(100, false) ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:29:00 | ++---------+----------------------+--+ +1 row selected (1.6 seconds) + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'").show(100, false) ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924222155 | GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +# Realtime View +scala> spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'").show(100, false) ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'").show(100, false) ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924224537 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +exit +exit +``` + +#### Step 7 : Incremental Query for COPY-ON-WRITE Table + +With 2 batches of data ingested, lets showcase the support for incremental queries in Hudi Copy-On-Write datasets + +Lets take the same projection query example +``` +docker exec -it adhoc-2 /bin/bash +beeline -u jdbc:hive2://hiveserver:10000 --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat --hiveconf hive.stats.autogather=false + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924064621 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924065039 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +``` + +As you notice from the above queries, there are 2 commits - 20180924064621 and 20180924065039 in timeline order. +When you follow the steps, you will be getting different timestamps for commits. Substitute them +in place of the above timestamps. + +To show the effects of incremental-query, let us assume that a reader has already seen the changes as part of +ingesting first batch. Now, for the reader to see effect of the second batch, he/she has to keep the start timestamp to +the commit time of the first batch (20180924064621) and run incremental query + +`Hudi incremental mode` provides efficient scanning for incremental queries by filtering out files that do not have any +candidate rows using hudi-managed metadata. + +``` +docker exec -it adhoc-2 /bin/bash +beeline -u jdbc:hive2://hiveserver:10000 --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat --hiveconf hive.stats.autogather=false +0: jdbc:hive2://hiveserver:10000> set hoodie.stock_ticks_cow.consume.mode=INCREMENTAL; +No rows affected (0.009 seconds) +0: jdbc:hive2://hiveserver:10000> set hoodie.stock_ticks_cow.consume.max.commits=3; +No rows affected (0.009 seconds) +0: jdbc:hive2://hiveserver:10000> set hoodie.stock_ticks_cow.consume.start.timestamp=20180924064621; + +# With the above setting, file-ids that do not have any updates from the commit 20180924065039 is filtered out without scanning. +# Here is the incremental query : + +0: jdbc:hive2://hiveserver:10000> +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG' and `_hoodie_commit_time` > '20180924064621'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924065039 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +1 row selected (0.83 seconds) +0: jdbc:hive2://hiveserver:10000> +``` + +##### Incremental Query with Spark SQL: +``` +docker exec -it adhoc-1 /bin/bash +bash-4.4# $SPARK_INSTALL/bin/spark-shell --jars $HUDI_SPARK_BUNDLE --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false --deploy-mode client --driver-memory 1G --master local[2] --executor-memory 3G --num-executors 1 --packages com.databricks:spark-avro_2.11:4.0.0 +Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version 2.3.1 + /_/ + +Using Scala version 2.11.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_181) +Type in expressions to have them evaluated. +Type :help for more information. + +scala> import com.uber.hoodie.DataSourceReadOptions +import com.uber.hoodie.DataSourceReadOptions + +# In the below query, 20180925045257 is the first commit's timestamp +scala> val hoodieIncViewDF = spark.read.format("com.uber.hoodie").option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL).option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "20180924064621").load("/user/hive/warehouse/stock_ticks_cow") +SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder". +SLF4J: Defaulting to no-operation (NOP) logger implementation +SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. +hoodieIncViewDF: org.apache.spark.sql.DataFrame = [_hoodie_commit_time: string, _hoodie_commit_seqno: string ... 15 more fields] + +scala> hoodieIncViewDF.registerTempTable("stock_ticks_cow_incr_tmp1") +warning: there was one deprecation warning; re-run with -deprecation for details + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow_incr_tmp1 where symbol = 'GOOG'").show(100, false); ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924065039 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +``` + + +#### Step 8: Schedule and Run Compaction for Merge-On-Read dataset + +Lets schedule and run a compaction to create a new version of columnar file so that read-optimized readers will see fresher data. +Again, You can use Hudi CLI to manually schedule and run compaction + +``` +docker exec -it adhoc-1 /bin/bash +^[[Aroot@adhoc-1:/opt# /var/hoodie/ws/hoodie-cli/hoodie-cli.sh +============================================ +* * +* _ _ _ _ * +* | | | | | (_) * +* | |__| | ___ ___ __| |_ ___ * +* | __ |/ _ \ / _ \ / _` | |/ _ \ * +* | | | | (_) | (_) | (_| | | __/ * +* |_| |_|\___/ \___/ \__,_|_|\___| * +* * +============================================ + +Welcome to Hoodie CLI. Please type help if you are looking for help. +hoodie->connect --path /user/hive/warehouse/stock_ticks_mor +18/09/24 06:59:34 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +18/09/24 06:59:35 INFO table.HoodieTableMetaClient: Loading HoodieTableMetaClient from /user/hive/warehouse/stock_ticks_mor +18/09/24 06:59:35 INFO util.FSUtils: Hadoop Configuration: fs.defaultFS: [hdfs://namenode:8020], Config:[Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml], FileSystem: [DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-1261652683_11, ugi=root (auth:SIMPLE)]]] +18/09/24 06:59:35 INFO table.HoodieTableConfig: Loading dataset properties from /user/hive/warehouse/stock_ticks_mor/.hoodie/hoodie.properties +18/09/24 06:59:36 INFO table.HoodieTableMetaClient: Finished Loading Table of type MERGE_ON_READ from /user/hive/warehouse/stock_ticks_mor +Metadata for table stock_ticks_mor loaded + +# Ensure no compactions are present + +hoodie:stock_ticks_mor->compactions show all +18/09/24 06:59:54 INFO timeline.HoodieActiveTimeline: Loaded instants [[20180924064636__clean__COMPLETED], [20180924064636__deltacommit__COMPLETED], [20180924065057__clean__COMPLETED], [20180924065057__deltacommit__COMPLETED]] + ___________________________________________________________________ + | Compaction Instant Time| State | Total FileIds to be Compacted| + |==================================================================| + +# Schedule a compaction. This will use Spark Launcher to schedule compaction +hoodie:stock_ticks_mor->compaction schedule +.... +Compaction successfully completed for 20180924070031 + +# Now refresh and check again. You will see that there is a new compaction requested + +hoodie:stock_ticks->connect --path /user/hive/warehouse/stock_ticks_mor +18/09/24 07:01:16 INFO table.HoodieTableMetaClient: Loading HoodieTableMetaClient from /user/hive/warehouse/stock_ticks_mor +18/09/24 07:01:16 INFO util.FSUtils: Hadoop Configuration: fs.defaultFS: [hdfs://namenode:8020], Config:[Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml], FileSystem: [DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-1261652683_11, ugi=root (auth:SIMPLE)]]] +18/09/24 07:01:16 INFO table.HoodieTableConfig: Loading dataset properties from /user/hive/warehouse/stock_ticks_mor/.hoodie/hoodie.properties +18/09/24 07:01:16 INFO table.HoodieTableMetaClient: Finished Loading Table of type MERGE_ON_READ from /user/hive/warehouse/stock_ticks_mor +Metadata for table stock_ticks_mor loaded + +hoodie:stock_ticks_mor->compactions show all +18/09/24 06:34:12 INFO timeline.HoodieActiveTimeline: Loaded instants [[20180924041125__clean__COMPLETED], [20180924041125__deltacommit__COMPLETED], [20180924042735__clean__COMPLETED], [20180924042735__deltacommit__COMPLETED], [==>20180924063245__compaction__REQUESTED]] + ___________________________________________________________________ + | Compaction Instant Time| State | Total FileIds to be Compacted| + |==================================================================| + | 20180924070031 | REQUESTED| 1 | + +# Execute the compaction. The compaction instant value passed below must be the one displayed in the above "compactions show all" query +hoodie:stock_ticks_mor->compaction run --compactionInstant 20180924070031 --parallelism 2 --sparkMemory 1G --schemaFilePath /var/demo/config/schema.avsc --retry 1 +.... +Compaction successfully completed for 20180924070031 + + +## Now check if compaction is completed + +hoodie:stock_ticks_mor->connect --path /user/hive/warehouse/stock_ticks_mor +18/09/24 07:03:00 INFO table.HoodieTableMetaClient: Loading HoodieTableMetaClient from /user/hive/warehouse/stock_ticks_mor +18/09/24 07:03:00 INFO util.FSUtils: Hadoop Configuration: fs.defaultFS: [hdfs://namenode:8020], Config:[Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml], FileSystem: [DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-1261652683_11, ugi=root (auth:SIMPLE)]]] +18/09/24 07:03:00 INFO table.HoodieTableConfig: Loading dataset properties from /user/hive/warehouse/stock_ticks_mor/.hoodie/hoodie.properties +18/09/24 07:03:00 INFO table.HoodieTableMetaClient: Finished Loading Table of type MERGE_ON_READ from /user/hive/warehouse/stock_ticks_mor +Metadata for table stock_ticks_mor loaded + +hoodie:stock_ticks->compactions show all +18/09/24 07:03:15 INFO timeline.HoodieActiveTimeline: Loaded instants [[20180924064636__clean__COMPLETED], [20180924064636__deltacommit__COMPLETED], [20180924065057__clean__COMPLETED], [20180924065057__deltacommit__COMPLETED], [20180924070031__commit__COMPLETED]] + ___________________________________________________________________ + | Compaction Instant Time| State | Total FileIds to be Compacted| + |==================================================================| + | 20180924070031 | COMPLETED| 1 | + +``` + +#### Step 9: Run Hive Queries including incremental queries + +You will see that both ReadOptimized and Realtime Views will show the latest committed data. +Lets also run the incremental query for MOR table. +From looking at the below query output, it will be clear that the fist commit time for the MOR table is 20180924064636 +and the second commit time is 20180924070031 + +``` +docker exec -it adhoc-2 /bin/bash +beeline -u jdbc:hive2://hiveserver:10000 --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat --hiveconf hive.stats.autogather=false + +# Read Optimized View +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ +1 row selected (1.6 seconds) + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924064636 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924070031 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +# Realtime View +0: jdbc:hive2://hiveserver:10000> select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'; +WARNING: Hive-on-MR is deprecated in Hive 2 and may not be available in the future versions. Consider using a different execution engine (i.e. spark, tez) or using Hive 1.X releases. ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924064636 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924070031 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +# Incremental View: + +0: jdbc:hive2://hiveserver:10000> set hoodie.stock_ticks_mor.consume.mode=INCREMENTAL; +No rows affected (0.008 seconds) +# Max-Commits covers both second batch and compaction commit +0: jdbc:hive2://hiveserver:10000> set hoodie.stock_ticks_mor.consume.max.commits=3; +No rows affected (0.007 seconds) +0: jdbc:hive2://hiveserver:10000> set hoodie.stock_ticks_mor.consume.start.timestamp=20180924064636; +No rows affected (0.013 seconds) +# Query: +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG' and `_hoodie_commit_time` > '20180924064636'; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924070031 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +exit +exit +``` + +##### Read Optimized and Realtime Views for MOR with Spark-SQL after compaction + +``` +docker exec -it adhoc-1 /bin/bash +bash-4.4# $SPARK_INSTALL/bin/spark-shell --jars $HUDI_SPARK_BUNDLE --driver-class-path $HADOOP_CONF_DIR --conf spark.sql.hive.convertMetastoreParquet=false --deploy-mode client --driver-memory 1G --master local[2] --executor-memory 3G --num-executors 1 --packages com.databricks:spark-avro_2.11:4.0.0 + +# Read Optimized View +scala> spark.sql("select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'").show(100, false) ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ +1 row selected (1.6 seconds) + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'").show(100, false) ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924064636 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924070031 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ + +# Realtime View +scala> spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'").show(100, false) ++---------+----------------------+--+ +| symbol | _c1 | ++---------+----------------------+--+ +| GOOG | 2018-08-31 10:59:00 | ++---------+----------------------+--+ + +scala> spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'").show(100, false) ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924064636 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924070031 | GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +``` + + +This brings the demo to an end. + +## Testing Hoodie in Local Docker environment + +You can bring up a hadoop docker environment containing Hadoop, Hive and Spark services with support for hoodie. +``` +$ mvn pre-integration-test -DskipTests +``` +The above command builds docker images for all the services with +current hoodie source installed at /var/hoodie/ws and also brings up the services using a compose file. We +currently use Hadoop (v2.8.4), Hive (v2.3.3) and Spark (v2.3.1) in docker images. + +To bring down the containers +``` +$ cd hoodie-integ-test +$ mvn docker-compose:down +``` + +If you want to bring up the docker containers, use +``` +$ cd hoodie-integ-test +$ mvn docker-compose:up -DdetachedMode=true +``` + +Hoodie is a library that is operated in a broader data analytics/ingestion environment +involving Hadoop, Hive and Spark. Interoperability with all these systems is a key objective for us. We are +actively adding integration-tests under __hoodie-integ-test/src/test/java__ that makes use of this +docker environment (See __hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java__ ) + + +#### Building Local Docker Containers: + +The docker images required for demo and running integration test are already in docker-hub. The docker images +and compose scripts are carefully implemented so that they serve dual-purpose + +1. The docker images have inbuilt hudi jar files with environment variable pointing to those jars (HUDI_HADOOP_BUNDLE, ...) +2. For running integration-tests, we need the jars generated locally to be used for running services within docker. The + docker-compose scripts (see `docker/compose/docker-compose_hadoop284_hive233_spark231.yml`) ensures local jars override + inbuilt jars by mounting local HUDI workspace over the docker location + +This helps avoid maintaining separate docker images and avoids the costly step of building HUDI docker images locally. +But if users want to test hudi from locations with lower network bandwidth, they can still build local images +run the script +`docker/build_local_docker_images.sh` to build local docker images before running `docker/setup_demo.sh` + +Here are the commands: + +``` +cd docker +./build_local_docker_images.sh +..... + +[INFO] Reactor Summary: +[INFO] +[INFO] Hoodie ............................................. SUCCESS [ 1.709 s] +[INFO] hoodie-common ...................................... SUCCESS [ 9.015 s] +[INFO] hoodie-hadoop-mr ................................... SUCCESS [ 1.108 s] +[INFO] hoodie-client ...................................... SUCCESS [ 4.409 s] +[INFO] hoodie-hive ........................................ SUCCESS [ 0.976 s] +[INFO] hoodie-spark ....................................... SUCCESS [ 26.522 s] +[INFO] hoodie-utilities ................................... SUCCESS [ 16.256 s] +[INFO] hoodie-cli ......................................... SUCCESS [ 11.341 s] +[INFO] hoodie-hadoop-mr-bundle ............................ SUCCESS [ 1.893 s] +[INFO] hoodie-hive-bundle ................................. SUCCESS [ 14.099 s] +[INFO] hoodie-spark-bundle ................................ SUCCESS [ 58.252 s] +[INFO] hoodie-hadoop-docker ............................... SUCCESS [ 0.612 s] +[INFO] hoodie-hadoop-base-docker .......................... SUCCESS [04:04 min] +[INFO] hoodie-hadoop-namenode-docker ...................... SUCCESS [ 6.142 s] +[INFO] hoodie-hadoop-datanode-docker ...................... SUCCESS [ 7.763 s] +[INFO] hoodie-hadoop-history-docker ....................... SUCCESS [ 5.922 s] +[INFO] hoodie-hadoop-hive-docker .......................... SUCCESS [ 56.152 s] +[INFO] hoodie-hadoop-sparkbase-docker ..................... SUCCESS [01:18 min] +[INFO] hoodie-hadoop-sparkmaster-docker ................... SUCCESS [ 2.964 s] +[INFO] hoodie-hadoop-sparkworker-docker ................... SUCCESS [ 3.032 s] +[INFO] hoodie-hadoop-sparkadhoc-docker .................... SUCCESS [ 2.764 s] +[INFO] hoodie-integ-test .................................. SUCCESS [ 1.785 s] +[INFO] ------------------------------------------------------------------------ +[INFO] BUILD SUCCESS +[INFO] ------------------------------------------------------------------------ +[INFO] Total time: 09:15 min +[INFO] Finished at: 2018-09-10T17:47:37-07:00 +[INFO] Final Memory: 236M/1848M +[INFO] ------------------------------------------------------------------------ +``` \ No newline at end of file diff --git a/hoodie-hive/run_sync_tool.sh b/hoodie-hive/run_sync_tool.sh old mode 100644 new mode 100755 diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java similarity index 92% rename from hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java rename to hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java index 610126c46298e..00475e1e9c8ca 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/MultiPartKeysValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java @@ -14,9 +14,8 @@ * limitations under the License. */ -package com.uber.hoodie.hive.util; +package com.uber.hoodie.hive; -import com.uber.hoodie.hive.PartitionValueExtractor; import java.util.Arrays; import java.util.List; diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java index be5a04e5d4664..06044feef5fee 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java @@ -26,7 +26,6 @@ import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent; import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType; -import com.uber.hoodie.hive.util.MultiPartKeysValueExtractor; import com.uber.hoodie.hive.util.SchemaUtil; import java.io.IOException; import java.net.URISyntaxException; diff --git a/hoodie-integ-test/pom.xml b/hoodie-integ-test/pom.xml new file mode 100644 index 0000000000000..ecacc9420fee2 --- /dev/null +++ b/hoodie-integ-test/pom.xml @@ -0,0 +1,212 @@ + + + + + + hoodie + com.uber.hoodie + 0.4.5-SNAPSHOT + ../pom.xml + + hoodie-integ-test + 4.0.0 + + + org.glassfish.jersey.connectors + jersey-apache-connector + 2.17 + + + org.glassfish.jersey.core + jersey-server + 2.17 + + + org.glassfish.jersey.containers + jersey-container-servlet-core + 2.17 + + + com.uber.hoodie + hoodie-spark + ${project.version} + + + org.glassfish.** + * + + + + + com.uber.hoodie + hoodie-common + ${project.version} + tests + test-jar + test + + + org.awaitility + awaitility + 3.1.2 + test + + + com.uber.hoodie + hoodie-spark + ${project.version} + tests + test-jar + test + + + org.glassfish.** + * + + + + + com.google.guava + guava + 20.0 + test + + + com.fasterxml.jackson.core + jackson-annotations + 2.6.4 + test + + + com.fasterxml.jackson.core + jackson-databind + 2.6.4 + test + + + com.fasterxml.jackson.datatype + jackson-datatype-guava + 2.9.4 + test + + + com.github.docker-java + docker-java + 3.1.0-rc-3 + test + + + org.glassfish.** + * + + + + + com.uber.hoodie + hoodie-hadoop-sparkworker-docker + ${project.version} + pom + import + + + junit + junit + ${junit.version} + test + + + + + ${project.basedir}/compose_env + ${project.basedir}/../docker/compose/docker-compose_hadoop284_hive233_spark231.yml + false + ${skipITs} + true + + + + + + exec-maven-plugin + org.codehaus.mojo + + + Setup HUDI_WS + generate-sources + + exec + + + /bin/bash + + -c + echo HUDI_WS=`dirname ${project.basedir}` + + ${dockerCompose.envFile} + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + 2.22.0 + + + + integration-test + + + + + + com.dkanejs.maven.plugins + docker-compose-maven-plugin + 2.0.1 + + + up + pre-integration-test + + up + + + ${docker.compose.skip} + unix:///var/run/docker.sock + ${project.basedir}/../docker/compose/docker-compose_hadoop284_hive233_spark231.yml + true + ${dockerCompose.envFile} + + + + down + integration-test + + down + + + ${docker.compose.skip} + ${project.basedir}/../docker/compose/docker-compose_hadoop284_hive233_spark231.yml + true + + + + + + + diff --git a/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestBase.java b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestBase.java new file mode 100644 index 0000000000000..dd093779efc64 --- /dev/null +++ b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestBase.java @@ -0,0 +1,178 @@ +/* + * 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.integ; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.awaitility.Awaitility.await; + +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.DockerCmdExecFactory; +import com.github.dockerjava.api.command.ExecCreateCmd; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import com.github.dockerjava.api.model.Container; +import com.github.dockerjava.core.DefaultDockerClientConfig; +import com.github.dockerjava.core.DockerClientBuilder; +import com.github.dockerjava.core.DockerClientConfig; +import com.github.dockerjava.core.command.ExecStartResultCallback; +import com.github.dockerjava.jaxrs.JerseyDockerCmdExecFactory; +import com.google.common.collect.ImmutableList; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.Assert; +import org.junit.Before; + +public abstract class ITTestBase { + + public static final Logger LOG = LogManager.getLogger(ITTestBase.class); + protected static final String SPARK_WORKER_CONTAINER = "/spark-worker-1"; + protected static final String ADHOC_1_CONTAINER = "/adhoc-1"; + protected static final String ADHOC_2_CONTAINER = "/adhoc-2"; + protected static final String HIVESERVER = "/hiveserver"; + protected static final String HOODIE_WS_ROOT = "/var/hoodie/ws"; + protected static final String HOODIE_JAVA_APP = HOODIE_WS_ROOT + "/hoodie-spark/run_hoodie_app.sh"; + protected static final String HUDI_HADOOP_BUNDLE = + HOODIE_WS_ROOT + "/docker/hoodie/hadoop/hive_base/target/hoodie-hadoop-mr-bundle.jar"; + protected static final String HUDI_HIVE_BUNDLE = + HOODIE_WS_ROOT + "/docker/hoodie/hadoop/hive_base/target/hoodie-hive-bundle.jar"; + protected static final String HUDI_SPARK_BUNDLE = + HOODIE_WS_ROOT + "/docker/hoodie/hadoop/hive_base/target/hoodie-spark-bundle.jar"; + protected static final String HIVE_SERVER_JDBC_URL = "jdbc:hive2://hiveserver:10000"; + // Skip these lines when capturing output from hive + protected static final Integer SLF4J_WARNING_LINE_COUNT_IN_HIVE_CMD = 9; + private static final String DEFAULT_DOCKER_HOST = "unix:///var/run/docker.sock"; + private static final String OVERRIDDEN_DOCKER_HOST = System.getenv("DOCKER_HOST"); + protected DockerClient dockerClient; + protected Map runningContainers; + + protected static String[] getHiveConsoleCommand(String rawCommand) { + String jarCommand = "add jar " + HUDI_HADOOP_BUNDLE + ";"; + String fullCommand = jarCommand + rawCommand; + + List cmd = new ImmutableList.Builder().add("hive") + .add("--hiveconf") + .add("hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat") + .add("--hiveconf") + .add("hive.stats.autogather=false") + .add("-e") + .add("\"" + fullCommand + "\"") + .build(); + return cmd.stream().toArray(String[]::new); + } + + @Before + public void init() throws IOException { + String dockerHost = (OVERRIDDEN_DOCKER_HOST != null) ? OVERRIDDEN_DOCKER_HOST : DEFAULT_DOCKER_HOST; + //Assuming insecure docker engine + DockerClientConfig config = DefaultDockerClientConfig.createDefaultConfigBuilder() + .withDockerHost(dockerHost) + .build(); + // using jaxrs/jersey implementation here (netty impl is also available) + DockerCmdExecFactory dockerCmdExecFactory = new JerseyDockerCmdExecFactory() + .withConnectTimeout(1000) + .withMaxTotalConnections(100) + .withMaxPerRouteConnections(10); + dockerClient = DockerClientBuilder.getInstance(config) + .withDockerCmdExecFactory(dockerCmdExecFactory) + .build(); + await().atMost(60, SECONDS).until(this::servicesUp); + } + + private boolean servicesUp() { + List containerList = dockerClient.listContainersCmd().exec(); + for (Container c : containerList) { + if (!c.getState().equalsIgnoreCase("running")) { + System.out.println("Container : " + Arrays.toString(c.getNames()) + + "not in running state, Curr State :" + c.getState()); + return false; + } + } + runningContainers = containerList.stream().map(c -> Pair.of(c.getNames()[0], c)) + .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + return true; + } + + protected TestExecStartResultCallback executeCommandInDocker(String containerName, String[] command, + boolean expectedToSucceed) + throws Exception { + LOG.info("Executing command (" + Arrays.toString(command) + ") in container " + containerName); + Container sparkWorkerContainer = runningContainers.get(containerName); + ExecCreateCmd cmd = dockerClient.execCreateCmd(sparkWorkerContainer.getId()) + .withCmd(command).withAttachStdout(true).withAttachStderr(true); + + ExecCreateCmdResponse createCmdResponse = cmd.exec(); + TestExecStartResultCallback callback = new TestExecStartResultCallback(new ByteArrayOutputStream(), + new ByteArrayOutputStream()); + dockerClient.execStartCmd(createCmdResponse.getId()).withDetach(false).withTty(false) + .exec(callback).awaitCompletion(); + int exitCode = dockerClient.inspectExecCmd(createCmdResponse.getId()).exec().getExitCode(); + LOG.info("Exit code for command (" + Arrays.toString(command) + ") is " + exitCode); + if (exitCode != 0) { + LOG.error("Command (" + Arrays.toString(command) + ") failed."); + LOG.error("Stdout is :" + callback.getStdout().toString()); + LOG.error("Stderr is :" + callback.getStderr().toString()); + } + if (expectedToSucceed) { + Assert.assertTrue("Command (" + Arrays.toString(command) + + ") expected to succeed. Exit (" + exitCode + ")", exitCode == 0); + } else { + Assert.assertTrue("Command (" + Arrays.toString(command) + + ") expected to fail. Exit (" + exitCode + ")", exitCode != 0); + } + cmd.close(); + return callback; + } + + public class TestExecStartResultCallback extends ExecStartResultCallback { + + // Storing the reference in subclass to expose to clients + private final ByteArrayOutputStream stdout; + private final ByteArrayOutputStream stderr; + + public TestExecStartResultCallback(ByteArrayOutputStream stdout, ByteArrayOutputStream stderr) { + super(stdout, stderr); + this.stdout = stdout; + this.stderr = stderr; + } + + @Override + public void onComplete() { + super.onComplete(); + LOG.info("onComplete called"); + try { + stderr.flush(); + stdout.flush(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public ByteArrayOutputStream getStdout() { + return stdout; + } + + public ByteArrayOutputStream getStderr() { + return stderr; + } + } +} diff --git a/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java new file mode 100644 index 0000000000000..7cc87469669dc --- /dev/null +++ b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java @@ -0,0 +1,139 @@ +/* + * 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.integ; + +import java.util.Arrays; +import org.junit.Assert; +import org.junit.Test; + +/** + * Smoke tests to run as part of verification. + */ +public class ITTestHoodieSanity extends ITTestBase { + + @Test + public void testRunEcho() throws Exception { + String[] cmd = new String[]{"echo", "Happy Testing"}; + TestExecStartResultCallback callback = executeCommandInDocker(ADHOC_1_CONTAINER, + cmd, true); + String stdout = callback.getStdout().toString(); + String stderr = callback.getStderr().toString(); + LOG.info("Got output for (" + Arrays.toString(cmd) + ") :" + stdout); + LOG.info("Got error output for (" + Arrays.toString(cmd) + ") :" + stderr); + } + + @Test + /** + * A basic integration test that runs HoodieJavaApp to create a sample COW Hoodie with single partition key + * data-set and performs upserts on it. Hive integration and upsert functionality is checked by running a count + * query in hive console. + */ + public void testRunHoodieJavaAppOnSinglePartitionKeyCOWTable() throws Exception { + String hiveTableName = "docker_hoodie_single_partition_key_cow_test"; + testRunHoodieJavaAppOnCOWTable(hiveTableName, true); + } + + @Test + /** + * A basic integration test that runs HoodieJavaApp to create a sample COW Hoodie with multiple partition-keys + * data-set and performs upserts on it. Hive integration and upsert functionality is checked by running a count + * query in hive console. + */ + public void testRunHoodieJavaAppOnMultiPartitionKeysCOWTable() throws Exception { + String hiveTableName = "docker_hoodie_multi_partition_key_cow_test"; + testRunHoodieJavaAppOnCOWTable(hiveTableName, false); + } + + /** + * A basic integration test that runs HoodieJavaApp to create a sample COW Hoodie + * data-set and performs upserts on it. Hive integration and upsert functionality is checked by running a count + * query in hive console. + * TODO: Add spark-shell test-case + */ + public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singlePartitionKey) throws Exception { + + // Drop Table if it exists + { + String[] hiveDropCmd = getHiveConsoleCommand("drop table if exists " + hiveTableName); + executeCommandInDocker(HIVESERVER, hiveDropCmd, true); + } + + // Ensure table does not exist + { + String[] hiveTableCheck = getHiveConsoleCommand("show tables like '" + hiveTableName + "'"); + TestExecStartResultCallback callback = + executeCommandInDocker(HIVESERVER, hiveTableCheck, true); + String stderr = callback.getStderr().toString(); + String stdout = callback.getStdout().toString(); + LOG.info("Got output for (" + Arrays.toString(hiveTableCheck) + ") :" + stdout); + LOG.info("Got error output for (" + Arrays.toString(hiveTableCheck) + ") :" + stderr); + Assert.assertTrue("Result :" + callback.getStdout().toString(), stdout.trim().isEmpty()); + } + + // Run Hoodie Java App + { + String[] cmd = null; + if (singlePartitionKey) { + cmd = new String[]{ + HOODIE_JAVA_APP, + "--hive-sync", + "--hive-url", HIVE_SERVER_JDBC_URL, + "--hive-table", hiveTableName + }; + } else { + cmd = new String[]{ + HOODIE_JAVA_APP, + "--hive-sync", + "--hive-url", HIVE_SERVER_JDBC_URL, + "--use-multi-partition-keys", + "--hive-table", hiveTableName + }; + } + TestExecStartResultCallback callback = executeCommandInDocker(ADHOC_1_CONTAINER, + cmd, true); + String stdout = callback.getStdout().toString().trim(); + String stderr = callback.getStderr().toString().trim(); + LOG.info("Got output for (" + Arrays.toString(cmd) + ") :" + stdout); + LOG.info("Got error output for (" + Arrays.toString(cmd) + ") :" + stderr); + } + + // Ensure table does exist + { + String[] hiveTableCheck = getHiveConsoleCommand("show tables like '" + hiveTableName + "'"); + TestExecStartResultCallback callback = + executeCommandInDocker(HIVESERVER, hiveTableCheck, true); + String stderr = callback.getStderr().toString().trim(); + String stdout = callback.getStdout().toString().trim(); + LOG.info("Got output for (" + Arrays.toString(hiveTableCheck) + ") : (" + stdout + ")"); + LOG.info("Got error output for (" + Arrays.toString(hiveTableCheck) + ") : (" + stderr + ")"); + Assert.assertEquals("Table exists", hiveTableName, stdout); + } + + // Ensure row count is 100 (without duplicates) + { + String[] hiveTableCheck = getHiveConsoleCommand("select count(1) from " + hiveTableName); + TestExecStartResultCallback callback = + executeCommandInDocker(ADHOC_1_CONTAINER, hiveTableCheck, true); + String stderr = callback.getStderr().toString().trim(); + String stdout = callback.getStdout().toString().trim(); + LOG.info("Got output for (" + Arrays.toString(hiveTableCheck) + ") : (" + stdout + ")"); + LOG.info("Got error output for (" + Arrays.toString(hiveTableCheck) + ") : (" + stderr + ")"); + Assert.assertEquals("Expecting 100 rows to be present in the new table", 100, + Integer.parseInt(stdout.trim())); + } + } +} diff --git a/hoodie-integ-test/src/test/resources/log4j-surefire.properties b/hoodie-integ-test/src/test/resources/log4j-surefire.properties new file mode 100644 index 0000000000000..3613e7d120a53 --- /dev/null +++ b/hoodie-integ-test/src/test/resources/log4j-surefire.properties @@ -0,0 +1,23 @@ +# +# 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. +# +log4j.rootLogger=WARN, A1 +log4j.category.com.uber=INFO +log4j.category.org.apache.parquet.hadoop=WARN +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 15fb3ae930b96..25d1fa7abbd74 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -111,6 +111,21 @@ + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + org.apache.rat apache-rat-plugin diff --git a/hoodie-spark/run_hoodie_app.sh b/hoodie-spark/run_hoodie_app.sh old mode 100644 new mode 100755 index ec82eddb912d5..2e070e9a5466d --- a/hoodie-spark/run_hoodie_app.sh +++ b/hoodie-spark/run_hoodie_app.sh @@ -21,4 +21,4 @@ fi OTHER_JARS=`ls -1 $DIR/target/lib/*jar | grep -v '*avro*-1.' | tr '\n' ':'` #TODO - Need to move TestDataGenerator and HoodieJavaApp out of tests echo "Running command : java -cp $DIR/target/test-classes/:$DIR/../hoodie-client/target/test-classes/:${HADOOP_CONF_DIR}:$HOODIE_JAR:${CLIENT_JAR}:$OTHER_JARS HoodieJavaApp $@" -java -cp $DIR/target/test-classes/:$DIR/../hoodie-client/target/test-classes/:${HADOOP_CONF_DIR}:$HOODIE_JAR:${CLIENT_JAR}:$OTHER_JARS HoodieJavaApp "$@" +java -Xmx1G -cp $DIR/target/test-classes/:$DIR/../hoodie-client/target/test-classes/:${HADOOP_CONF_DIR}:$HOODIE_JAR:${CLIENT_JAR}:$OTHER_JARS HoodieJavaApp "$@" diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala index 26040337f6e03..e52a0e93a0d76 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala @@ -153,7 +153,7 @@ object DataSourceWriteOptions { val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table" val HIVE_USER_OPT_KEY = "hoodie.datasource.hive_sync.username" val HIVE_PASS_OPT_KEY = "hoodie.datasource.hive_sync.password" - val HIVE_URL_OPT_KEY = "hoodie.datasource.hive_sync.jdbcUrl" + val HIVE_URL_OPT_KEY = "hoodie.datasource.hive_sync.jdbcurl" val HIVE_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.hive_sync.partition_fields" val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.hive_sync.partition_extractor_class" val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = "hoodie.datasource.hive_sync.assume_date_partitioning" diff --git a/hoodie-spark/src/test/java/HoodieJavaApp.java b/hoodie-spark/src/test/java/HoodieJavaApp.java index e7693183679e0..3cdc55136c5a1 100644 --- a/hoodie-spark/src/test/java/HoodieJavaApp.java +++ b/hoodie-spark/src/test/java/HoodieJavaApp.java @@ -24,6 +24,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.hive.MultiPartKeysValueExtractor; import java.util.List; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; @@ -65,7 +66,10 @@ public class HoodieJavaApp { private String hivePass = "hive"; @Parameter(names = {"--hive-url", "-hl"}, description = "hive JDBC URL") - private String hiveJdbcUrl = "jdbc:hive://localhost:10000"; + private String hiveJdbcUrl = "jdbc:hive2://localhost:10000"; + + @Parameter(names = {"--use-multi-partition-keys", "-mp"}, description = "Use Multiple Partition Keys") + private Boolean useMultiPartitionKeys = false; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -188,10 +192,16 @@ private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable) .option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB) .option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl) - .option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr") .option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser) .option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass) .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true"); + if (useMultiPartitionKeys) { + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day") + .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(), + MultiPartKeysValueExtractor.class.getCanonicalName()); + } else { + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr"); + } } return writer; } diff --git a/pom.xml b/pom.xml index 05501b8ab202c..e19f8199c39d7 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,8 @@ packaging/hoodie-hadoop-mr-bundle packaging/hoodie-hive-bundle packaging/hoodie-spark-bundle + docker/hoodie/hadoop + hoodie-integ-test From 67bd08690d8269057af814e26749bfce407e5b20 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Tue, 2 Oct 2018 10:30:39 +0530 Subject: [PATCH 117/374] Reflect new committership, id changes for devs --- pom.xml | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index e19f8199c39d7..b9ff0e50a8fe1 100644 --- a/pom.xml +++ b/pom.xml @@ -59,15 +59,20 @@ - vinoth + vinothchandar Vinoth Chandar Uber - prasanna + prasannarajaperumal Prasanna Rajaperumal Snowflake + + n3nash + Nishith Agarwal + Uber + @@ -95,10 +100,6 @@ Kaushik Devarajaiah Uber - - Nishith Agarwal - Uber - Balaji Varadharajan Uber From 37ce6f61c650ecbf5b69475578bb11e51374bc58 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 3 Oct 2018 18:02:09 +0100 Subject: [PATCH 118/374] Add --filter-dupes to DeltaStreamer - Optionally filter out duplicates before inserting data - Unit tests --- .../java/com/uber/hoodie/DataSourceUtils.java | 16 ++++++++++ .../deltastreamer/HoodieDeltaStreamer.java | 18 ++++++++++- .../utilities/TestHoodieDeltaStreamer.java | 32 +++++++++++++++++++ 3 files changed, 65 insertions(+), 1 deletion(-) diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index 5c0d27248daa7..f953b2a3d9dc7 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -26,6 +26,7 @@ import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.index.HoodieIndex; @@ -142,4 +143,19 @@ public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable order HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal); return new HoodieRecord<>(hKey, payload); } + + @SuppressWarnings("unchecked") + public static JavaRDD dropDuplicates(JavaSparkContext jssc, + JavaRDD incomingHoodieRecords, + HoodieWriteConfig writeConfig) throws Exception { + try { + HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig); + return client.tagLocation(incomingHoodieRecords) + .filter(r -> !((HoodieRecord) r).isCurrentLocationKnown()); + } catch (DatasetNotFoundException e) { + // this will be executed when there is no hoodie dataset yet + // so no dups to drop + return incomingHoodieRecords; + } + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index 835e7fa66a610..c3573fdeb0c42 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -179,8 +179,20 @@ public void sync() throws Exception { return new HoodieRecord<>(keyGenerator.getKey(gr), payload); }); - // Perform the write + // filter dupes if needed HoodieWriteConfig hoodieCfg = getHoodieClientConfig(); + if (cfg.filterDupes) { + // turn upserts to insert + cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation; + records = DataSourceUtils.dropDuplicates(jssc, records, hoodieCfg); + } + + if (records.isEmpty()) { + log.info("No new data, nothing to commit.. "); + return; + } + + // Perform the write HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg); String commitTime = client.startCommit(); log.info("Starting commit : " + commitTime); @@ -285,6 +297,10 @@ public static class Config implements Serializable { converter = OperationConvertor.class) public Operation operation = Operation.UPSERT; + @Parameter(names = {"--filter-dupes"}, description = "Should duplicate records from source be dropped/filtered out" + + "before insert/bulk-insert") + public Boolean filterDupes = false; + @Parameter(names = {"--spark-master"}, description = "spark master to use.") public String sparkMaster = "local[2]"; diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java index 1a17a687c00b9..63d93b4144a59 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java @@ -32,10 +32,12 @@ import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.Operation; import com.uber.hoodie.utilities.sources.TestDataSource; import java.io.IOException; +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.sql.Row; import org.apache.spark.sql.SQLContext; import org.junit.After; import org.junit.AfterClass; @@ -103,6 +105,11 @@ static void assertRecordCount(long expected, String datasetPath, SQLContext sqlC assertEquals(expected, recordCount); } + static List countsPerCommit(String datasetPath, SQLContext sqlContext) { + return sqlContext.read().format("com.uber.hoodie").load(datasetPath).groupBy("_hoodie_commit_time").count() + .sort("_hoodie_commit_time").collectAsList(); + } + static void assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits) throws IOException { HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath); @@ -159,5 +166,30 @@ public void testBulkInsertsAndUpserts() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2); + List counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext); + assertEquals(2000, counts.get(0).getLong(1)); + } + + @Test + public void testFilterDupes() throws Exception { + String datasetBasePath = dfsBasePath + "/test_dupes_dataset"; + + // Initial bulk insert + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT); + new HoodieDeltaStreamer(cfg, jsc).sync(); + TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); + + // Generate the same 1000 records + 1000 new ones for upsert + cfg.filterDupes = true; + cfg.sourceLimit = 2000; + cfg.operation = Operation.UPSERT; + new HoodieDeltaStreamer(cfg, jsc).sync(); + TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2); + // 1000 records for commit 00000 & 1000 for commit 00001 + List counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext); + assertEquals(1000, counts.get(0).getLong(1)); + assertEquals(1000, counts.get(1).getLong(1)); } } From 7e9b4d837a735befc9d0c6c09c9d6628e7ac21ab Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 11 Oct 2018 11:35:32 -0700 Subject: [PATCH 119/374] Fix regression which broke HudiInputFormat handling of non-hoodie datasets --- .../uber/hoodie/hadoop/HoodieInputFormat.java | 3 +- .../uber/hoodie/integ/ITTestHoodieSanity.java | 30 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index 4c0d548d7d0dc..c7931b5ce7e12 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -23,6 +23,7 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidDatasetException; import java.io.IOException; @@ -161,7 +162,7 @@ private Map> groupFileStatus(FileStatus[ metadata = getTableMetaClient(status.getPath().getFileSystem(conf), status.getPath().getParent()); nonHoodieBasePath = null; - } catch (InvalidDatasetException e) { + } catch (DatasetNotFoundException | InvalidDatasetException e) { LOG.info("Handling a non-hoodie path " + status.getPath()); metadata = null; nonHoodieBasePath = status.getPath().getParent().toString(); diff --git a/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java index 7cc87469669dc..9a1b694ff33ac 100644 --- a/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java +++ b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java @@ -66,6 +66,9 @@ public void testRunHoodieJavaAppOnMultiPartitionKeysCOWTable() throws Exception */ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singlePartitionKey) throws Exception { + String hdfsPath = "/" + hiveTableName; + String hdfsUrl = "hdfs://namenode" + hdfsPath; + // Drop Table if it exists { String[] hiveDropCmd = getHiveConsoleCommand("drop table if exists " + hiveTableName); @@ -91,6 +94,7 @@ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singleP cmd = new String[]{ HOODIE_JAVA_APP, "--hive-sync", + "--table-path", hdfsUrl, "--hive-url", HIVE_SERVER_JDBC_URL, "--hive-table", hiveTableName }; @@ -98,6 +102,7 @@ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singleP cmd = new String[]{ HOODIE_JAVA_APP, "--hive-sync", + "--table-path", hdfsUrl, "--hive-url", HIVE_SERVER_JDBC_URL, "--use-multi-partition-keys", "--hive-table", hiveTableName @@ -135,5 +140,30 @@ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singleP Assert.assertEquals("Expecting 100 rows to be present in the new table", 100, Integer.parseInt(stdout.trim())); } + + // Make the HDFS dataset non-hoodie and run the same query + // Checks for interoperability with non-hoodie tables + { + // Delete Hoodie directory to make it non-hoodie dataset + String[] cmd = new String[]{ + "hadoop", "fs", "-rm", "-r", hdfsPath + "/.hoodie" + }; + TestExecStartResultCallback callback = + executeCommandInDocker(ADHOC_1_CONTAINER, cmd, true); + String stderr = callback.getStderr().toString().trim(); + String stdout = callback.getStdout().toString().trim(); + LOG.info("Got output for (" + Arrays.toString(cmd) + ") : (" + stdout + ")"); + LOG.info("Got error output for (" + Arrays.toString(cmd) + ") : (" + stderr + ")"); + + // Run the count query again. Without Hoodie, all versions are included. So we get a wrong count + String[] hiveTableCheck = getHiveConsoleCommand("select count(1) from " + hiveTableName); + callback = executeCommandInDocker(ADHOC_1_CONTAINER, hiveTableCheck, true); + stderr = callback.getStderr().toString().trim(); + stdout = callback.getStdout().toString().trim(); + LOG.info("Got output for (" + Arrays.toString(hiveTableCheck) + ") : (" + stdout + ")"); + LOG.info("Got error output for (" + Arrays.toString(hiveTableCheck) + ") : (" + stderr + ")"); + Assert.assertEquals("Expecting 200 rows to be present in the new table", 200, + Integer.parseInt(stdout.trim())); + } } } From 42fc7f8598ada12cb1a75a6665c355c51dbbc1ab Mon Sep 17 00:00:00 2001 From: "jiale.tan" Date: Thu, 4 Oct 2018 17:56:51 -0700 Subject: [PATCH 120/374] feat(SparkDataSource): add additional feature to drop later arriving dups --- .../common/HoodieTestDataGenerator.java | 10 ++ .../java/com/uber/hoodie/DataSourceUtils.java | 11 ++ .../com/uber/hoodie/DataSourceOptions.scala | 7 + .../scala/com/uber/hoodie/DefaultSource.scala | 169 ++++++++++-------- .../src/test/scala/DataSourceTest.scala | 43 +++++ 5 files changed, 170 insertions(+), 70 deletions(-) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index b056bd805923d..06d66aeb408ab 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -196,6 +196,16 @@ public List generateInserts(String commitTime, Integer n) throws I return inserts; } + public List generateSameKeyInserts(String commitTime, List origin) throws IOException { + List copy = new ArrayList<>(); + for (HoodieRecord r: origin) { + HoodieKey key = r.getKey(); + HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime)); + copy.add(record); + } + return copy; + } + public List generateDeletes(String commitTime, Integer n) throws IOException { List inserts = generateInserts(commitTime, n); return generateDeletesFromExistingRecords(inserts); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index f953b2a3d9dc7..b02c36675a2fd 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -158,4 +158,15 @@ public static JavaRDD dropDuplicates(JavaSparkContext jssc, return incomingHoodieRecords; } } + + @SuppressWarnings("unchecked") + public static JavaRDD dropDuplicates(JavaSparkContext jssc, + JavaRDD incomingHoodieRecords, + Map parameters) throws Exception { + HoodieWriteConfig writeConfig = HoodieWriteConfig + .newBuilder() + .withPath(parameters.get("path")) + .withProps(parameters).build(); + return dropDuplicates(jssc, incomingHoodieRecords, writeConfig); + } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala index e52a0e93a0d76..63e1d19c5ed0e 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala @@ -145,6 +145,13 @@ object DataSourceWriteOptions { val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix" val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_" + /** + * Flag to indicate whether to drop duplicates upon insert. + * By default insert will accept duplicates, to gain extra performance. + */ + val INSERT_DROP_DUPS_OPT_KEY = "hoodie.datasource.write.insert.drop.duplicates" + val DEFAULT_INSERT_DROP_DUPS_OPT_VAL = "false" + // HIVE SYNC SPECIFIC CONFIGS //NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes // unexpected issues with config getting reset diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 68ac868e87e2f..608974baf7333 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -122,6 +122,7 @@ class DefaultSource extends RelationProvider defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL) defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL) + defaultsMap.putIfAbsent(INSERT_DROP_DUPS_OPT_KEY, DEFAULT_INSERT_DROP_DUPS_OPT_VAL) defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL) defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL) defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL) @@ -159,7 +160,21 @@ class DefaultSource extends RelationProvider } val storageType = parameters(STORAGE_TYPE_OPT_KEY) - val operation = parameters(OPERATION_OPT_KEY) + val operation = + // It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true + // Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly + // or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) . + if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean && + parameters(OPERATION_OPT_KEY) == UPSERT_OPERATION_OPT_VAL) { + + log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " + + s"when $INSERT_DROP_DUPS_OPT_KEY is set to be true, " + + s"overriding the $OPERATION_OPT_KEY to be $INSERT_OPERATION_OPT_VAL") + + INSERT_OPERATION_OPT_VAL + } else { + parameters(OPERATION_OPT_KEY) + } // register classes & schemas val structName = s"${tblName.get}_record" @@ -177,90 +192,104 @@ class DefaultSource extends RelationProvider toProperties(parameters) ) val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) - val hoodieRecords = genericRecords.map(gr => { + val hoodieAllIncomingRecords = genericRecords.map(gr => { val orderingVal = DataSourceUtils.getNestedFieldValAsString( gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]] DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY)) }).toJavaRDD(); + val jsc = new JavaSparkContext(sparkContext) - val basePath = new Path(parameters.get("path").get) - val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) - var exists = fs.exists(basePath) - - // Handle various save modes - if (mode == SaveMode.ErrorIfExists && exists) { - throw new HoodieException(s"basePath ${basePath} already exists.") - } - if (mode == SaveMode.Ignore && exists) { - log.warn(s" basePath ${basePath} already exists. Ignoring & not performing actual writes.") - return createRelation(sqlContext, parameters, df.schema) - } - if (mode == SaveMode.Overwrite && exists) { - log.warn(s" basePath ${basePath} already exists. Deleting existing data & overwriting with new data.") - fs.delete(basePath, true) - exists = false - } - - // Create the dataset if not present (APPEND mode) - if (!exists) { - HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType, - tblName.get, "archived") - } - - // Create a HoodieWriteClient & issue the write. - val jsc = new JavaSparkContext(sparkContext); - val client = DataSourceUtils.createHoodieClient(jsc, - schema.toString, - path.get, - tblName.get, - mapAsJavaMap(parameters) - ) - val commitTime = client.startCommit(); - - val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, commitTime, operation) - // Check for errors and commit the write. - val errorCount = writeStatuses.rdd.filter(ws => ws.hasErrors).count() - if (errorCount == 0) { - log.info("No errors. Proceeding to commit the write."); - val metaMap = parameters.filter(kv => - kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY))) - val success = if (metaMap.isEmpty) { - client.commit(commitTime, writeStatuses) + val hoodieRecords = + if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) { + DataSourceUtils.dropDuplicates( + jsc, + hoodieAllIncomingRecords, + mapAsJavaMap(parameters)) } else { - client.commit(commitTime, writeStatuses, - Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) + hoodieAllIncomingRecords } - if (success) { - log.info("Commit " + commitTime + " successful!") + if (!hoodieRecords.isEmpty()) { + val basePath = new Path(parameters.get("path").get) + val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) + var exists = fs.exists(basePath) + + // Handle various save modes + if (mode == SaveMode.ErrorIfExists && exists) { + throw new HoodieException(s"basePath ${basePath} already exists.") + } + if (mode == SaveMode.Ignore && exists) { + log.warn(s" basePath ${basePath} already exists. Ignoring & not performing actual writes.") + return createRelation(sqlContext, parameters, df.schema) } - else { - log.info("Commit " + commitTime + " failed!") + if (mode == SaveMode.Overwrite && exists) { + log.warn(s" basePath ${basePath} already exists. Deleting existing data & overwriting with new data.") + fs.delete(basePath, true) + exists = false } - val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false) - if (hiveSyncEnabled) { - log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")") - val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration) - syncHive(basePath, fs, parameters) + // Create the dataset if not present (APPEND mode) + if (!exists) { + HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType, + tblName.get, "archived") } - client.close - } else { - log.error(s"Upsert failed with ${errorCount} errors :"); - if (log.isTraceEnabled) { - log.trace("Printing out the top 100 errors") - writeStatuses.rdd.filter(ws => ws.hasErrors) - .take(100) - .foreach(ws => { - log.trace("Global error :", ws.getGlobalError) - if (ws.getErrors.size() > 0) { - ws.getErrors.foreach(kt => - log.trace(s"Error for key: ${kt._1}", kt._2)) - } - }) + + // Create a HoodieWriteClient & issue the write. + val client = DataSourceUtils.createHoodieClient(jsc, + schema.toString, + path.get, + tblName.get, + mapAsJavaMap(parameters) + ) + val commitTime = client.startCommit(); + + val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, commitTime, operation) + // Check for errors and commit the write. + val errorCount = writeStatuses.rdd.filter(ws => ws.hasErrors).count() + if (errorCount == 0) { + log.info("No errors. Proceeding to commit the write."); + val metaMap = parameters.filter(kv => + kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY))) + val success = if (metaMap.isEmpty) { + client.commit(commitTime, writeStatuses) + } else { + client.commit(commitTime, writeStatuses, + Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) + } + + if (success) { + log.info("Commit " + commitTime + " successful!") + } + else { + log.info("Commit " + commitTime + " failed!") + } + + val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false) + if (hiveSyncEnabled) { + log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")") + val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration) + syncHive(basePath, fs, parameters) + } + client.close + } else { + log.error(s"$operation failed with ${errorCount} errors :"); + if (log.isTraceEnabled) { + log.trace("Printing out the top 100 errors") + writeStatuses.rdd.filter(ws => ws.hasErrors) + .take(100) + .foreach(ws => { + log.trace("Global error :", ws.getGlobalError) + if (ws.getErrors.size() > 0) { + ws.getErrors.foreach(kt => + log.trace(s"Error for key: ${kt._1}", kt._2)) + } + }) + } } + } else { + log.info("new batch has no new records, skipping...") } createRelation(sqlContext, parameters, df.schema) } diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index f75192b352e6c..1ad42e7777ca5 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -139,4 +139,47 @@ class DataSourceTest extends AssertionsForJUnit { val hoodieROViewDF1 = spark.read.format("com.uber.hoodie").load(basePath + "/*/*/*/*") assertEquals(100, hoodieROViewDF1.count()) // still 100, since we only updated } + + @Test def testDropInsertDup(): Unit = { + val insert1Cnt = 10 + val insert2DupKeyCnt = 9 + val insert2NewKeyCnt = 2 + + val totalUniqueKeyToGenerate = insert1Cnt + insert2NewKeyCnt + val allRecords = dataGen.generateInserts("001", totalUniqueKeyToGenerate) + val inserts1 = allRecords.subList(0, insert1Cnt) + val inserts2New = dataGen.generateSameKeyInserts("002", allRecords.subList(insert1Cnt, insert1Cnt + insert2NewKeyCnt)) + val inserts2Dup = dataGen.generateSameKeyInserts("002", inserts1.subList(0, insert2DupKeyCnt)) + + val records1 = DataSourceTestUtils.convertToStringList(inserts1).toList + val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("com.uber.hoodie") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + val hoodieROViewDF1 = spark.read.format("com.uber.hoodie") + .load(basePath + "/*/*/*/*") + assertEquals(insert1Cnt, hoodieROViewDF1.count()) + + val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath) + val records2 = DataSourceTestUtils + .convertToStringList(inserts2Dup ++ inserts2New) + .toList + val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("com.uber.hoodie") + .options(commonOpts) + .option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true") + .mode(SaveMode.Append) + .save(basePath) + val hoodieROViewDF2 = spark.read.format("com.uber.hoodie") + .load(basePath + "/*/*/*/*") + assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate) + + val hoodieIncViewDF2 = spark.read.format("com.uber.hoodie") + .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .load(basePath) + assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt) + } } From a5d13ae529054dc3b3fb9125e178c60c990c30d9 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 25 Sep 2018 16:04:50 -0700 Subject: [PATCH 121/374] Adding documentation for migration guide and COW vs MOR tradeoffs, moving some docs around for more clarity --- docs/_data/sidebars/mydoc_sidebar.yml | 12 +++-- docs/concepts.md | 55 ++++++++++++++------- docs/configurations.md | 3 +- docs/incremental_processing.md | 2 +- docs/migration_guide.md | 71 +++++++++++++++++++++++++++ docs/powered_by.md | 6 +++ docs/quickstart.md | 26 ++++++---- 7 files changed, 143 insertions(+), 32 deletions(-) create mode 100644 docs/migration_guide.md diff --git a/docs/_data/sidebars/mydoc_sidebar.yml b/docs/_data/sidebars/mydoc_sidebar.yml index e790c982f7ae2..c4f6c46f2a2c9 100644 --- a/docs/_data/sidebars/mydoc_sidebar.yml +++ b/docs/_data/sidebars/mydoc_sidebar.yml @@ -35,18 +35,22 @@ entries: url: /concepts.html output: web - - title: Configurations - url: /configurations.html - output: web - - title: Implementation url: /implementation.html output: web + - title: Configurations + url: /configurations.html + output: web + - title: SQL Queries url: /sql_queries.html output: web + - title: Migration Guide + url: /migration_guide.html + output: web + - title: Incremental Processing url: /incremental_processing.html output: web diff --git a/docs/concepts.md b/docs/concepts.md index 4097efbb21aa0..e757a8944b5df 100644 --- a/docs/concepts.md +++ b/docs/concepts.md @@ -20,6 +20,8 @@ Such key activities include * `COMMITS` - A single commit captures information about an **atomic write** of a batch of records into a dataset. Commits are identified by a monotonically increasing timestamp, denoting the start of the write operation. * `CLEANS` - Background activity that gets rid of older versions of files in the dataset, that are no longer needed. + * `DELTA_COMMITS` - A single commit captures information about an **atomic write** of a batch of records into a + MergeOnRead storage type of dataset * `COMPACTIONS` - Background activity to reconcile differential data structures within Hoodie e.g: moving updates from row based log files to columnar formats. @@ -33,6 +35,24 @@ When there is late arriving data (data intended for 9:00 arriving >1 hr late at With the help of the timeline, an incremental query attempting to get all new data that was committed successfully since 10:00 hours, is able to very efficiently consume only the changed files without say scanning all the time buckets > 07:00. +## Terminologies + + * `Hudi Dataset` + A structured hive/spark dataset managed by Hudi. Hudi supports both partitioned and non-partitioned Hive tables. + * `Commit` + A commit marks a new batch of data applied to a dataset. Hudi maintains monotonically increasing timestamps to track commits and guarantees that a commit is atomically + published. + * `Commit Timeline` + Commit Timeline refers to the sequence of Commits that was applied in order on a dataset over its lifetime. + * `File Slice` + Hudi provides efficient handling of updates by having a fixed mapping between record key to a logical file Id. + Hudi uses MVCC to provide atomicity and isolation of readers from a writer. This means that a logical fileId will + have many physical versions of it. Each of these physical version of a file represents a complete view of the + file as of a commit and is called File Slice + * `File Group` + A file-group is a file-slice timeline. It is a list of file-slices in commit order. It is identified by `file id` + + ## Storage Types Hoodie storage types capture how data is indexed & laid out on the filesystem, and how the above primitives and timeline activities are implemented on top of @@ -62,23 +82,6 @@ Each record is uniquely identified by a `record key` and mapped to a file id for and file id, never changes once the first version of a record has been written to a file. In short, the `file id` identifies a group of files, that contain all versions of a group of records. -## Terminologies - - * `Hudi Dataset` - A structured hive/spark table managed by Hudi. Hudi supports both partitioned and non-partitioned Hive tables. - * `Commit` - A commit marks a new batch of data applied to a dataset. Hudi maintains monotonically increasing timestamps to track commits and guarantees that a commit is atomically - published. - * `Commit Timeline` - Commit Timeline refers to the sequence of Commits that was applied in order on a dataset over its lifetime. - * `File Slice` - Hudi provides efficient handling of updates by having a fixed mapping between record key to a logical file Id. - Hudi uses MVCC to provide atomicity and isolation of readers from a writer. This means that a logical fileId will - have many physical versions of it. Each of these physical version of a file represents a complete view of the - file as of a commit and is called File Slice - * `File Group` - A file-group is a file-slice timeline. It is a list of file-slices in commit order. It is identified by `file id` - ## Copy On Write @@ -136,3 +139,21 @@ There are lot of interesting things happening in this example, which bring out t The intention of merge on read storage, is to enable near real-time processing directly on top of Hadoop, as opposed to copying data out to specialized systems, which may not be able to handle the data volume. + +## Trade offs when choosing different storage types and views + +### Storage Types + +| Trade-off | CopyOnWrite | MergeOnRead | +|-------------- |------------------| ------------------| +| Data Latency | Higher | Lower | +| Update cost (I/O) | Higher (rewrite entire parquet) | Lower (append to delta file) | +| Parquet File Size | Smaller (high update(I/0) cost) | Larger (low update cost) | +| Write Amplification | Higher | Lower (depending on compaction strategy) | + +### Hudi Views + +| Trade-off | ReadOptimized | RealTime | +|-------------- |------------------| ------------------| +| Data Latency | Higher | Lower | +| Query Latency | Lower (raw columnar performance) | Higher (merge columnar + row based delta) | \ No newline at end of file diff --git a/docs/configurations.md b/docs/configurations.md index 74d3608548a63..ec3caf9e92fb5 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -160,7 +160,8 @@ summary: "Here we list all possible configurations and what they mean" Writing data via Hoodie happens as a Spark job and thus general rules of spark debugging applies here too. Below is a list of things to keep in mind, if you are looking to improving performance or reliability. - - **Right operations** : Use `bulkinsert` to load new data into a table, and there on use `upsert`/`insert`. Difference between them is that bulk insert uses a disk based write path to scale to load large inputs without need to cache it. + - **Write operations** : Use `bulkinsert` to load new data into a table, and there on use `upsert`/`insert`. + Difference between them is that bulk insert uses a disk based write path to scale to load large inputs without need to cache it. - **Input Parallelism** : By default, Hoodie tends to over-partition input (i.e `withParallelism(1500)`), to ensure each Spark partition stays within the 2GB limit for inputs upto 500GB. Bump this up accordingly if you have larger inputs. We recommend having shuffle parallelism `hoodie.[insert|upsert|bulkinsert].shuffle.parallelism` such that its atleast input_data_size/500MB - **Off-heap memory** : Hoodie writes parquet files and that needs good amount of off-heap memory proportional to schema width. Consider setting something like `spark.yarn.executor.memoryOverhead` or `spark.yarn.driver.memoryOverhead`, if you are running into such failures. - **Spark Memory** : Typically, hoodie needs to be able to read a single file into memory to perform merges or compactions and thus the executor memory should be sufficient to accomodate this. In addition, Hoodie caches the input to be able to intelligently place data and thus leaving some `spark.storage.memoryFraction` will generally help boost performance. diff --git a/docs/incremental_processing.md b/docs/incremental_processing.md index 3ea9b82de06b0..973875a0bb972 100644 --- a/docs/incremental_processing.md +++ b/docs/incremental_processing.md @@ -11,7 +11,7 @@ As discussed in the concepts section, the two basic primitives needed for [incre data using Hoodie are `upserts` (to apply changes to a dataset) and `incremental pulls` (to obtain a change stream/log from a dataset). This section discusses a few tools that can be used to achieve these on different contexts. -## Ingesting/Writing Data +## Incremental Ingestion Following means can be used to apply a delta or an incremental change to a Hoodie dataset. For e.g, the incremental changes could be from a Kafka topic or files uploaded to HDFS or even changes pulled from another Hoodie dataset. diff --git a/docs/migration_guide.md b/docs/migration_guide.md new file mode 100644 index 0000000000000..a5d5506c45e3a --- /dev/null +++ b/docs/migration_guide.md @@ -0,0 +1,71 @@ +--- +title: Migration Guide +keywords: migration guide +sidebar: mydoc_sidebar +permalink: migration_guide.html +toc: false +summary: In this page, we will discuss some available tools for migrating your existing dataset into a Hudi managed +dataset + + +Hudi maintains metadata such as commit timeline and indexes to manage a dataset. The commit timelines helps to understand the actions happening on a dataset as well as the current state of a dataset. Indexes are used by Hudi to maintain a record key to file id mapping to efficiently locate a record. At the moment, Hudi supports writing only parquet columnar formats. +To be able to start using Hudi for your existing dataset, you will need to migrate your existing dataset into a Hudi managed dataset. There are a couple of ways to achieve this. + + +## Approaches + + +### Approach 1 + +Hudi can be used to manage an existing dataset without affecting/altering the historical data already present in the +dataset. Hudi has been implemented to be compatible with such a mixed dataset with a caveat that either the complete +Hive partition is Hudi managed or not. Thus the lowest granularity at which Hudi manages a dataset is a Hive +partition. Start using the datasource API or the WriteClient to write to the dataset and make sure you start writing +to a new partition or convert your last N partitions into Hudi instead of the entire table. Note, since the historical + partitions are not managed by HUDI, none of the primitives provided by HUDI work on the data in those partitions. More concretely, one cannot perform upserts or incremental pull on such older partitions not managed by the HUDI dataset. +Take this approach if your dataset is an append only type of dataset and you do not expect to perform any updates to existing (or non Hudi managed) partitions. + + +### Approach 2 + +Import your existing dataset into a Hudi managed dataset. Since all the data is Hudi managed, none of the limitations + of Approach 1 apply here. Updates spanning any partitions can be applied to this dataset and Hudi will efficiently + make the update available to queries. Note that not only do you get to use all Hoodie primitives on this dataset, + there are other additional advantages of doing this. Hudi automatically manages file sizes of a Hudi managed dataset + . You can define the desired file size when converting this dataset and Hudi will ensure it writes out files + adhering to the config. It will also ensure that smaller files later get corrected by routing some new inserts into + small files rather than writing new small ones thus maintaining the health of your cluster. + +There are a few options when choosing this approach. +#### Option 1 +Use the HDFSParquetImporter tool. As the name suggests, this only works if your existing dataset is in +parquet file +format. This tool essentially starts a Spark Job to read the existing parquet dataset and converts it into a HUDI managed dataset by re-writing all the data. +#### Option 2 +For huge datasets, this could be as simple as : for partition in [list of partitions in source dataset] { + val inputDF = spark.read.format("any_input_format").load("partition_path") + inputDF.write.format("com.uber.hoodie").option()....save("basePath") + } +#### Option 3 +Write your own custom logic of how to load an existing dataset into a Hudi managed one. Please read about the RDD API + [here](quickstart.md). + +``` +Using the HDFSParquetImporter Tool. Once hoodie has been built via `mvn clean install -DskipTests`, the shell can be +fired by via `cd hoodie-cli && ./hoodie-cli.sh`. + +hoodie->hdfsparquetimport + --upsert false + --srcPath /user/parquet/dataset/basepath + --targetPath + /user/hoodie/dataset/basepath + --tableName hoodie_table + --tableType COPY_ON_WRITE + --rowKeyField _row_key + --partitionPathField partitionStr + --parallelism 1500 + --schemaFilePath /user/table/schema + --format parquet + --sparkMemory 6g + --retry 2 +``` \ No newline at end of file diff --git a/docs/powered_by.md b/docs/powered_by.md index 63abc226a53ff..d4d6fe1bf7003 100644 --- a/docs/powered_by.md +++ b/docs/powered_by.md @@ -26,7 +26,13 @@ It also powers several incremental Hive ETL pipelines and being currently integr 3. ["Incremental Processing on Large Analytical Datasets"](https://spark-summit.org/2017/events/incremental-processing-on-large-analytical-datasets/) - By Prasanna Rajaperumal June 2017, Spark Summit 2017, San Francisco, CA. [Slides](https://www.slideshare.net/databricks/incremental-processing-on-large-analytical-datasets-with-prasanna-rajaperumal-and-vinoth-chandar) [Video](https://www.youtube.com/watch?v=3HS0lQX-cgo&feature=youtu.be) +4. ["Hudi: Unifying storage and serving for batch and near-real-time analytics"](https://conferences.oreilly.com/strata/strata-ny/public/schedule/detail/70937) - By Nishith Agarwal & Balaji Vardarajan + September 2018, Strata Data Conference, New York, NY +5. ["Hudi: Large-Scale, Near Real-Time Pipelines at Uber"](https://databricks +.com/session/hudi-near-real-time-spark-pipelines-at-petabyte-scale) - By Vinoth Chander & Nishith Agarwal + October 2018, Spark+AI Summit Europe, London, UK + ## Articles 1. ["The Case for incremental processing on Hadoop"](https://www.oreilly.com/ideas/ubers-case-for-incremental-processing-on-hadoop) - O'reilly Ideas article by Vinoth Chandar diff --git a/docs/quickstart.md b/docs/quickstart.md index d025f3e0d670d..41ec9a96d0f6b 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -27,7 +27,7 @@ $ mvn clean install -DskipTests -DskipITs -Dhive11 {% include note.html content="Setup your local hadoop/hive test environment, so you can play with entire ecosystem. See [this](http://www.bytearray.io/2016/05/setting-up-hadoopyarnsparkhive-on-mac.html) for reference" %} -## Supported Versions +## Version Compatibility Hoodie requires Java 8 to be installed. Hoodie works with Spark-2.x versions. We have verified that hoodie works with the following combination of Hadoop/Hive/Spark. @@ -58,7 +58,13 @@ export SPARK_CONF_DIR=$SPARK_HOME/conf export PATH=$JAVA_HOME/bin:$HIVE_HOME/bin:$HADOOP_HOME/bin:$SPARK_INSTALL/bin:$PATH ``` -### DataSource API +### Supported API's + +Use the DataSource API to quickly start reading or writing hoodie datasets in few lines of code. Ideal for most +ingestion use-cases. +Use the RDD API to perform more involved actions on a hoodie dataset + +#### DataSource API Run __hoodie-spark/src/test/java/HoodieJavaApp.java__ class, to place a two commits (commit 1 => 100 inserts, commit 2 => 100 updates to previously inserted 100 records) onto your HDFS/local filesystem. Use the wrapper script to run from command-line @@ -86,14 +92,16 @@ Usage:
    [options] The class lets you choose table names, output paths and one of the storage types. In your own applications, be sure to include the `hoodie-spark` module as dependency and follow a similar pattern to write/read datasets via the datasource. -### RDD API +#### RDD API RDD level APIs give you more power and control over things, via the `hoodie-client` module . Refer to __hoodie-client/src/test/java/HoodieClientExample.java__ class for an example. -## Register Dataset to Hive Metastore +## Query a Hoodie dataset + +### Register Dataset to Hive Metastore Now, lets see how we can publish this data into Hive. @@ -215,11 +223,11 @@ ALTER TABLE `hoodie_rt` ADD IF NOT EXISTS PARTITION (datestr='2015-03-17') LOCAT -## Querying The Dataset +### Using different query engines Now, we can proceed to query the dataset, as we would normally do across all the three query engines supported. -### HiveQL +#### HiveQL Let's first perform a query on the latest committed snapshot of the table @@ -232,7 +240,7 @@ Time taken: 18.05 seconds, Fetched: 1 row(s) hive> ``` -### SparkSQL +#### SparkSQL Spark is super easy, once you get Hive working as above. Just spin up a Spark Shell as below @@ -249,7 +257,7 @@ scala> sqlContext.sql("select count(*) from hoodie_test").show(10000) You can also use the sample queries in __hoodie-utilities/src/test/java/HoodieSparkSQLExample.java__ for running on `hoodie_rt` -### Presto +#### Presto Checkout the 'master' branch on OSS Presto, build it, and place your installation somewhere. @@ -263,7 +271,7 @@ select count(*) from hive.default.hoodie_test -## Incremental Queries +## Incremental Queries of a Hoodie dataset Let's now perform a query, to obtain the __ONLY__ changed rows since a commit in the past. From 69bc152af8c970dd05a36f8aecbea19edc105246 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Thu, 11 Oct 2018 13:48:04 -0700 Subject: [PATCH 122/374] Fix addMetadataFields() to carry over 'props' --- .../hoodie/common/util/HoodieAvroUtils.java | 8 ++- .../common/util/TestHoodieAvroUtils.java | 57 +++++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/TestHoodieAvroUtils.java diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index 9c1dc2273e9e9..ae92b00f90262 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; @@ -40,6 +41,7 @@ import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.EncoderFactory; +import org.codehaus.jackson.JsonNode; /** * Helper class to do common stuff across Avro. @@ -108,7 +110,11 @@ public static Schema addMetadataFields(Schema schema) { parentFields.add(fileNameField); for (Schema.Field field : schema.getFields()) { if (!isMetadataField(field.name())) { - parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null)); + Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), null); + for (Map.Entry prop : field.getJsonProps().entrySet()) { + newField.addProp(prop.getKey(), prop.getValue()); + } + parentFields.add(newField); } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestHoodieAvroUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestHoodieAvroUtils.java new file mode 100644 index 0000000000000..fdae208c168f2 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestHoodieAvroUtils.java @@ -0,0 +1,57 @@ +/* + * 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.common.util; + +import java.util.Map; + +import org.apache.avro.Schema; +import org.codehaus.jackson.JsonNode; +import org.junit.Assert; +import org.junit.Test; + + +public class TestHoodieAvroUtils { + + private static String EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"testrec\"," + "\"fields\": [ " + + "{\"name\": \"timestamp\",\"type\": \"double\"}," + + "{\"name\": \"_row_key\", \"type\": \"string\"}," + + "{\"name\": \"non_pii_col\", \"type\": \"string\"}," + + "{\"name\": \"pii_col\", \"type\": \"string\", \"column_category\": \"user_profile\"}]}"; + + @Test + public void testPropsPresent() { + Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(EXAMPLE_SCHEMA)); + boolean piiPresent = false; + for (Schema.Field field : schema.getFields()) { + if (HoodieAvroUtils.isMetadataField(field.name())) { + continue; + } + + Assert.assertTrue("field name is null", field.name() != null); + Map props = field.getJsonProps(); + Assert.assertTrue("The property is null", props != null); + + if (field.name().equals("pii_col")) { + piiPresent = true; + Assert.assertTrue("sensitivity_level is removed in field 'pii_col'", props.containsKey("column_category")); + } else { + Assert.assertTrue("The property shows up but not set", props.size() == 0); + } + } + Assert.assertTrue("column pii_col doesn't show up", piiPresent); + } +} From 80d5be7c9fe5e83a8e6881d6e8f792e4d161f433 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 3 Oct 2018 10:39:10 -0700 Subject: [PATCH 123/374] Compaction validate, unschedule and repair --- docs/admin_guide.md | 80 ++- .../cli/commands/CompactionCommand.java | 287 ++++++++- .../uber/hoodie/cli/commands/SparkMain.java | 94 ++- .../uber/hoodie/CompactionAdminClient.java | 546 ++++++++++++++++++ .../com/uber/hoodie/func/OperationResult.java | 77 +++ .../hoodie/TestCompactionAdminClient.java | 363 ++++++++++++ .../hoodie/common/util/CompactionUtils.java | 19 +- .../common/util/CompactionTestUtils.java | 207 +++++++ .../common/util/TestCompactionUtils.java | 160 +---- .../utilities/HoodieCompactionAdminTool.java | 154 +++++ 10 files changed, 1830 insertions(+), 157 deletions(-) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java create mode 100644 hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java diff --git a/docs/admin_guide.md b/docs/admin_guide.md index 9b8c9981e44a1..f7fe91f8d4b13 100644 --- a/docs/admin_guide.md +++ b/docs/admin_guide.md @@ -161,6 +161,33 @@ hoodie:trips->commit showfiles --commit 20161005165855 --sortBy "Partition Path" .... ``` + +#### FileSystem View + +Hudi views each partition as a collection of file-groups with each file-group containing a list of file-slices in commit +order (See Concepts). The below commands allow users to view the file-slices for a data-set. + +``` + hoodie:stock_ticks_mor->show fsview all + .... + _______________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________ + | Partition | FileId | Base-Instant | Data-File | Data-File Size| Num Delta Files| Total Delta File Size| Delta Files | + |==============================================================================================================================================================================================================================================================================================================================================================================================================| + | 2018/08/31| 111415c3-f26d-4639-86c8-f9956f245ac3| 20181002180759| hdfs://namenode:8020/user/hive/warehouse/stock_ticks_mor/2018/08/31/111415c3-f26d-4639-86c8-f9956f245ac3_0_20181002180759.parquet| 432.5 KB | 1 | 20.8 KB | [HoodieLogFile {hdfs://namenode:8020/user/hive/warehouse/stock_ticks_mor/2018/08/31/.111415c3-f26d-4639-86c8-f9956f245ac3_20181002180759.log.1}]| + + + + hoodie:stock_ticks_mor->show fsview latest --partitionPath "2018/08/31" + ...... + __________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________ + | Partition | FileId | Base-Instant | Data-File | Data-File Size| Num Delta Files| Total Delta Size| Delta Size - compaction scheduled| Delta Size - compaction unscheduled| Delta To Base Ratio - compaction scheduled| Delta To Base Ratio - compaction unscheduled| Delta Files - compaction scheduled | Delta Files - compaction unscheduled| + |=================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================| + | 2018/08/31| 111415c3-f26d-4639-86c8-f9956f245ac3| 20181002180759| hdfs://namenode:8020/user/hive/warehouse/stock_ticks_mor/2018/08/31/111415c3-f26d-4639-86c8-f9956f245ac3_0_20181002180759.parquet| 432.5 KB | 1 | 20.8 KB | 20.8 KB | 0.0 B | 0.0 B | 0.0 B | [HoodieLogFile {hdfs://namenode:8020/user/hive/warehouse/stock_ticks_mor/2018/08/31/.111415c3-f26d-4639-86c8-f9956f245ac3_20181002180759.log.1}]| [] | + + hoodie:stock_ticks_mor-> +``` + + #### Statistics Since Hoodie directly manages file sizes for HDFS dataset, it might be good to get an overall picture @@ -280,31 +307,76 @@ Description: Run Compaction for given instant time * compaction run - Run Compaction for given instant time ``` -##### Up-Coming CLI for Compaction - -In the next release, more useful CLI to revert/repair compaction schedules will be added. Here is a preview of them: +##### Validate Compaction Validating a compaction plan : Check if all the files necessary for compactions are present and are valid ``` -hoodie:trips->compaction validate --compactionInstant +hoodie:stock_ticks_mor->compaction validate --instant 20181005222611 +... + + COMPACTION PLAN VALID + + ___________________________________________________________________________________________________________________________________________________________________________________________________________________________ + | File Id | Base Instant Time| Base Data File | Num Delta Files| Valid| Error| + |==========================================================================================================================================================================================================================| + | 05320e98-9a57-4c38-b809-a6beaaeb36bd| 20181005222445 | hdfs://namenode:8020/user/hive/warehouse/stock_ticks_mor/2018/08/31/05320e98-9a57-4c38-b809-a6beaaeb36bd_0_20181005222445.parquet| 1 | true | | + + + +hoodie:stock_ticks_mor->compaction validate --instant 20181005222601 + + COMPACTION PLAN INVALID + + _______________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________ + | File Id | Base Instant Time| Base Data File | Num Delta Files| Valid| Error | + |=====================================================================================================================================================================================================================================================================================================| + | 05320e98-9a57-4c38-b809-a6beaaeb36bd| 20181005222445 | hdfs://namenode:8020/user/hive/warehouse/stock_ticks_mor/2018/08/31/05320e98-9a57-4c38-b809-a6beaaeb36bd_0_20181005222445.parquet| 1 | false| All log files specified in compaction operation is not present. Missing .... | + + ``` +##### NOTE + The following commands must be executed without any other writer/ingestion application running. Sometimes, it becomes necessary to remove a fileId from a compaction-plan inorder to speed-up or unblock compaction operation. Any new log-files that happened on this file after the compaction got scheduled will be safely renamed so that are preserved. Hudi provides the following CLI to support it + +##### UnScheduling Compaction + ``` hoodie:trips->compaction unscheduleFileId --fileId +.... +No File renames needed to unschedule file from pending compaction. Operation successful. + ``` In other cases, an entire compaction plan needs to be reverted. This is supported by the following CLI ``` hoodie:trips->compaction unschedule --compactionInstant +..... +No File renames needed to unschedule pending compaction. Operation successful. ``` +##### Repair Compaction + +The above compaction unscheduling operations could sometimes fail partially (e:g -> HDFS temporarily unavailable). With +partial failures, the compaction operation could become inconsistent with the state of file-slices. When you run +`compaction validate`, you can notice invalid compaction operations if there is one. In these cases, the repair +command comes to the rescue, it will rearrange the file-slices so that there is no loss and the file-slices are +consistent with the compaction plan + +``` +hoodie:stock_ticks_mor->compaction repair --instant 20181005222611 +...... +Compaction successfully repaired +..... +``` + + ## Metrics Once the Hoodie Client is configured with the right datasetname and environment for metrics, it produces the following graphite metrics, that aid in debugging hoodie datasets diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java index 866e9315b418a..f5e3c894d3ee1 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java @@ -16,6 +16,10 @@ package com.uber.hoodie.cli.commands; +import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; + +import com.uber.hoodie.CompactionAdminClient.RenameOpResult; +import com.uber.hoodie.CompactionAdminClient.ValidationOpResult; import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.cli.HoodieCLI; @@ -32,14 +36,20 @@ import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; +import java.io.ObjectInputStream; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; +import org.apache.hadoop.fs.FSDataInputStream; +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.launcher.SparkLauncher; @@ -53,7 +63,9 @@ @Component public class CompactionCommand implements CommandMarker { - private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class); + private static Logger log = LogManager.getLogger(CompactionCommand.class); + + private static final String TMP_DIR = "/tmp/"; @CliAvailabilityIndicator({"compactions show all", "compaction show", "compaction run", "compaction schedule"}) public boolean isAvailable() { @@ -84,7 +96,7 @@ public String compactionsAll( for (int i = 0; i < instants.size(); i++) { HoodieInstant instant = instants.get(i); HoodieCompactionPlan workload = null; - if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) { + if (!instant.getAction().equals(COMPACTION_ACTION)) { try { // This could be a completed compaction. Assume a compaction request file is present but skip if fails workload = AvroUtils.deserializeCompactionPlan( @@ -203,7 +215,8 @@ public String compact( final String schemaFilePath, @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G", help = "Spark executor memory") final String sparkMemory, - @CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") final String retry, + @CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") + final String retry, @CliOption(key = "compactionInstant", mandatory = true, help = "Base path for the target hoodie dataset") final String compactionInstantTime) throws Exception { boolean initialized = HoodieCLI.initConf(); @@ -227,4 +240,272 @@ public String compact( throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); } } + + private static String getTmpSerializerFile() { + return TMP_DIR + UUID.randomUUID().toString() + ".ser"; + } + + private T deSerializeOperationResult(String inputP, FileSystem fs) throws Exception { + Path inputPath = new Path(inputP); + FSDataInputStream fsDataInputStream = fs.open(inputPath); + ObjectInputStream in = new ObjectInputStream(fsDataInputStream); + try { + T result = (T) in.readObject(); + log.info("Result : " + result); + return result; + } finally { + in.close(); + fsDataInputStream.close(); + } + } + + @CliCommand(value = "compaction validate", help = "Validate Compaction") + public String validateCompaction( + @CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant, + @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism, + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master, + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory, + @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit, + @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField, + @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending, + @CliOption(key = { + "headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly) + throws Exception { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + String outputPathStr = getTmpSerializerFile(); + Path outputPath = new Path(outputPathStr); + String output = null; + if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { + try { + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkCommand.COMPACT_VALIDATE.toString(), + HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master, + sparkMemory); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + if (exitCode != 0) { + return "Failed to validate compaction for " + compactionInstant; + } + List res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs); + boolean valid = res.stream().map(r -> r.isSuccess()).reduce(Boolean::logicalAnd).orElse(true); + String message = "\n\n\t COMPACTION PLAN " + (valid ? "VALID" : "INVALID") + "\n\n"; + List rows = new ArrayList<>(); + res.stream().forEach(r -> { + Comparable[] row = new Comparable[]{r.getOperation().getFileId(), + r.getOperation().getBaseInstantTime(), + r.getOperation().getDataFilePath().isPresent() ? r.getOperation().getDataFilePath().get() : "", + r.getOperation().getDeltaFilePaths().size(), r.isSuccess(), + r.getException().isPresent() ? r.getException().get().getMessage() : ""}; + rows.add(row); + }); + + Map> fieldNameToConverterMap = new HashMap<>(); + TableHeader header = new TableHeader() + .addTableHeaderField("File Id") + .addTableHeaderField("Base Instant Time") + .addTableHeaderField("Base Data File") + .addTableHeaderField("Num Delta Files") + .addTableHeaderField("Valid") + .addTableHeaderField("Error"); + + output = message + HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, + headerOnly, rows); + } finally { + // Delete tmp file used to serialize result + if (HoodieCLI.fs.exists(outputPath)) { + HoodieCLI.fs.delete(outputPath, false); + } + } + return output; + } else { + throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); + } + } + + @CliCommand(value = "compaction unschedule", help = "Unschedule Compaction") + public String unscheduleCompaction( + @CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant, + @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism, + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master, + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory, + @CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV, + @CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun, + @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit, + @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField, + @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending, + @CliOption(key = { + "headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly) + throws Exception { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + String outputPathStr = getTmpSerializerFile(); + Path outputPath = new Path(outputPathStr); + String output = ""; + if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { + try { + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_PLAN.toString(), + HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master, + sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString()); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + if (exitCode != 0) { + return "Failed to unschedule compaction for " + compactionInstant; + } + List res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs); + output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly, + "unschedule pending compaction"); + } finally { + // Delete tmp file used to serialize result + if (HoodieCLI.fs.exists(outputPath)) { + HoodieCLI.fs.delete(outputPath, false); + } + } + return output; + } else { + throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); + } + } + + @CliCommand(value = "compaction unscheduleFileId", help = "UnSchedule Compaction for a fileId") + public String unscheduleCompactFile( + @CliOption(key = "fileId", mandatory = true, help = "File Id") final String fileId, + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master, + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory, + @CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV, + @CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun, + @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit, + @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField, + @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending, + @CliOption(key = {"headeronly"}, help = "Header Only", unspecifiedDefaultValue = "false") boolean headerOnly) + throws Exception { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + String outputPathStr = getTmpSerializerFile(); + Path outputPath = new Path(outputPathStr); + String output = ""; + if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { + try { + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_FILE.toString(), + HoodieCLI.tableMetadata.getBasePath(), fileId, outputPathStr, "1", master, + sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString()); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + if (exitCode != 0) { + return "Failed to unschedule compaction for file " + fileId; + } + List res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs); + output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly, + "unschedule file from pending compaction"); + } finally { + // Delete tmp file used to serialize result + if (HoodieCLI.fs.exists(outputPath)) { + HoodieCLI.fs.delete(outputPath, false); + } + } + return output; + } else { + throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); + } + } + + @CliCommand(value = "compaction repair", help = "Renames the files to make them consistent with the timeline as " + + "dictated by Hoodie metadata. Use when compaction unschedule fails partially.") + public String repairCompaction( + @CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant, + @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism, + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master, + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory, + @CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun, + @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit, + @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField, + @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending, + @CliOption(key = { + "headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly) + throws Exception { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + String outputPathStr = getTmpSerializerFile(); + Path outputPath = new Path(outputPathStr); + String output = ""; + if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { + try { + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkCommand.COMPACT_REPAIR.toString(), + HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master, + sparkMemory, Boolean.valueOf(dryRun).toString()); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + if (exitCode != 0) { + return "Failed to unschedule compaction for " + compactionInstant; + } + List res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs); + output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly, "repair compaction"); + } finally { + // Delete tmp file used to serialize result + if (HoodieCLI.fs.exists(outputPath)) { + HoodieCLI.fs.delete(outputPath, false); + } + } + return output; + } else { + throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); + } + } + + private String getRenamesToBePrinted(List res, Integer limit, + String sortByField, boolean descending, boolean headerOnly, String operation) { + + Optional result = res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd); + if (result.isPresent()) { + System.out.println("There were some file renames that needed to be done to " + operation); + + if (result.get()) { + System.out.println("All renames successfully completed to " + operation + " done !!"); + } else { + System.out.println("Some renames failed. DataSet could be in inconsistent-state. " + + "Try running compaction repair"); + } + + List rows = new ArrayList<>(); + res.stream().forEach(r -> { + Comparable[] row = new Comparable[] { + r.getOperation().fileId, r.getOperation().srcPath, r.getOperation().destPath, + r.isExecuted(), r.isSuccess(), r.getException().isPresent() ? r.getException().get().getMessage() : "" + }; + rows.add(row); + }); + + Map> fieldNameToConverterMap = new HashMap<>(); + TableHeader header = new TableHeader() + .addTableHeaderField("File Id") + .addTableHeaderField("Source File Path") + .addTableHeaderField("Destination File Path") + .addTableHeaderField("Rename Executed?") + .addTableHeaderField("Rename Succeeded?") + .addTableHeaderField("Error"); + + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, + limit, headerOnly, rows); + } else { + return "No File renames needed to " + operation + ". Operation successful."; + } + } } \ No newline at end of file diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index bb9189ccb1d17..165ae67de3198 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -25,6 +25,8 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; import com.uber.hoodie.utilities.HDFSParquetImporter; +import com.uber.hoodie.utilities.HoodieCompactionAdminTool; +import com.uber.hoodie.utilities.HoodieCompactionAdminTool.Operation; import com.uber.hoodie.utilities.HoodieCompactor; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -39,6 +41,7 @@ public class SparkMain { */ enum SparkCommand { ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN, + COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR } public static void main(String[] args) throws Exception { @@ -78,10 +81,32 @@ public static void main(String[] args) throws Exception { returnCode = compact(jsc, args[1], args[2], args[3], 1, "", args[4], 0, true); break; + case COMPACT_VALIDATE: + assert (args.length == 7); + doCompactValidate(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6]); + returnCode = 0; + break; + case COMPACT_REPAIR: + assert (args.length == 8); + doCompactRepair(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6], + Boolean.valueOf(args[7])); + returnCode = 0; + break; + case COMPACT_UNSCHEDULE_FILE: + assert (args.length == 9); + doCompactUnscheduleFile(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6], + Boolean.valueOf(args[7]), Boolean.valueOf(args[8])); + returnCode = 0; + break; + case COMPACT_UNSCHEDULE_PLAN: + assert (args.length == 9); + doCompactUnschedule(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6], + Boolean.valueOf(args[7]), Boolean.valueOf(args[8])); + returnCode = 0; + break; default: break; } - System.exit(returnCode); } @@ -103,6 +128,73 @@ private static int dataLoad(JavaSparkContext jsc, String command, return new HDFSParquetImporter(cfg).dataImport(jsc, retry); } + private static void doCompactValidate(JavaSparkContext jsc, String basePath, String compactionInstant, + String outputPath, int parallelism, String sparkMaster, String sparkMemory) throws Exception { + HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config(); + cfg.basePath = basePath; + cfg.operation = Operation.VALIDATE; + cfg.outputPath = outputPath; + cfg.compactionInstantTime = compactionInstant; + cfg.parallelism = parallelism; + if ((null != sparkMaster) && (!sparkMaster.isEmpty())) { + jsc.getConf().setMaster(sparkMaster); + } + jsc.getConf().set("spark.executor.memory", sparkMemory); + new HoodieCompactionAdminTool(cfg).run(jsc); + } + + private static void doCompactRepair(JavaSparkContext jsc, String basePath, String compactionInstant, + String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean dryRun) throws Exception { + HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config(); + cfg.basePath = basePath; + cfg.operation = Operation.REPAIR; + cfg.outputPath = outputPath; + cfg.compactionInstantTime = compactionInstant; + cfg.parallelism = parallelism; + cfg.dryRun = dryRun; + if ((null != sparkMaster) && (!sparkMaster.isEmpty())) { + jsc.getConf().setMaster(sparkMaster); + } + jsc.getConf().set("spark.executor.memory", sparkMemory); + new HoodieCompactionAdminTool(cfg).run(jsc); + } + + private static void doCompactUnschedule(JavaSparkContext jsc, String basePath, String compactionInstant, + String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation, + boolean dryRun) throws Exception { + HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config(); + cfg.basePath = basePath; + cfg.operation = Operation.UNSCHEDULE_PLAN; + cfg.outputPath = outputPath; + cfg.compactionInstantTime = compactionInstant; + cfg.parallelism = parallelism; + cfg.dryRun = dryRun; + cfg.skipValidation = skipValidation; + if ((null != sparkMaster) && (!sparkMaster.isEmpty())) { + jsc.getConf().setMaster(sparkMaster); + } + jsc.getConf().set("spark.executor.memory", sparkMemory); + new HoodieCompactionAdminTool(cfg).run(jsc); + } + + private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId, + String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation, + boolean dryRun) throws Exception { + HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config(); + cfg.basePath = basePath; + cfg.operation = Operation.UNSCHEDULE_FILE; + cfg.outputPath = outputPath; + cfg.fileId = fileId; + cfg.parallelism = parallelism; + cfg.dryRun = dryRun; + cfg.skipValidation = skipValidation; + if ((null != sparkMaster) && (!sparkMaster.isEmpty())) { + jsc.getConf().setMaster(sparkMaster); + } + jsc.getConf().set("spark.executor.memory", sparkMemory); + new HoodieCompactionAdminTool(cfg).run(jsc); + } + private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant, int parallelism, String schemaFile, String sparkMemory, int retry, boolean schedule) throws Exception { HoodieCompactor.Config cfg = new HoodieCompactor.Config(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java new file mode 100644 index 0000000000000..2f6495f0540bf --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java @@ -0,0 +1,546 @@ +/* + * 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; + +import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; + +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.model.CompactionOperation; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieLogFile; +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.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.collection.Pair; +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.func.OperationResult; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +/** + * Client to perform admin operations related to compaction + */ +public class CompactionAdminClient implements Serializable { + + private static Logger log = LogManager.getLogger(CompactionAdminClient.class); + + private final transient JavaSparkContext jsc; + private final String basePath; + + public CompactionAdminClient(JavaSparkContext jsc, String basePath) { + this.jsc = jsc; + this.basePath = basePath; + } + + /** + * Validate all compaction operations in a compaction plan. Verifies the file-slices are consistent with corresponding + * compaction operations. + * + * @param metaClient Hoodie Table Meta Client + * @param compactionInstant Compaction Instant + */ + public List validateCompactionPlan(HoodieTableMetaClient metaClient, + String compactionInstant, int parallelism) throws IOException { + HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant); + HoodieTableFileSystemView fsView = + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + + if (plan.getOperations() != null) { + List ops = plan.getOperations().stream() + .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); + return jsc.parallelize(ops, parallelism).map(op -> { + try { + return validateCompactionOperation(metaClient, compactionInstant, op, Optional.of(fsView)); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }).collect(); + } + return new ArrayList<>(); + } + + /** + * Un-schedules compaction plan. Remove All compaction operation scheduled and re-arrange delta-files that were + * created after the compaction was scheduled. + * + * This operation MUST be executed with compactions and writer turned OFF. + * + * @param compactionInstant Compaction Instant + * @param skipValidation Skip validation step + * @param parallelism Parallelism + * @param dryRun Dry Run + */ + public List unscheduleCompactionPlan( + String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + List> renameActions = + getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism, + Optional.absent(), skipValidation); + + List res = + runRenamingOps(metaClient, renameActions, parallelism, dryRun); + + java.util.Optional success = + res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd); + Optional allSuccess = success.isPresent() ? Optional.of(success.get()) : Optional.absent(); + + // Only if all operations are successfully executed + if (!dryRun && allSuccess.isPresent() && allSuccess.get()) { + // Overwrite compaction request with empty compaction operations + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, compactionInstant); + HoodieCompactionPlan newPlan = + HoodieCompactionPlan.newBuilder().setOperations(new ArrayList<>()).setExtraMetadata(plan.getExtraMetadata()) + .build(); + HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant); + Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName()); + if (metaClient.getFs().exists(inflightPath)) { + // We need to rollback data-files because of this inflight compaction before unscheduling + throw new IllegalStateException("Please rollback the inflight compaction before unscheduling"); + } + metaClient.getActiveTimeline().saveToCompactionRequested( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant), + AvroUtils.serializeCompactionPlan(newPlan)); + } + return res; + } + + /** + * Remove a fileId from pending compaction. Removes the associated compaction operation and rename delta-files + * that were generated for that file-id after the compaction operation was scheduled. + * + * This operation MUST be executed with compactions and writer turned OFF. + * + * @param fileId FileId to be unscheduled + * @param skipValidation Skip validation + * @param dryRun Dry Run Mode + */ + public List unscheduleCompactionFileId(String fileId, + boolean skipValidation, boolean dryRun) throws Exception { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + List> renameActions = + getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fileId, Optional.absent(), skipValidation); + + List res = runRenamingOps(metaClient, renameActions, 1, dryRun); + + if (!dryRun && !res.isEmpty() && res.get(0).isExecuted() && res.get(0).isSuccess()) { + // Ready to remove this file-Id from compaction request + Pair compactionOperationWithInstant = + CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fileId); + HoodieCompactionPlan plan = CompactionUtils + .getCompactionPlan(metaClient, compactionOperationWithInstant.getKey()); + List newOps = plan.getOperations().stream() + .filter(op -> !op.getFileId().equals(fileId)).collect(Collectors.toList()); + HoodieCompactionPlan newPlan = + HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build(); + HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, + compactionOperationWithInstant.getLeft()); + Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName()); + if (metaClient.getFs().exists(inflightPath)) { + // revert if in inflight state + metaClient.getActiveTimeline().revertCompactionInflightToRequested(inflight); + } + metaClient.getActiveTimeline().saveToCompactionRequested( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()), + AvroUtils.serializeCompactionPlan(newPlan)); + } + return res; + } + + /** + * Renames delta files to make file-slices consistent with the timeline as dictated by Hoodie metadata. + * Use when compaction unschedule fails partially. + * + * This operation MUST be executed with compactions and writer turned OFF. + * @param compactionInstant Compaction Instant to be repaired + * @param dryRun Dry Run Mode + */ + public List repairCompaction(String compactionInstant, + int parallelism, boolean dryRun) throws Exception { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + List validationResults = + validateCompactionPlan(metaClient, compactionInstant, parallelism); + List failed = validationResults.stream() + .filter(v -> !v.isSuccess()).collect(Collectors.toList()); + if (failed.isEmpty()) { + return new ArrayList<>(); + } + + final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsAndCompactionTimeline()); + List> renameActions = failed.stream().flatMap(v -> + getRenamingActionsToAlignWithCompactionOperation(metaClient, compactionInstant, + v.getOperation(), Optional.of(fsView)).stream()).collect(Collectors.toList()); + return runRenamingOps(metaClient, renameActions, parallelism, dryRun); + } + + /** + * Construction Compaction Plan from compaction instant + */ + private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, + String compactionInstant) throws IOException { + HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( + metaClient.getActiveTimeline().getInstantAuxiliaryDetails( + HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get()); + return compactionPlan; + } + + /** + * Get Renaming actions to ensure the log-files of merged file-slices is aligned with compaction operation. This + * method is used to recover from failures during unschedule compaction operations. + * + * @param metaClient Hoodie Table Meta Client + * @param compactionInstant Compaction Instant + * @param op Compaction Operation + * @param fsViewOpt File System View + */ + protected static List> getRenamingActionsToAlignWithCompactionOperation( + HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation op, + Optional fsViewOpt) { + HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() : + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get(); + FileSlice merged = + fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp()) + .filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get(); + final int maxVersion = + op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf))) + .reduce((x, y) -> x > y ? x : y).map(x -> x).orElse(0); + List logFilesToBeMoved = + merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList()); + return logFilesToBeMoved.stream().map(lf -> { + Preconditions.checkArgument(lf.getLogVersion() - maxVersion > 0, + "Expect new log version to be sane"); + HoodieLogFile newLogFile = new HoodieLogFile(new Path(lf.getPath().getParent(), + FSUtils.makeLogFileName(lf.getFileId(), "." + FSUtils.getFileExtensionFromLog(lf.getPath()), + compactionInstant, lf.getLogVersion() - maxVersion))); + return Pair.of(lf, newLogFile); + }).collect(Collectors.toList()); + } + + /** + * Rename log files. This is done for un-scheduling a pending compaction operation NOTE: Can only be used safely when + * no writer (ingestion/compaction) is running. + * + * @param metaClient Hoodie Table Meta-Client + * @param oldLogFile Old Log File + * @param newLogFile New Log File + */ + protected static void renameLogFile(HoodieTableMetaClient metaClient, HoodieLogFile oldLogFile, + HoodieLogFile newLogFile) throws IOException { + FileStatus[] statuses = metaClient.getFs().listStatus(oldLogFile.getPath()); + Preconditions.checkArgument(statuses.length == 1, "Only one status must be present"); + Preconditions.checkArgument(statuses[0].isFile(), "Source File must exist"); + Preconditions.checkArgument(oldLogFile.getPath().getParent().equals(newLogFile.getPath().getParent()), + "Log file must only be moved within the parent directory"); + metaClient.getFs().rename(oldLogFile.getPath(), newLogFile.getPath()); + } + + /** + * Check if a compaction operation is valid + * + * @param metaClient Hoodie Table Meta client + * @param compactionInstant Compaction Instant + * @param operation Compaction Operation + * @param fsViewOpt File System View + */ + private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient metaClient, + String compactionInstant, CompactionOperation operation, Optional fsViewOpt) + throws IOException { + HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() : + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + java.util.Optional lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant(); + try { + if (lastInstant.isPresent()) { + java.util.Optional fileSliceOptional = + fileSystemView.getLatestUnCompactedFileSlices(operation.getPartitionPath()) + .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst(); + if (fileSliceOptional.isPresent()) { + FileSlice fs = fileSliceOptional.get(); + java.util.Optional df = fs.getDataFile(); + if (operation.getDataFilePath().isPresent()) { + String expPath = metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath() + .toString(); + Preconditions.checkArgument(df.isPresent(), "Data File must be present. File Slice was : " + + fs + ", operation :" + operation); + Preconditions.checkArgument(df.get().getPath().equals(expPath), + "Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath()); + } + Set logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet()); + Set logFilesInCompactionOp = operation.getDeltaFilePaths().stream() + .map(dp -> { + try { + FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(dp)); + Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status"); + return new HoodieLogFile(fileStatuses[0]); + } catch (FileNotFoundException fe) { + throw new CompactionValidationException(fe.getMessage()); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }).collect(Collectors.toSet()); + Set missing = + logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf)) + .collect(Collectors.toSet()); + Preconditions.checkArgument(missing.isEmpty(), + "All log files specified in compaction operation is not present. Missing :" + missing + + ", Exp :" + logFilesInCompactionOp + ", Got :" + logFilesInFileSlice); + Set diff = + logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf)) + .collect(Collectors.toSet()); + Preconditions.checkArgument(diff.stream() + .filter(lf -> !lf.getBaseCommitTime().equals(compactionInstant)).count() == 0, + "There are some log-files which are neither specified in compaction plan " + + "nor present after compaction request instant. Some of these :" + diff); + } else { + throw new CompactionValidationException("Unable to find file-slice for file-id (" + operation.getFileId() + + " Compaction operation is invalid."); + } + } else { + throw new CompactionValidationException("Unable to find any committed instant. Compaction Operation may " + + "be pointing to stale file-slices"); + } + } catch (CompactionValidationException | IllegalArgumentException e) { + return new ValidationOpResult(operation, false, Optional.of(e)); + } + return new ValidationOpResult(operation, true, Optional.absent()); + } + + /** + * Execute Renaming operation + * + * @param metaClient HoodieTable MetaClient + * @param renameActions List of rename operations + */ + private List runRenamingOps(HoodieTableMetaClient metaClient, + List> renameActions, int parallelism, boolean dryRun) { + if (renameActions.isEmpty()) { + log.info("No renaming of log-files needed. Proceeding to removing file-id from compaction-plan"); + return new ArrayList<>(); + } else { + log.info("The following compaction renaming operations needs to be performed to un-schedule"); + if (!dryRun) { + return jsc.parallelize(renameActions, parallelism).map(lfPair -> { + try { + log.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath()); + renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight()); + return new RenameOpResult(lfPair, true, Optional.absent()); + } catch (IOException e) { + log.error("Error renaming log file", e); + log.error("\n\n\n***NOTE Compaction is in inconsistent state. Try running \"compaction repair " + + lfPair.getLeft().getBaseCommitTime() + "\" to recover from failure ***\n\n\n"); + return new RenameOpResult(lfPair, false, Optional.of(e)); + } + }).collect(); + } else { + log.info("Dry-Run Mode activated for rename operations"); + return renameActions.parallelStream() + .map(lfPair -> new RenameOpResult(lfPair, false, false, Optional.absent())) + .collect(Collectors.toList()); + } + } + } + + /** + * Generate renaming actions for unscheduling a pending compaction plan. NOTE: Can only be used safely when no writer + * (ingestion/compaction) is running. + * + * @param metaClient Hoodie Table MetaClient + * @param compactionInstant Compaction Instant to be unscheduled + * @param fsViewOpt Cached File System View + * @param skipValidation Skip Validation + * @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule + * compaction. + */ + protected List> getRenamingActionsForUnschedulingCompactionPlan( + HoodieTableMetaClient metaClient, String compactionInstant, int parallelism, + Optional fsViewOpt, boolean skipValidation) throws IOException { + HoodieTableFileSystemView fsView = fsViewOpt.isPresent() ? fsViewOpt.get() : + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant); + if (plan.getOperations() != null) { + log.info("Number of Compaction Operations :" + plan.getOperations().size() + + " for instant :" + compactionInstant); + List ops = plan.getOperations().stream() + .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); + return jsc.parallelize(ops, parallelism).flatMap(op -> { + try { + return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, + op, Optional.of(fsView), skipValidation).iterator(); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } catch (CompactionValidationException ve) { + throw new HoodieException(ve); + } + }).collect(); + } + log.warn("No operations for compaction instant : " + compactionInstant); + return new ArrayList<>(); + } + + /** + * Generate renaming actions for unscheduling a compaction operation NOTE: Can only be used safely when no writer + * (ingestion/compaction) is running. + * + * @param metaClient Hoodie Table MetaClient + * @param compactionInstant Compaction Instant + * @param operation Compaction Operation + * @param fsViewOpt Cached File System View + * @param skipValidation Skip Validation + * @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule + * compaction. + */ + public List> getRenamingActionsForUnschedulingCompactionOperation( + HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation operation, + Optional fsViewOpt, boolean skipValidation) throws IOException { + List> result = new ArrayList<>(); + HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() : + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + if (!skipValidation) { + validateCompactionOperation(metaClient, compactionInstant, operation, Optional.of(fileSystemView)); + } + HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get(); + FileSlice merged = + fileSystemView.getLatestMergedFileSlicesBeforeOrOn(operation.getPartitionPath(), lastInstant.getTimestamp()) + .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get(); + List logFilesToRepair = + merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant)) + .collect(Collectors.toList()); + logFilesToRepair.sort(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed()); + FileSlice fileSliceForCompaction = + fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime()) + .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get(); + int maxUsedVersion = + fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getLogVersion()) + .orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1); + String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension()) + .orElse(HoodieLogFile.DELTA_EXTENSION); + String parentPath = fileSliceForCompaction.getDataFile().map(df -> new Path(df.getPath()).getParent().toString()) + .orElse(fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getPath().getParent().toString()).get()); + for (HoodieLogFile toRepair : logFilesToRepair) { + int version = maxUsedVersion + 1; + HoodieLogFile newLf = new HoodieLogFile(new Path(parentPath, FSUtils.makeLogFileName(operation.getFileId(), + logExtn, operation.getBaseInstantTime(), version))); + result.add(Pair.of(toRepair, newLf)); + maxUsedVersion = version; + } + return result; + } + + /** + * Generate renaming actions for unscheduling a fileId from pending compaction. NOTE: Can only be used safely when no + * writer (ingestion/compaction) is running. + * + * @param metaClient Hoodie Table MetaClient + * @param fileId FileId to remove compaction + * @param fsViewOpt Cached File System View + * @param skipValidation Skip Validation + * @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule + * compaction. + */ + public List> getRenamingActionsForUnschedulingCompactionForFileId( + HoodieTableMetaClient metaClient, String fileId, Optional fsViewOpt, + boolean skipValidation) throws IOException { + Map> allPendingCompactions = + CompactionUtils.getAllPendingCompactionOperations(metaClient); + if (allPendingCompactions.containsKey(fileId)) { + Pair opWithInstant = allPendingCompactions.get(fileId); + return getRenamingActionsForUnschedulingCompactionOperation(metaClient, opWithInstant.getKey(), + CompactionOperation.convertFromAvroRecordInstance(opWithInstant.getValue()), fsViewOpt, skipValidation); + } + throw new HoodieException("FileId " + fileId + " not in pending compaction"); + } + + /** + * Holds Operation result for Renaming + */ + public static class RenameOpResult extends OperationResult { + + public RenameOpResult() { + } + + public RenameOpResult(Pair op, boolean success, + Optional exception) { + super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(), + op.getRight().getPath().toString()), success, exception); + } + + public RenameOpResult( + Pair op, boolean executed, boolean success, + Optional exception) { + super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(), + op.getRight().getPath().toString()), executed, success, exception); + } + } + + /** + * Holds Operation result for Renaming + */ + public static class ValidationOpResult extends OperationResult { + + public ValidationOpResult() { + } + + public ValidationOpResult( + CompactionOperation operation, boolean success, Optional exception) { + super(operation, success, exception); + } + } + + public static class RenameInfo implements Serializable { + + public String fileId; + public String srcPath; + public String destPath; + + public RenameInfo() { + } + + public RenameInfo(String fileId, String srcPath, String destPath) { + this.fileId = fileId; + this.srcPath = srcPath; + this.destPath = destPath; + } + } + + public static class CompactionValidationException extends RuntimeException { + + public CompactionValidationException(String msg) { + super(msg); + } + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java b/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java new file mode 100644 index 0000000000000..38f368bb4dba3 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java @@ -0,0 +1,77 @@ +/* + * 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.func; + +import com.google.common.base.Optional; +import java.io.Serializable; + +/** + * Holds Operation result. Used as a result container for Compaction Admin Client (running as part of Spark-launcher + * process) to communicate results back to Hoodie CLI process. + */ +public class OperationResult implements Serializable { + + private T operation; + private boolean executed; + private boolean success; + private Optional exception; + + public OperationResult() { + } + + public OperationResult(T operation, boolean success, Optional exception) { + this.operation = operation; + this.success = success; + this.exception = exception; + this.executed = true; + } + + public OperationResult(T operation, boolean executed, boolean success, Optional exception) { + this.operation = operation; + this.success = success; + this.exception = exception; + this.executed = executed; + } + + public T getOperation() { + return operation; + } + + public boolean isSuccess() { + return success; + } + + public boolean isExecuted() { + return executed; + } + + public Optional getException() { + return exception; + } + + @Override + public String toString() { + return "OperationResult{" + + "operation=" + operation + + ", executed=" + executed + + ", success=" + success + + ", exception=" + exception + + '}'; + } +} \ No newline at end of file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java new file mode 100644 index 0000000000000..a385f09475fb4 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java @@ -0,0 +1,363 @@ +/* + * 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; + +import static com.uber.hoodie.common.model.HoodieTestUtils.getDefaultHadoopConf; + +import com.google.common.base.Optional; +import com.uber.hoodie.CompactionAdminClient.ValidationOpResult; +import com.uber.hoodie.common.model.CompactionOperation; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.CompactionTestUtils; +import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.collection.Pair; +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestCompactionAdminClient extends TestHoodieClientBase { + + private HoodieTableMetaClient metaClient; + private CompactionAdminClient client; + + @Before + public void init() throws IOException { + super.init(); + metaClient = HoodieTestUtils.initTableType(getDefaultHadoopConf(), basePath, HoodieTableType.MERGE_ON_READ); + client = new CompactionAdminClient(jsc, basePath); + } + + @Override + public void tearDown() throws IOException { + super.tearDown(); + } + + @Test + public void testUnscheduleCompactionPlan() throws Exception { + int numEntriesPerInstant = 10; + CompactionTestUtils + .setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant, + numEntriesPerInstant, numEntriesPerInstant); + // THere are delta-commits after compaction instant + validateUnSchedulePlan(client, + "000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant); + // THere are delta-commits after compaction instant + validateUnSchedulePlan(client, + "002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant); + // THere are no delta-commits after compaction instant + validateUnSchedulePlan(client, + "004", "005", numEntriesPerInstant, 0); + // THere are no delta-commits after compaction instant + validateUnSchedulePlan(client, + "006", "007", numEntriesPerInstant, 0); + } + + @Test + public void testUnscheduleCompactionFileId() throws Exception { + int numEntriesPerInstant = 10; + CompactionTestUtils + .setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant, + numEntriesPerInstant, numEntriesPerInstant); + Map instantsWithOp = + Arrays.asList("001", "003", "005", "007").stream().map(instant -> { + try { + return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant)); + } catch (IOException ioe) { + throw new HoodieException(ioe); + } + }).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream().map(op -> Pair.of( + instantWithPlan.getLeft(), CompactionOperation.convertFromAvroRecordInstance(op))).findFirst().get()) + .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + // THere are delta-commits after compaction instant + validateUnScheduleFileId(client, + "000", "001", instantsWithOp.get("001"), 2); + // THere are delta-commits after compaction instant + validateUnScheduleFileId(client, + "002", "003", instantsWithOp.get("003"), 2); + // THere are no delta-commits after compaction instant + validateUnScheduleFileId(client, + "004", "005", instantsWithOp.get("005"), 0); + // THere are no delta-commits after compaction instant + validateUnScheduleFileId(client, + "006", "007", instantsWithOp.get("007"), 0); + } + + @Test + public void testRepairCompactionPlan() throws Exception { + int numEntriesPerInstant = 10; + CompactionTestUtils + .setupAndValidateCompactionOperations(metaClient,false, numEntriesPerInstant, numEntriesPerInstant, + numEntriesPerInstant, numEntriesPerInstant); + // THere are delta-commits after compaction instant + validateRepair("000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant); + // THere are delta-commits after compaction instant + validateRepair("002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant); + // THere are no delta-commits after compaction instant + validateRepair("004", "005", numEntriesPerInstant, 0); + // THere are no delta-commits after compaction instant + validateRepair("006", "007", numEntriesPerInstant, 0); + } + + private void validateRepair(String ingestionInstant, String compactionInstant, int numEntriesPerInstant, + int expNumRepairs) throws Exception { + List> renameFiles = + validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRepairs, true); + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + List result = client.validateCompactionPlan(metaClient, compactionInstant, 1); + if (expNumRepairs > 0) { + Assert.assertTrue("Expect some failures in validation", result.stream().filter(r -> !r.isSuccess()).count() > 0); + } + // Now repair + List> undoFiles = result.stream().flatMap(r -> + client.getRenamingActionsToAlignWithCompactionOperation(metaClient, + compactionInstant, r.getOperation(), Optional.absent()).stream()) + .map(rn -> { + try { + client.renameLogFile(metaClient, rn.getKey(), rn.getValue()); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + return rn; + }).collect(Collectors.toList()); + Map renameFilesFromUndo = + undoFiles.stream().collect(Collectors.toMap(p -> p.getRight().getPath().toString(), + x -> x.getLeft().getPath().toString())); + Map expRenameFiles = + renameFiles.stream().collect(Collectors.toMap(p -> p.getLeft().getPath().toString(), + x -> x.getRight().getPath().toString())); + if (expNumRepairs > 0) { + Assert.assertFalse("Rename Files must be non-empty", renameFiles.isEmpty()); + } else { + Assert.assertTrue("Rename Files must be empty", renameFiles.isEmpty()); + } + expRenameFiles.entrySet().stream().forEach(r -> { + System.out.println("Key :" + r.getKey() + " renamed to " + r.getValue() + " rolled back to " + + renameFilesFromUndo.get(r.getKey())); + }); + + Assert.assertEquals("Undo must completely rollback renames", expRenameFiles, renameFilesFromUndo); + // Now expect validation to succeed + result = client.validateCompactionPlan(metaClient, compactionInstant, 1); + Assert.assertTrue("Expect no failures in validation", result.stream().filter(r -> !r.isSuccess()).count() == 0); + Assert.assertEquals("Expected Num Repairs", expNumRepairs, undoFiles.size()); + } + + /** + * Enssure compaction plan is valid + * @param compactionInstant Compaction Instant + * @throws Exception + */ + private void ensureValidCompactionPlan(String compactionInstant) throws Exception { + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + // Ensure compaction-plan is good to begin with + List validationResults = client.validateCompactionPlan(metaClient, + compactionInstant, 1); + Assert.assertFalse("Some validations failed", + validationResults.stream().filter(v -> !v.isSuccess()).findAny().isPresent()); + } + + private void validateRenameFiles(List> renameFiles, + String ingestionInstant, String compactionInstant, HoodieTableFileSystemView fsView) { + // Ensure new names of log-files are on expected lines + Set uniqNewLogFiles = new HashSet<>(); + Set uniqOldLogFiles = new HashSet<>(); + + renameFiles.stream().forEach(lfPair -> { + Assert.assertFalse("Old Log File Names do not collide", uniqOldLogFiles.contains(lfPair.getKey())); + Assert.assertFalse("New Log File Names do not collide", uniqNewLogFiles.contains(lfPair.getValue())); + uniqOldLogFiles.add(lfPair.getKey()); + uniqNewLogFiles.add(lfPair.getValue()); + }); + + renameFiles.stream().forEach(lfPair -> { + HoodieLogFile oldLogFile = lfPair.getLeft(); + HoodieLogFile newLogFile = lfPair.getValue(); + Assert.assertEquals("Base Commit time is expected", ingestionInstant, newLogFile.getBaseCommitTime()); + Assert.assertEquals("Base Commit time is expected", compactionInstant, oldLogFile.getBaseCommitTime()); + Assert.assertEquals("File Id is expected", oldLogFile.getFileId(), newLogFile.getFileId()); + HoodieLogFile lastLogFileBeforeCompaction = + fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], ingestionInstant) + .filter(fs -> fs.getFileId().equals(oldLogFile.getFileId())) + .map(fs -> fs.getLogFiles().findFirst().get()).findFirst().get(); + Assert.assertEquals("Log Version expected", + lastLogFileBeforeCompaction.getLogVersion() + oldLogFile.getLogVersion(), + newLogFile.getLogVersion()); + Assert.assertTrue("Log version does not collide", + newLogFile.getLogVersion() > lastLogFileBeforeCompaction.getLogVersion()); + }); + } + + /** + * Validate Unschedule operations + */ + private List> validateUnSchedulePlan(CompactionAdminClient client, + String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames) + throws Exception { + return validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, + expNumRenames, false); + } + + /** + * Validate Unschedule operations + */ + private List> validateUnSchedulePlan(CompactionAdminClient client, + String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames, + boolean skipUnSchedule) throws Exception { + + ensureValidCompactionPlan(compactionInstant); + + // Check suggested rename operations + List> renameFiles = + client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1, + Optional.absent(), false); + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + + // Log files belonging to file-slices created because of compaction request must be renamed + + Set gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet()); + final HoodieTableFileSystemView fsView = + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + Set expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]) + .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)) + .flatMap(fs -> fs.getLogFiles()) + .collect(Collectors.toSet()); + Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed", + expLogFilesToBeRenamed, gotLogFilesToBeRenamed); + + if (skipUnSchedule) { + // Do the renaming only but do not touch the compaction plan - Needed for repair tests + renameFiles.stream().forEach(lfPair -> { + try { + client.renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight()); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + } else { + validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView); + } + + Map fileIdToCountsBeforeRenaming = + fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant) + .filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant)) + .map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count())) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + // Call the main unschedule API + + client.unscheduleCompactionPlan(compactionInstant, false, 1, false); + + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + final HoodieTableFileSystemView newFsView = + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + // Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files + newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant) + .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).forEach(fs -> { + Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent()); + Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0); + }); + + // Ensure same number of log-files before and after renaming per fileId + Map fileIdToCountsAfterRenaming = + newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices()) + .filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant)) + .map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count())) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + Assert.assertEquals("Each File Id has same number of log-files", + fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming); + Assert.assertEquals("Not Empty", numEntriesPerInstant, fileIdToCountsAfterRenaming.size()); + Assert.assertEquals("Expected number of renames", expNumRenames, renameFiles.size()); + return renameFiles; + } + + /** + * Validate Unschedule operations + */ + private void validateUnScheduleFileId(CompactionAdminClient client, String ingestionInstant, + String compactionInstant, CompactionOperation op, int expNumRenames) throws Exception { + + ensureValidCompactionPlan(compactionInstant); + + // Check suggested rename operations + List> renameFiles = + client.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op, + Optional.absent(), false); + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + + // Log files belonging to file-slices created because of compaction request must be renamed + + Set gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet()); + final HoodieTableFileSystemView fsView = + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + Set expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]) + .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)) + .filter(fs -> fs.getFileId().equals(op.getFileId())) + .flatMap(fs -> fs.getLogFiles()) + .collect(Collectors.toSet()); + Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed", + expLogFilesToBeRenamed, gotLogFilesToBeRenamed); + validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView); + + Map fileIdToCountsBeforeRenaming = + fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant) + .filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant)) + .filter(fs -> fs.getFileId().equals(op.getFileId())) + .map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count())) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + // Call the main unschedule API + client.unscheduleCompactionFileId(op.getFileId(), false, false); + + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); + final HoodieTableFileSystemView newFsView = + new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); + // Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files + newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant) + .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)) + .filter(fs -> fs.getFileId().equals(op.getFileId())).forEach(fs -> { + Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent()); + Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0); + }); + + // Ensure same number of log-files before and after renaming per fileId + Map fileIdToCountsAfterRenaming = + newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices()) + .filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant)) + .filter(fs -> fs.getFileId().equals(op.getFileId())) + .map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count())) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + Assert.assertEquals("Each File Id has same number of log-files", + fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming); + Assert.assertEquals("Not Empty", 1, fileIdToCountsAfterRenaming.size()); + Assert.assertEquals("Expected number of renames", expNumRenames, renameFiles.size()); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java index 6ce2aa8d395ab..dec7244db5c9e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -33,12 +33,16 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** * Helper class to generate compaction plan from FileGroup/FileSlice abstraction */ public class CompactionUtils { + private static final Logger LOG = LogManager.getLogger(CompactionUtils.class); + /** * Generate compaction operation from file-slice * @@ -47,7 +51,7 @@ public class CompactionUtils { * @param metricsCaptureFunction Metrics Capture function * @return Compaction Operation */ - public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice, + public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice, Optional, Map>> metricsCaptureFunction) { HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder(); builder.setPartitionPath(partitionPath); @@ -114,16 +118,21 @@ public static List> getAllPendingCompa metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); return pendingCompactionInstants.stream().map(instant -> { try { - HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( - metaClient.getActiveTimeline().getInstantAuxiliaryDetails( - HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); - return Pair.of(instant, compactionPlan); + return Pair.of(instant, getCompactionPlan(metaClient, instant.getTimestamp())); } catch (IOException e) { throw new HoodieException(e); } }).collect(Collectors.toList()); } + public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, + String compactionInstant) throws IOException { + HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( + metaClient.getActiveTimeline().getInstantAuxiliaryDetails( + HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get()); + return compactionPlan; + } + /** * Get all file-ids with pending Compaction operations and their target compaction instant time * diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java new file mode 100644 index 0000000000000..5fae7a3b3a7eb --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java @@ -0,0 +1,207 @@ +/* + * 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; + +import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS; +import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; +import static com.uber.hoodie.common.table.HoodieTimeline.DELTA_COMMIT_ACTION; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.uber.hoodie.avro.model.HoodieCompactionOperation; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.collection.Pair; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; + +public class CompactionTestUtils { + + public static Map> setupAndValidateCompactionOperations( + HoodieTableMetaClient metaClient, boolean inflight, + int numEntriesInPlan1, int numEntriesInPlan2, + int numEntriesInPlan3, int numEntriesInPlan4) throws IOException { + HoodieCompactionPlan plan1 = createCompactionPlan(metaClient, "000", "001", numEntriesInPlan1, true, true); + HoodieCompactionPlan plan2 = createCompactionPlan(metaClient, "002", "003", numEntriesInPlan2, false, true); + HoodieCompactionPlan plan3 = createCompactionPlan(metaClient, "004", "005", numEntriesInPlan3, true, false); + HoodieCompactionPlan plan4 = createCompactionPlan(metaClient, "006", "007", numEntriesInPlan4, false, false); + + if (inflight) { + scheduleInflightCompaction(metaClient, "001", plan1); + scheduleInflightCompaction(metaClient, "003", plan2); + scheduleInflightCompaction(metaClient, "005", plan3); + scheduleInflightCompaction(metaClient, "007", plan4); + } else { + scheduleCompaction(metaClient, "001", plan1); + scheduleCompaction(metaClient, "003", plan2); + scheduleCompaction(metaClient, "005", plan3); + scheduleCompaction(metaClient, "007", plan4); + } + + createDeltaCommit(metaClient, "000"); + createDeltaCommit(metaClient, "002"); + createDeltaCommit(metaClient, "004"); + createDeltaCommit(metaClient, "006"); + + Map baseInstantsToCompaction = + new ImmutableMap.Builder().put("000", "001").put("002", "003") + .put("004", "005").put("006", "007").build(); + List expectedNumEntries = + Arrays.asList(numEntriesInPlan1, numEntriesInPlan2, numEntriesInPlan3, numEntriesInPlan4); + List plans = new ImmutableList.Builder() + .add(plan1, plan2, plan3, plan4).build(); + IntStream.range(0, 4).boxed().forEach(idx -> { + if (expectedNumEntries.get(idx) > 0) { + Assert.assertEquals("check if plan " + idx + " has exp entries", + expectedNumEntries.get(idx).longValue(), plans.get(idx).getOperations().size()); + } else { + Assert.assertNull("Plan " + idx + " has null ops", plans.get(idx).getOperations()); + } + }); + + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath(), true); + Map> pendingCompactionMap = + CompactionUtils.getAllPendingCompactionOperations(metaClient); + + Map> expPendingCompactionMap = + generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4), baseInstantsToCompaction); + + // Ensure Compaction operations are fine. + Assert.assertEquals(expPendingCompactionMap, pendingCompactionMap); + return expPendingCompactionMap; + } + + public static Map> generateExpectedCompactionOperations( + List plans, Map baseInstantsToCompaction) { + return plans.stream() + .flatMap(plan -> { + if (plan.getOperations() != null) { + return plan.getOperations().stream().map(op -> Pair.of(op.getFileId(), + Pair.of(baseInstantsToCompaction.get(op.getBaseInstantTime()), op))); + } + return Stream.empty(); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } + + public static void scheduleCompaction(HoodieTableMetaClient metaClient, + String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { + metaClient.getActiveTimeline().saveToCompactionRequested( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), + AvroUtils.serializeCompactionPlan(compactionPlan)); + } + + public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException { + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Optional.empty()); + } + + public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime, + HoodieCompactionPlan compactionPlan) throws IOException { + scheduleCompaction(metaClient, instantTime, compactionPlan); + metaClient.getActiveTimeline().transitionCompactionRequestedToInflight( + new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime)); + } + + public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient metaClient, String instantId, + String compactionInstantId, int numFileIds, boolean createDataFile, + boolean deltaCommitsAfterCompactionRequests) { + List ops = IntStream.range(0, numFileIds).boxed().map(idx -> { + try { + String fileId = UUID.randomUUID().toString(); + if (createDataFile) { + HoodieTestUtils.createDataFile(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId); + } + HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], + instantId, fileId, Optional.of(1)); + HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], + instantId, fileId, Optional.of(2)); + FileSlice slice = new FileSlice(instantId, fileId); + if (createDataFile) { + slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + + "/" + FSUtils.makeDataFileName(instantId, 1, fileId))); + } + String logFilePath1 = HoodieTestUtils + .getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId, + Optional.of(1)); + String logFilePath2 = HoodieTestUtils + .getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId, + Optional.of(2)); + slice.addLogFile(new HoodieLogFile(new Path(logFilePath1))); + slice.addLogFile(new HoodieLogFile(new Path(logFilePath2))); + HoodieCompactionOperation op = + CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Optional.empty()); + if (deltaCommitsAfterCompactionRequests) { + HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], + compactionInstantId, fileId, Optional.of(1)); + HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], + compactionInstantId, fileId, Optional.of(2)); + } + return op; + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }).collect(Collectors.toList()); + return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>()); + } + + public static class TestHoodieDataFile extends HoodieDataFile { + + private final String path; + + public TestHoodieDataFile(String path) { + super(null); + this.path = path; + } + + @Override + public String getPath() { + return path; + } + + @Override + public String getFileId() { + return UUID.randomUUID().toString(); + } + + @Override + public String getCommitTime() { + return "100"; + } + + @Override + public long getFileSize() { + return 0; + } + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java index 7557b50878dd5..c65216260ec07 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java @@ -17,32 +17,29 @@ package com.uber.hoodie.common.util; import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS; -import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; +import static com.uber.hoodie.common.model.HoodieTestUtils.getDefaultHadoopConf; +import static com.uber.hoodie.common.util.CompactionTestUtils.createCompactionPlan; +import static com.uber.hoodie.common.util.CompactionTestUtils.scheduleCompaction; +import static com.uber.hoodie.common.util.CompactionTestUtils.setupAndValidateCompactionOperations; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.FileSlice; -import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; -import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.CompactionTestUtils.TestHoodieDataFile; import com.uber.hoodie.common.util.collection.Pair; -import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.Stream; import org.apache.hadoop.fs.Path; import org.junit.Assert; import org.junit.Before; @@ -64,7 +61,8 @@ public class TestCompactionUtils { @Before public void init() throws IOException { - metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath()); + metaClient = HoodieTestUtils.initTableType(getDefaultHadoopConf(), + tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); basePath = metaClient.getBasePath(); } @@ -156,12 +154,12 @@ public void testCompactionTransformation() { @Test(expected = IllegalStateException.class) public void testGetAllPendingCompactionOperationsWithDupFileId() throws IOException { // Case where there is duplicate fileIds in compaction requests - HoodieCompactionPlan plan1 = createCompactionPlan("000", 10); - HoodieCompactionPlan plan2 = createCompactionPlan("001", 10); - scheduleCompaction("000", plan1); - scheduleCompaction("001", plan2); + HoodieCompactionPlan plan1 = createCompactionPlan(metaClient, "000", "001", 10, true, true); + HoodieCompactionPlan plan2 = createCompactionPlan(metaClient, "002", "003", 0, false, false); + scheduleCompaction(metaClient, "001", plan1); + scheduleCompaction(metaClient, "003", plan2); // schedule same plan again so that there will be duplicates - scheduleCompaction("003", plan1); + scheduleCompaction(metaClient, "005", plan1); metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); Map> res = CompactionUtils.getAllPendingCompactionOperations(metaClient); @@ -170,114 +168,19 @@ public void testGetAllPendingCompactionOperationsWithDupFileId() throws IOExcept @Test public void testGetAllPendingCompactionOperations() throws IOException { // Case where there are 4 compaction requests where 1 is empty. - testGetAllPendingCompactionOperations(false, 10, 10, 10, 0); + setupAndValidateCompactionOperations(metaClient, false, 10, 10, 10, 0); } @Test public void testGetAllPendingInflightCompactionOperations() throws IOException { // Case where there are 4 compaction requests where 1 is empty. All of them are marked inflight - testGetAllPendingCompactionOperations(true, 10, 10, 10, 0); + setupAndValidateCompactionOperations(metaClient, true, 10, 10, 10, 0); } @Test public void testGetAllPendingCompactionOperationsForEmptyCompactions() throws IOException { // Case where there are 4 compaction requests and all are empty. - testGetAllPendingCompactionOperations(false, 0, 0, 0, 0); - } - - private void testGetAllPendingCompactionOperations(boolean inflight, int numEntriesInPlan1, int numEntriesInPlan2, - int numEntriesInPlan3, int numEntriesInPlan4) throws IOException { - HoodieCompactionPlan plan1 = createCompactionPlan("000", numEntriesInPlan1); - HoodieCompactionPlan plan2 = createCompactionPlan("001", numEntriesInPlan2); - HoodieCompactionPlan plan3 = createCompactionPlan("002", numEntriesInPlan3); - HoodieCompactionPlan plan4 = createCompactionPlan("003", numEntriesInPlan4); - - if (inflight) { - scheduleInflightCompaction("000", plan1); - scheduleInflightCompaction("001", plan2); - scheduleInflightCompaction("002", plan3); - scheduleInflightCompaction("003", plan4); - } else { - scheduleCompaction("000", plan1); - scheduleCompaction("001", plan2); - scheduleCompaction("002", plan3); - scheduleCompaction("003", plan4); - } - - List expectedNumEntries = - Arrays.asList(numEntriesInPlan1, numEntriesInPlan2, numEntriesInPlan3, numEntriesInPlan4); - List plans = new ImmutableList.Builder() - .add(plan1, plan2, plan3, plan4).build(); - IntStream.range(0, 4).boxed().forEach(idx -> { - if (expectedNumEntries.get(idx) > 0) { - Assert.assertEquals("check if plan " + idx + " has exp entries", - expectedNumEntries.get(idx).longValue(), plans.get(idx).getOperations().size()); - } else { - Assert.assertNull("Plan " + idx + " has null ops", plans.get(idx).getOperations()); - } - }); - - metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); - Map> pendingCompactionMap = - CompactionUtils.getAllPendingCompactionOperations(metaClient); - - Map> expPendingCompactionMap = - generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4)); - - // Ensure all the - Assert.assertEquals(expPendingCompactionMap, pendingCompactionMap); - } - - private Map> generateExpectedCompactionOperations( - List plans) { - return plans.stream() - .flatMap(plan -> { - if (plan.getOperations() != null) { - return plan.getOperations().stream().map(op -> Pair.of(op.getFileId(), - Pair.of(op.getBaseInstantTime(), op))); - } - return Stream.empty(); - }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - } - - private void scheduleCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { - metaClient.getActiveTimeline().saveToCompactionRequested( - new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), - AvroUtils.serializeCompactionPlan(compactionPlan)); - } - - private void scheduleInflightCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { - metaClient.getActiveTimeline().saveToCompactionRequested( - new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), - AvroUtils.serializeCompactionPlan(compactionPlan)); - metaClient.getActiveTimeline().transitionCompactionRequestedToInflight( - new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime)); - } - - private HoodieCompactionPlan createCompactionPlan(String instantId, int numFileIds) { - List ops = IntStream.range(0, numFileIds).boxed().map(idx -> { - try { - String fileId = - HoodieTestUtils.createNewDataFile(basePath, DEFAULT_PARTITION_PATHS[0], instantId); - HoodieTestUtils.createNewLogFile(metaClient.getFs(), basePath, DEFAULT_PARTITION_PATHS[0], - instantId, fileId, Optional.of(1)); - HoodieTestUtils.createNewLogFile(metaClient.getFs(), basePath, DEFAULT_PARTITION_PATHS[0], - instantId, fileId, Optional.of(2)); - FileSlice slice = new FileSlice(instantId, fileId); - slice.setDataFile(new TestHoodieDataFile(HoodieTestUtils.createDataFile(basePath, DEFAULT_PARTITION_PATHS[0], - instantId, fileId))); - String logFilePath1 = HoodieTestUtils.getLogFilePath(basePath, DEFAULT_PARTITION_PATHS[0], instantId, fileId, - Optional.of(1)); - String logFilePath2 = HoodieTestUtils.getLogFilePath(basePath, DEFAULT_PARTITION_PATHS[0], instantId, fileId, - Optional.of(2)); - slice.addLogFile(new HoodieLogFile(new Path(logFilePath1))); - slice.addLogFile(new HoodieLogFile(new Path(logFilePath2))); - return CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Optional.empty()); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } - }).collect(Collectors.toList()); - return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>()); + setupAndValidateCompactionOperations(metaClient, false, 0, 0, 0, 0); } /** @@ -315,35 +218,4 @@ private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompaction }); Assert.assertEquals("Metrics set", metrics, op.getMetrics()); } - - - private static class TestHoodieDataFile extends HoodieDataFile { - - private final String path; - - public TestHoodieDataFile(String path) { - super(null); - this.path = path; - } - - @Override - public String getPath() { - return path; - } - - @Override - public String getFileId() { - return UUID.randomUUID().toString(); - } - - @Override - public String getCommitTime() { - return "100"; - } - - @Override - public long getFileSize() { - return 0; - } - } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java new file mode 100644 index 0000000000000..0a225e91ae68b --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java @@ -0,0 +1,154 @@ +package com.uber.hoodie.utilities; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.uber.hoodie.CompactionAdminClient; +import com.uber.hoodie.CompactionAdminClient.RenameOpResult; +import com.uber.hoodie.CompactionAdminClient.ValidationOpResult; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.util.FSUtils; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.List; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaSparkContext; + +public class HoodieCompactionAdminTool { + + private final Config cfg; + + public HoodieCompactionAdminTool(Config cfg) { + this.cfg = cfg; + } + + /** + * + * @param args + * @throws Exception + */ + 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); + } + HoodieCompactionAdminTool admin = new HoodieCompactionAdminTool(cfg); + admin.run(UtilHelpers.buildSparkContext("admin-compactor", cfg.sparkMaster, cfg.sparkMemory)); + } + + /** + * Executes one of compaction admin operations + */ + public void run(JavaSparkContext jsc) throws Exception { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath); + CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath); + final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); + if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) { + throw new IllegalStateException("Output File Path already exists"); + } + switch (cfg.operation) { + case VALIDATE: + List res = + admin.validateCompactionPlan(metaClient, cfg.compactionInstantTime, cfg.parallelism); + if (cfg.printOutput) { + printOperationResult("Result of Validation Operation :", res); + } + serializeOperationResult(fs, res); + break; + case UNSCHEDULE_FILE: + List r = + admin.unscheduleCompactionFileId(cfg.fileId, cfg.skipValidation, cfg.dryRun); + if (cfg.printOutput) { + System.out.println(r); + } + serializeOperationResult(fs, r); + break; + case UNSCHEDULE_PLAN: + List r2 = + admin.unscheduleCompactionPlan(cfg.compactionInstantTime, cfg.skipValidation, cfg.parallelism, cfg.dryRun); + if (cfg.printOutput) { + printOperationResult("Result of Unscheduling Compaction Plan :", r2); + } + serializeOperationResult(fs, r2); + break; + case REPAIR: + List r3 = + admin.repairCompaction(cfg.compactionInstantTime, cfg.parallelism, cfg.dryRun); + if (cfg.printOutput) { + printOperationResult("Result of Repair Operation :", r3); + } + serializeOperationResult(fs, r3); + break; + default: + throw new IllegalStateException("Not yet implemented !!"); + } + } + + private void serializeOperationResult(FileSystem fs, T result) throws Exception { + if ((cfg.outputPath != null) && (result != null)) { + Path outputPath = new Path(cfg.outputPath); + FSDataOutputStream fsout = fs.create(outputPath, true); + ObjectOutputStream out = new ObjectOutputStream(fsout); + out.writeObject(result); + out.close(); + fsout.close(); + } + } + + /** + * Print Operation Result + * + * @param initialLine Initial Line + * @param result Result + */ + private void printOperationResult(String initialLine, List result) { + System.out.println(initialLine); + for (T r : result) { + System.out.print(r); + } + } + + /** + * Operation Types + */ + public enum Operation { + VALIDATE, + UNSCHEDULE_PLAN, + UNSCHEDULE_FILE, + REPAIR + } + + /** + * Admin Configuration Options + */ + public static class Config implements Serializable { + + @Parameter(names = {"--operation", "-op"}, description = "Operation", required = true) + public Operation operation = Operation.VALIDATE; + @Parameter(names = {"--base-path", "-bp"}, description = "Base path for the dataset", required = true) + public String basePath = null; + @Parameter(names = {"--instant-time", "-in"}, description = "Compaction Instant time", required = false) + public String compactionInstantTime = null; + @Parameter(names = {"--file-id", "-id"}, description = "File Id", required = false) + public String fileId = null; + @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false) + public int parallelism = 3; + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = true) + public String sparkMaster = null; + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) + public String sparkMemory = null; + @Parameter(names = {"--dry-run", "-dr"}, description = "Dry Run Mode", required = false) + public boolean dryRun = false; + @Parameter(names = {"--skip-validation", "-sv"}, description = "Skip Validation", required = false) + public boolean skipValidation = false; + @Parameter(names = {"--output-path", "-ot"}, description = "Output Path", required = false) + public String outputPath = null; + @Parameter(names = {"--print-output", "-pt"}, description = "Print Output", required = false) + public boolean printOutput = true; + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + } +} From 42cc873926c7bbeadea4fb4ef75415fa7f3f3a77 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 2 Oct 2018 10:12:20 -0700 Subject: [PATCH 124/374] Useful Hudi CLI commands to debug/analyze production workloads --- .../cli/commands/ArchivedCommitsCommand.java | 34 ++- .../cli/commands/FileSystemViewCommand.java | 273 ++++++++++++++++++ .../hoodie/cli/commands/RollbacksCommand.java | 130 +++++++++ .../table/view/HoodieTableFileSystemView.java | 4 + 4 files changed, 432 insertions(+), 9 deletions(-) create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RollbacksCommand.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 51545d842dbaa..5f4d5de0d6a1d 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -50,6 +50,8 @@ public boolean isShowArchivedCommitAvailable() { @CliCommand(value = "show archived commits", help = "Read commits from archived files and show details") public String showCommits( + @CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata", unspecifiedDefaultValue = "true") + boolean skipMetadata, @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit, @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField, @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending, @@ -74,51 +76,65 @@ public String showCommits( List records = blk.getRecords(); readRecords.addAll(records); } - List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r)) + List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> + readCommit(r, skipMetadata)) .collect(Collectors.toList()); allCommits.addAll(readCommits); reader.close(); } TableHeader header = new TableHeader().addTableHeaderField("CommitTime") - .addTableHeaderField("CommitType") - .addTableHeaderField("CommitDetails"); + .addTableHeaderField("CommitType"); + + if (!skipMetadata) { + header = header.addTableHeaderField("CommitDetails"); + } return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allCommits); } - private Comparable[] readCommit(GenericRecord record) { + private Comparable[] readCommit(GenericRecord record, boolean skipMetadata) { List commitDetails = new ArrayList<>(); try { switch (record.get("actionType").toString()) { case HoodieTimeline.CLEAN_ACTION: { commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); - commitDetails.add(record.get("hoodieCleanMetadata").toString()); + if (!skipMetadata) { + commitDetails.add(record.get("hoodieCleanMetadata").toString()); + } break; } case HoodieTimeline.COMMIT_ACTION: { commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); - commitDetails.add(record.get("hoodieCommitMetadata").toString()); + if (!skipMetadata) { + commitDetails.add(record.get("hoodieCommitMetadata").toString()); + } break; } case HoodieTimeline.DELTA_COMMIT_ACTION: { commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); - commitDetails.add(record.get("hoodieCommitMetadata").toString()); + if (!skipMetadata) { + commitDetails.add(record.get("hoodieCommitMetadata").toString()); + } break; } case HoodieTimeline.ROLLBACK_ACTION: { commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); - commitDetails.add(record.get("hoodieRollbackMetadata").toString()); + if (!skipMetadata) { + commitDetails.add(record.get("hoodieRollbackMetadata").toString()); + } break; } case HoodieTimeline.SAVEPOINT_ACTION: { commitDetails.add(record.get("commitTime")); commitDetails.add(record.get("actionType").toString()); - commitDetails.add(record.get("hoodieSavePointMetadata").toString()); + if (!skipMetadata) { + commitDetails.add(record.get("hoodieSavePointMetadata").toString()); + } break; } default: diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java new file mode 100644 index 0000000000000..dd42c47e203fe --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java @@ -0,0 +1,273 @@ +/* + * 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.cli.commands; + +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.NumericUtils; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiPredicate; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +@Component +public class FileSystemViewCommand implements CommandMarker { + + @CliCommand(value = "show fsview all", help = "Show entire file-system view") + public String showAllFileSlices( + @CliOption(key = {"pathRegex"}, + help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") String globRegex, + @CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view", + unspecifiedDefaultValue = "false") boolean readOptimizedOnly, + @CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed", + unspecifiedDefaultValue = "") String maxInstant, + @CliOption(key = { + "includeMax"}, help = "Include Max Instant", unspecifiedDefaultValue = "false") boolean includeMaxInstant, + @CliOption(key = { + "includeInflight"}, help = "Include Inflight Instants", unspecifiedDefaultValue = "false") + boolean includeInflight, + @CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants", unspecifiedDefaultValue = "false") + boolean excludeCompaction, + @CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") 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 { + + HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, readOptimizedOnly, includeMaxInstant, + includeInflight, excludeCompaction); + List rows = new ArrayList<>(); + fsView.getAllFileGroups().forEach(fg -> fg.getAllFileSlices().forEach(fs -> { + int idx = 0; + // For ReadOptimized Views, do not display any delta-file related columns + Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 8]; + row[idx++] = fg.getPartitionPath(); + row[idx++] = fg.getId(); + row[idx++] = fs.getBaseInstantTime(); + row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : ""; + row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1; + if (!readOptimizedOnly) { + row[idx++] = fs.getLogFiles().count(); + row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .mapToLong(lf -> lf.getFileSize().get()).sum(); + row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .collect(Collectors.toList()).toString(); + } + rows.add(row); + })); + Function converterFunction = + entry -> NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString()))); + Map> fieldNameToConverterMap = new HashMap<>(); + fieldNameToConverterMap.put("Total Delta File Size", converterFunction); + fieldNameToConverterMap.put("Data-File Size", converterFunction); + + TableHeader header = new TableHeader() + .addTableHeaderField("Partition") + .addTableHeaderField("FileId") + .addTableHeaderField("Base-Instant") + .addTableHeaderField("Data-File") + .addTableHeaderField("Data-File Size"); + if (!readOptimizedOnly) { + header = header.addTableHeaderField("Num Delta Files") + .addTableHeaderField("Total Delta File Size") + .addTableHeaderField("Delta Files"); + } + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); + } + + @CliCommand(value = "show fsview latest", help = "Show latest file-system view") + public String showLatestFileSlices( + @CliOption(key = {"partitionPath"}, + help = "A valid paritition path", mandatory = true) String partition, + @CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view", + unspecifiedDefaultValue = "false") boolean readOptimizedOnly, + @CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed", + unspecifiedDefaultValue = "") String maxInstant, + @CliOption(key = {"merge"}, help = "Merge File Slices due to pending compaction", + unspecifiedDefaultValue = "true") final boolean merge, + @CliOption(key = {"includeMax"}, help = "Include Max Instant", unspecifiedDefaultValue = "false") + boolean includeMaxInstant, + @CliOption(key = {"includeInflight"}, help = "Include Inflight Instants", unspecifiedDefaultValue = "false") + boolean includeInflight, + @CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants", unspecifiedDefaultValue = "false") + boolean excludeCompaction, + @CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") 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 { + + HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, readOptimizedOnly, includeMaxInstant, + includeInflight, excludeCompaction); + List rows = new ArrayList<>(); + + final Stream fileSliceStream; + if (!merge) { + fileSliceStream = fsView.getLatestFileSlices(partition); + } else { + if (maxInstant.isEmpty()) { + maxInstant = HoodieCLI.tableMetadata.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant() + .get().getTimestamp(); + } + fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(partition, maxInstant); + } + + fileSliceStream.forEach(fs -> { + int idx = 0; + Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 13]; + row[idx++] = partition; + row[idx++] = fs.getFileId(); + row[idx++] = fs.getBaseInstantTime(); + row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : ""; + + long dataFileSize = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1; + row[idx++] = dataFileSize; + + if (!readOptimizedOnly) { + row[idx++] = fs.getLogFiles().count(); + row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .mapToLong(lf -> lf.getFileSize().get()).sum(); + long logFilesScheduledForCompactionTotalSize = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime())) + .mapToLong(lf -> lf.getFileSize().get()).sum(); + row[idx++] = logFilesScheduledForCompactionTotalSize; + + long logFilesUnscheduledTotalSize = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime())) + .mapToLong(lf -> lf.getFileSize().get()).sum(); + row[idx++] = logFilesUnscheduledTotalSize; + + double logSelectedForCompactionToBaseRatio = + dataFileSize > 0 ? logFilesScheduledForCompactionTotalSize / (dataFileSize * 1.0) : -1; + row[idx++] = logSelectedForCompactionToBaseRatio; + double logUnscheduledToBaseRatio = + dataFileSize > 0 ? logFilesUnscheduledTotalSize / (dataFileSize * 1.0) : -1; + row[idx++] = logUnscheduledToBaseRatio; + + row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime())) + .collect(Collectors.toList()).toString(); + row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent()) + .filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime())) + .collect(Collectors.toList()).toString(); + } + rows.add(row); + }); + + Function converterFunction = + entry -> NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString()))); + Map> fieldNameToConverterMap = new HashMap<>(); + fieldNameToConverterMap.put("Data-File Size", converterFunction); + if (!readOptimizedOnly) { + fieldNameToConverterMap.put("Total Delta Size", converterFunction); + fieldNameToConverterMap.put("Delta Size - compaction scheduled", converterFunction); + fieldNameToConverterMap.put("Delta Size - compaction unscheduled", converterFunction); + } + + TableHeader header = new TableHeader() + .addTableHeaderField("Partition") + .addTableHeaderField("FileId") + .addTableHeaderField("Base-Instant") + .addTableHeaderField("Data-File") + .addTableHeaderField("Data-File Size"); + + if (!readOptimizedOnly) { + header = header.addTableHeaderField("Num Delta Files") + .addTableHeaderField("Total Delta Size") + .addTableHeaderField("Delta Size - compaction scheduled") + .addTableHeaderField("Delta Size - compaction unscheduled") + .addTableHeaderField("Delta To Base Ratio - compaction scheduled") + .addTableHeaderField("Delta To Base Ratio - compaction unscheduled") + .addTableHeaderField("Delta Files - compaction scheduled") + .addTableHeaderField("Delta Files - compaction unscheduled"); + } + return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows); + } + + /** + * Build File System View + * @param globRegex Path Regex + * @param maxInstant Max Instants to be used for displaying file-instants + * @param readOptimizedOnly Include only read optimized view + * @param includeMaxInstant Include Max instant + * @param includeInflight Include inflight instants + * @param excludeCompaction Exclude Compaction instants + * @return + * @throws IOException + */ + private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean readOptimizedOnly, + boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieCLI.tableMetadata.getHadoopConf(), + HoodieCLI.tableMetadata.getBasePath(), true); + FileSystem fs = HoodieCLI.fs; + String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex); + FileStatus[] statuses = fs.globStatus(new Path(globPath)); + Stream instantsStream = null; + + HoodieTimeline timeline = null; + if (readOptimizedOnly) { + timeline = metaClient.getActiveTimeline().getCommitTimeline(); + } else if (excludeCompaction) { + timeline = metaClient.getActiveTimeline().getCommitsTimeline(); + } else { + timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); + } + + if (!includeInflight) { + timeline = timeline.filterCompletedInstants(); + } + + instantsStream = timeline.getInstants(); + + if (!maxInstant.isEmpty()) { + final BiPredicate predicate; + if (includeMaxInstant) { + predicate = HoodieTimeline.GREATER_OR_EQUAL; + } else { + predicate = HoodieTimeline.GREATER; + } + instantsStream = instantsStream.filter(is -> predicate.test(maxInstant, is.getTimestamp())); + } + + HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream, + (Function> & Serializable) metaClient.getActiveTimeline()::getInstantDetails); + return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses); + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RollbacksCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RollbacksCommand.java new file mode 100644 index 0000000000000..2aed5b50b1498 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RollbacksCommand.java @@ -0,0 +1,130 @@ +/* + * 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.cli.commands; + +import static com.uber.hoodie.common.table.HoodieTimeline.ROLLBACK_ACTION; + +import com.google.common.collect.ImmutableSet; +import com.uber.hoodie.avro.model.HoodieRollbackMetadata; +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.timeline.HoodieInstant.State; +import com.uber.hoodie.common.util.AvroUtils; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.stream.Stream; +import org.apache.commons.lang3.tuple.Pair; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +@Component +public class RollbacksCommand implements CommandMarker { + + @CliCommand(value = "show rollbacks", help = "List all rollback instants") + public String showRollbacks( + @CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") 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 { + HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.tableMetadata); + HoodieTimeline rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants(); + + final List rows = new ArrayList<>(); + rollback.getInstants().forEach(instant -> { + try { + HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata( + activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class); + metadata.getCommitsRollback().forEach(c -> { + Comparable[] row = new Comparable[5]; + row[0] = metadata.getStartRollbackTime(); + row[1] = c; + row[2] = metadata.getTotalFilesDeleted(); + row[3] = metadata.getTimeTakenInMillis(); + row[4] = metadata.getPartitionMetadata() != null ? metadata.getPartitionMetadata().size() : 0; + rows.add(row); + }); + } catch (IOException e) { + e.printStackTrace(); + } + }); + TableHeader header = new TableHeader() + .addTableHeaderField("Instant") + .addTableHeaderField("Rolledback Instant") + .addTableHeaderField("Total Files Deleted") + .addTableHeaderField("Time taken in millis") + .addTableHeaderField("Total Partitions"); + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); + } + + @CliCommand(value = "show rollback", help = "Show details of a rollback instant") + public String showRollback( + @CliOption(key = {"instant"}, help = "Rollback instant", mandatory = true) String rollbackInstant, + @CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") 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 { + HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.tableMetadata); + final List rows = new ArrayList<>(); + HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata( + activeTimeline.getInstantDetails(new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)) + .get(), HoodieRollbackMetadata.class); + metadata.getPartitionMetadata().entrySet().forEach(e -> { + Stream.concat(e.getValue().getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)), + e.getValue().getFailedDeleteFiles().stream().map(f -> Pair.of(f, false))) + .forEach(fileWithDeleteStatus -> { + Comparable[] row = new Comparable[5]; + row[0] = metadata.getStartRollbackTime(); + row[1] = metadata.getCommitsRollback().toString(); + row[2] = e.getKey(); + row[3] = fileWithDeleteStatus.getLeft(); + row[4] = fileWithDeleteStatus.getRight(); + rows.add(row); + }); + }); + + TableHeader header = new TableHeader() + .addTableHeaderField("Instant") + .addTableHeaderField("Rolledback Instants") + .addTableHeaderField("Partition") + .addTableHeaderField("Deleted File") + .addTableHeaderField("Succeeded"); + return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows); + } + + /** + * An Active timeline containing only rollbacks + */ + class RollbackTimeline extends HoodieActiveTimeline { + + public RollbackTimeline(HoodieTableMetaClient metaClient) { + super(metaClient, ImmutableSet.builder().add(ROLLBACK_EXTENSION).build()); + } + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index d33667e319ff6..7e7792d3dd1f1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -440,4 +440,8 @@ public Stream getAllFileGroups(String partitionPathStr) { public Map> getFileIdToPendingCompaction() { return fileIdToPendingCompaction; } + + public Stream getAllFileGroups() { + return fileGroupMap.values().stream(); + } } From 77d3021415f446eb8431f794054bea5334a3d8bd Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 7 Nov 2018 17:14:53 -0800 Subject: [PATCH 125/374] Enabling auto tuning of insert splits by default --- docs/configurations.md | 2 +- .../java/com/uber/hoodie/config/HoodieCompactionConfig.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configurations.md b/docs/configurations.md index ec3caf9e92fb5..8c3ebe9d1d893 100644 --- a/docs/configurations.md +++ b/docs/configurations.md @@ -74,7 +74,7 @@ summary: "Here we list all possible configurations and what they mean" Small files can always happen because of the number of insert records in a paritition in a batch. Hoodie has an option to auto-resolve small files by masking inserts into this partition as updates to existing small files. The size here is the minimum file size considered as a "small file size". This should be less < maxFileSize and setting it to 0, turns off this feature. - [insertSplitSize](#insertSplitSize) (size = 500000)
    Insert Write Parallelism. Number of inserts grouped for a single partition. Writing out 100MB files, with atleast 1kb records, means 100K records per file. Default is to overprovision to 500K. To improve insert latency, tune this to match the number of records in a single file. Setting this to a low number, will result in small files (particularly when compactionSmallFileSize is 0) - - [autoTuneInsertSplits](#autoTuneInsertSplits) (false)
    + - [autoTuneInsertSplits](#autoTuneInsertSplits) (true)
    Should hoodie dynamically compute the insertSplitSize based on the last 24 commit's metadata. Turned off by default. - [approxRecordSize](#approxRecordSize) ()
    The average record size. If specified, hoodie will use this and not compute dynamically based on the last 24 commit's metadata. No value set as default. This is critical in computing the insert parallelism and bin-packing inserts into small files. See above. diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index cd84505336fee..b1709955a93f9 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -63,7 +63,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert" + ".auto.split"; // its off by default - public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false); + public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(true); // This value is used as a guessimate for the record size, if we can't determine this from // previous commits public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = From c2010deb658fd04b3f9c2779b64bea43a2f5e289 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 13 Nov 2018 15:42:36 -0800 Subject: [PATCH 126/374] Fixing number of insert buckets to be generated by rounding off to the closest greater integer --- .../hoodie/table/HoodieCopyOnWriteTable.java | 2 +- .../hoodie/table/TestCopyOnWriteTable.java | 48 ++++++++++++------- 2 files changed, 33 insertions(+), 17 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 3fa8a175ac9ad..67411627b0a27 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -726,7 +726,7 @@ private void assignInserts(WorkloadProfile profile) { insertRecordsPerBucket = config.getParquetMaxFileSize() / averageRecordSize; } - int insertBuckets = (int) Math.max(totalUnassignedInserts / insertRecordsPerBucket, 1L); + int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket); logger.info( "After small file assignment: unassignedInserts => " + totalUnassignedInserts + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index b9be42b715549..2536c47eb9740 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -40,6 +40,7 @@ import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.HoodieCreateHandle; +import com.uber.hoodie.table.HoodieCopyOnWriteTable.UpsertPartitioner; import java.io.File; import java.util.ArrayList; import java.util.Arrays; @@ -401,9 +402,8 @@ public void testFileSizeUpsertRecords() throws Exception { } - private List testUpsertPartitioner(int smallFileSize, int numInserts, - int numUpdates, int fileSize, boolean autoSplitInserts) throws Exception { - final String testPartitionPath = "2016/09/26"; + private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, + int numUpdates, int fileSize, String testPartitionPath, boolean autoSplitInserts) throws Exception { HoodieWriteConfig config = makeHoodieClientConfigBuilder().withCompactionConfig( HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize).insertSplitSize(100) .autoTuneInsertSplits(autoSplitInserts).build()).withStorageConfig( @@ -427,41 +427,57 @@ private List testUpsertPartitioner(int smal WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records)); HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile); - - assertEquals("Should have 3 partitions", 3, partitioner.numPartitions()); - assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE, - partitioner.getBucketInfo(0).bucketType); - assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, - partitioner.getBucketInfo(1).bucketType); - assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, - partitioner.getBucketInfo(2).bucketType); assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition( new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation())))); - return partitioner.getInsertBuckets(testPartitionPath); + return partitioner; } @Test public void testUpsertPartitioner() throws Exception { + final String testPartitionPath = "2016/09/26"; // Inserts + Updates... Check all updates go together & inserts subsplit - List insertBuckets = testUpsertPartitioner(0, 200, 100, 1024, false); + UpsertPartitioner partitioner = getUpsertPartitioner(0, 200, 100, 1024, + testPartitionPath, false); + List insertBuckets = partitioner.getInsertBuckets(testPartitionPath); assertEquals("Total of 2 insert buckets", 2, insertBuckets.size()); } @Test public void testUpsertPartitionerWithSmallInsertHandling() throws Exception { + final String testPartitionPath = "2016/09/26"; // Inserts + Updates .. Check updates go together & inserts subsplit, after expanding // smallest file - List insertBuckets = testUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, + UpsertPartitioner partitioner = getUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024,testPartitionPath, false); + List insertBuckets = partitioner.getInsertBuckets(testPartitionPath); + + assertEquals("Should have 3 partitions", 3, partitioner.numPartitions()); + assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE, + partitioner.getBucketInfo(0).bucketType); + assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, + partitioner.getBucketInfo(1).bucketType); + assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, + partitioner.getBucketInfo(2).bucketType); assertEquals("Total of 3 insert buckets", 3, insertBuckets.size()); assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber); assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, 0.01); // Now with insert split size auto tuned - insertBuckets = testUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, true); - assertEquals("Total of 3 insert buckets", 3, insertBuckets.size()); + partitioner = getUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, testPartitionPath, true); + insertBuckets = partitioner.getInsertBuckets(testPartitionPath); + + assertEquals("Should have 4 partitions", 4, partitioner.numPartitions()); + assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE, + partitioner.getBucketInfo(0).bucketType); + assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, + partitioner.getBucketInfo(1).bucketType); + assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, + partitioner.getBucketInfo(2).bucketType); + assertEquals("Bucket 3 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, + partitioner.getBucketInfo(3).bucketType); + assertEquals("Total of 4 insert buckets", 4, insertBuckets.size()); assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber); assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, insertBuckets.get(0).weight, 0.01); } From 10f656b4a3fd527125fa42e31c9538ed0c362f16 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Tue, 27 Nov 2018 16:54:46 -0800 Subject: [PATCH 127/374] Explicitly handle lack of append() support during LogWriting --- .../io/storage/HoodieWrapperFileSystem.java | 19 +----- .../hoodie/common/storage/StorageSchemes.java | 66 +++++++++++++++++++ .../table/log/HoodieLogFormatWriter.java | 35 ++++++---- .../common/storage/TestStorageSchemes.java | 41 ++++++++++++ 4 files changed, 130 insertions(+), 31 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/storage/TestStorageSchemes.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java index 390e73abedfb8..57c32311b381f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java @@ -16,16 +16,15 @@ package com.uber.hoodie.io.storage; +import com.uber.hoodie.common.storage.StorageSchemes; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.util.EnumSet; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.conf.Configuration; @@ -60,20 +59,6 @@ public class HoodieWrapperFileSystem extends FileSystem { public static final String HOODIE_SCHEME_PREFIX = "hoodie-"; - private static final Set SUPPORT_SCHEMES; - - static { - SUPPORT_SCHEMES = new HashSet<>(); - SUPPORT_SCHEMES.add("file"); - SUPPORT_SCHEMES.add("hdfs"); - SUPPORT_SCHEMES.add("s3"); - SUPPORT_SCHEMES.add("s3a"); - - // Hoodie currently relies on underlying object store being fully - // consistent so only regional buckets should be used. - SUPPORT_SCHEMES.add("gs"); - SUPPORT_SCHEMES.add("viewfs"); - } private ConcurrentMap openStreams = new ConcurrentHashMap<>(); @@ -104,7 +89,7 @@ private static Path convertPathWithScheme(Path oldPath, String newScheme) { public static String getHoodieScheme(String scheme) { String newScheme; - if (SUPPORT_SCHEMES.contains(scheme)) { + if (StorageSchemes.isSchemeSupported(scheme)) { newScheme = HOODIE_SCHEME_PREFIX + scheme; } else { throw new IllegalArgumentException( diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java new file mode 100644 index 0000000000000..f42238ed6721a --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java @@ -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.common.storage; + +import java.util.Arrays; + +/** + * All the supported storage schemes in Hoodie. + */ +public enum StorageSchemes { + // Local filesystem + FILE("file", false), + // Hadoop File System + HDFS("hdfs", true), + // Apache Ignite FS + IGNITE("igfs", true), + // AWS S3 + S3A("s3a", false), + S3("s3", false), + // Google Cloud Storage + GCS("gs", false), + // View FS for federated setups. If federating across cloud stores, then append support is false + VIEWFS("viewfs", true); + + private String scheme; + private boolean supportsAppend; + + StorageSchemes(String scheme, boolean supportsAppend) { + this.scheme = scheme; + this.supportsAppend = supportsAppend; + } + + public String getScheme() { + return scheme; + } + + public boolean supportsAppend() { + return supportsAppend; + } + + public static boolean isSchemeSupported(String scheme) { + return Arrays.stream(values()).filter(s -> s.getScheme().equals(scheme)).count() > 0; + } + + public static boolean isAppendSupported(String scheme) { + if (!isSchemeSupported(scheme)) { + throw new IllegalArgumentException("Unsupported scheme :" + scheme); + } + return Arrays.stream(StorageSchemes.values()) + .filter(s -> s.supportsAppend() && s.scheme.equals(scheme)).count() > 0; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index c75e666bd6949..28b2501d00f2e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -17,6 +17,7 @@ package com.uber.hoodie.common.table.log; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.storage.StorageSchemes; import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; import com.uber.hoodie.common.table.log.HoodieLogFormat.WriterBuilder; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; @@ -68,26 +69,32 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { Path path = logFile.getPath(); if (fs.exists(path)) { - log.info(logFile + " exists. Appending to existing file"); - try { - this.output = fs.append(path, bufferSize); - } catch (RemoteException e) { - log.warn("Remote Exception, attempting to handle or recover lease", e); - handleAppendExceptionOrRecoverLease(path, e); - } catch (IOException ioe) { - if (ioe.getMessage().equalsIgnoreCase("Not supported")) { - log.info("Append not supported. Opening a new log file.."); - this.logFile = logFile.rollOver(fs); - createNewFile(); - } else { - throw ioe; + boolean isAppendSupported = StorageSchemes.isAppendSupported(fs.getScheme()); + if (isAppendSupported) { + log.info(logFile + " exists. Appending to existing file"); + try { + this.output = fs.append(path, bufferSize); + } catch (RemoteException e) { + log.warn("Remote Exception, attempting to handle or recover lease", e); + handleAppendExceptionOrRecoverLease(path, e); + } catch (IOException ioe) { + if (ioe.getMessage().toLowerCase().contains("not supported")) { + // may still happen if scheme is viewfs. + isAppendSupported = false; + } else { + throw ioe; + } } } + if (!isAppendSupported) { + this.logFile = logFile.rollOver(fs); + log.info("Append not supported.. Rolling over to " + logFile); + createNewFile(); + } } else { log.info(logFile + " does not exist. Create a new file"); // Block size does not matter as we will always manually autoflush createNewFile(); - // TODO - append a file level meta block } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/storage/TestStorageSchemes.java b/hoodie-common/src/test/java/com/uber/hoodie/common/storage/TestStorageSchemes.java new file mode 100644 index 0000000000000..0b8a04914275b --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/storage/TestStorageSchemes.java @@ -0,0 +1,41 @@ +/* + * 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.common.storage; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.junit.Test; + +public class TestStorageSchemes { + + @Test + public void testStorageSchemes() { + assertTrue(StorageSchemes.isSchemeSupported("hdfs")); + assertFalse(StorageSchemes.isSchemeSupported("s2")); + assertFalse(StorageSchemes.isAppendSupported("s3a")); + assertFalse(StorageSchemes.isAppendSupported("gs")); + assertTrue(StorageSchemes.isAppendSupported("viewfs")); + try { + StorageSchemes.isAppendSupported("s2"); + fail("Should throw exception for unsupported schemes"); + } catch (IllegalArgumentException ignore) { + // expected. + } + } +} From 10599062ad7160813b23b1f09215105c6ca4f9f9 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 27 Nov 2018 23:21:34 -0800 Subject: [PATCH 128/374] Avoid WriteStatus collect() call when committing batch --- .../com/uber/hoodie/HoodieWriteClient.java | 21 +++++++++++-------- .../uber/hoodie/io/HoodieAppendHandle.java | 1 + .../uber/hoodie/io/HoodieCreateHandle.java | 1 + .../com/uber/hoodie/io/HoodieMergeHandle.java | 1 + .../uber/hoodie/metrics/HoodieMetrics.java | 2 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 10 ++++----- .../hoodie/table/HoodieMergeOnReadTable.java | 4 ++-- .../com/uber/hoodie/table/HoodieTable.java | 11 +++++----- 8 files changed, 29 insertions(+), 22 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 147ba56a0f6f4..7389cf6e0b3de 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -498,19 +498,21 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - List writeStatusList = writeStatuses.collect(); - updateMetadataAndRollingStats(actionType, metadata, writeStatusList); + + List stats = writeStatuses.map(status -> status.getStat()).collect(); + + updateMetadataAndRollingStats(actionType, metadata, stats); // Finalize write final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); try { - table.finalizeWrite(jsc, writeStatusList); + table.finalizeWrite(jsc, stats); if (finalizeCtx != null) { Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); durationInMs.ifPresent(duration -> { logger.info("Finalize write elapsed time (milliseconds): " + duration); - metrics.updateFinalizeWriteMetrics(duration, writeStatusList.size()); + metrics.updateFinalizeWriteMetrics(duration, stats.size()); }); } } catch (HoodieIOException ioe) { @@ -1260,7 +1262,7 @@ private Optional forceCompact(Optional> extraMetadat } private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetadata metadata, - List writeStatusList) { + List writeStats) { // TODO : make sure we cannot rollback / archive last commit file try { // Create a Hoodie table which encapsulated the commits and files visible @@ -1273,14 +1275,15 @@ private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetada // 2. Now, first read the existing rolling stats and merge with the result of current metadata. // Need to do this on every commit (delta or commit) to support COW and MOR. - for (WriteStatus status : writeStatusList) { - HoodieWriteStat stat = status.getStat(); + + for (HoodieWriteStat stat : writeStats) { + String partitionPath = stat.getPartitionPath(); //TODO: why is stat.getPartitionPath() null at times here. - metadata.addWriteStat(status.getPartitionPath(), stat); + metadata.addWriteStat(partitionPath, stat); HoodieRollingStat hoodieRollingStat = new HoodieRollingStat(stat.getFileId(), stat.getNumWrites() - (stat.getNumUpdateWrites() - stat.getNumDeletes()), stat.getNumUpdateWrites(), stat.getNumDeletes(), stat.getTotalWriteBytes()); - rollingStatMetadata.addRollingStat(status.getPartitionPath(), hoodieRollingStat); + rollingStatMetadata.addRollingStat(partitionPath, hoodieRollingStat); } // The last rolling stat should be present in the completed timeline Optional lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index c60384475b2c8..5434c576630d6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -127,6 +127,7 @@ private void init(HoodieRecord record) { writeStatus.getStat().setPrevCommit(baseInstantTime); writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setPartitionPath(partitionPath); writeStatus.getStat().setFileId(fileId); averageRecordSize = SizeEstimator.estimate(record); try { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 0781faf36de41..c2688b8f0362f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -149,6 +149,7 @@ public WriteStatus close() { storageWriter.close(); HoodieWriteStat stat = new HoodieWriteStat(); + stat.setPartitionPath(status.getPartitionPath()); stat.setNumWrites(recordsWritten); stat.setNumDeletes(recordsDeleted); stat.setNumInserts(insertRecordsWritten); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index f11fce57c1c74..621b37a76c3fd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -127,6 +127,7 @@ private void init(String fileId, String partitionPath, Optional // file name is same for all records, in this bunch writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setPartitionPath(partitionPath); writeStatus.getStat().setFileId(fileId); writeStatus.getStat().setPaths(new Path(config.getBasePath()), newFilePath, tempPath); // Create the writer for writing the new version file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index ead65d70d27bd..aa9afa6fc3b06 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -162,7 +162,7 @@ public void updateCleanMetrics(long durationInMs, int numFilesDeleted) { } } - public void updateFinalizeWriteMetrics(long durationInMs, int numFilesFinalized) { + public void updateFinalizeWriteMetrics(long durationInMs, long numFilesFinalized) { if (config.isMetricsOn()) { logger.info(String .format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 67411627b0a27..fc6dcf90415f8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -29,6 +29,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRollingStatMetadata; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; @@ -376,20 +377,19 @@ public List rollback(JavaSparkContext jsc, List comm /** * Finalize the written data files * - * @param writeStatuses List of WriteStatus + * @param stats List of HoodieWriteStats * @return number of files finalized */ @Override @SuppressWarnings("unchecked") - public void finalizeWrite(JavaSparkContext jsc, List writeStatuses) + public void finalizeWrite(JavaSparkContext jsc, List stats) throws HoodieIOException { - super.finalizeWrite(jsc, writeStatuses); + super.finalizeWrite(jsc, stats); if (config.shouldUseTempFolderForCopyOnWrite()) { // This is to rename each data file from temporary path to its final location - jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism()) - .map(status -> status.getStat()) + jsc.parallelize(stats, config.getFinalizeWriteParallelism()) .foreach(writeStat -> { final FileSystem fs = getMetaClient().getFs(); final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 9968d72d40b89..ad065fbfcadf5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -295,10 +295,10 @@ public List rollback(JavaSparkContext jsc, List comm } @Override - public void finalizeWrite(JavaSparkContext jsc, List writeStatuses) + public void finalizeWrite(JavaSparkContext jsc, List stats) throws HoodieIOException { // delegate to base class for MOR tables - super.finalizeWrite(jsc, writeStatuses); + super.finalizeWrite(jsc, stats); } @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 28de32b902ae1..10ef94ed25f3d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -23,6 +23,7 @@ import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; @@ -252,15 +253,15 @@ public abstract List rollback(JavaSparkContext jsc, List writeStatuses) + public void finalizeWrite(JavaSparkContext jsc, List stats) throws HoodieIOException { if (config.isConsistencyCheckEnabled()) { - List pathsToCheck = writeStatuses.stream() - .map(ws -> ws.getStat().getTempPath() != null - ? ws.getStat().getTempPath() : ws.getStat().getPath()) + List pathsToCheck = stats.stream() + .map(stat -> stat.getTempPath() != null + ? stat.getTempPath() : stat.getPath()) .collect(Collectors.toList()); List failingPaths = new ConsistencyCheck(config.getBasePath(), pathsToCheck, jsc, From 3dfeb04147d3d3b6fa75164c17092a6903a4409c Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Fri, 30 Nov 2018 16:21:20 -0800 Subject: [PATCH 129/374] Update committership for balaji --- pom.xml | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index b9ff0e50a8fe1..f4255a0782ae8 100644 --- a/pom.xml +++ b/pom.xml @@ -73,6 +73,11 @@ Nishith Agarwal Uber + + bvaradar + Balaji Varadharajan + Uber + @@ -100,10 +105,6 @@ Kaushik Devarajaiah Uber - - Balaji Varadharajan - Uber - Anbu Cheeralan DoubleVerify From f42aa0bbdc6d588913562d5a8f2c42d11fb1560f Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Mon, 26 Nov 2018 23:17:32 -0800 Subject: [PATCH 130/374] Returning empty Statues for an empty spark partition caused due to incorrect bin packing --- .../com/uber/hoodie/table/HoodieCopyOnWriteTable.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index fc6dcf90415f8..0a7eeaecab6e7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -176,6 +176,11 @@ public JavaRDD compact(JavaSparkContext jsc, String compactionInsta public Iterator> handleUpdate(String commitTime, String fileId, Iterator> recordItr) throws IOException { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + logger.info("Empty partition with fileId => " + fileId); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } // these are updates HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, recordItr); return handleUpdateInternal(upsertHandle, commitTime, fileId); @@ -235,6 +240,11 @@ protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId, public Iterator> handleInsert(String commitTime, Iterator> recordItr) throws Exception { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + logger.info("Empty partition"); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this); } From 78926cafd951a7a2e827d47c562c27829c22a3ae Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 4 Nov 2018 16:03:56 -0800 Subject: [PATCH 131/374] Serializing the complete payload object instead of serializing just the GenericRecord Removing Converter hierarchy as we now depend purely on JavaSerialization and require the payload to be java serializable --- .../com/uber/hoodie/io/HoodieMergeHandle.java | 6 +- .../common/model/HoodieAvroPayload.java | 12 +- .../log/HoodieMergedLogRecordScanner.java | 3 - .../common/util/collection/DiskBasedMap.java | 22 +- .../util/collection/ExternalSpillableMap.java | 13 +- .../util/collection/LazyFileIterable.java | 17 +- .../util/collection/converter/Converter.java | 34 --- .../converter/HoodieRecordConverter.java | 109 --------- .../collection/converter/StringConverter.java | 35 --- .../util/collection/TestDiskBasedMap.java | 214 ------------------ .../collection/TestExternalSpillableMap.java | 20 +- .../java/com/uber/hoodie/BaseAvroPayload.java | 19 +- .../OverwriteWithLatestAvroPayload.java | 3 +- .../test/scala/DataSourceDefaultsTest.scala | 4 +- 14 files changed, 53 insertions(+), 458 deletions(-) delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java delete mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 621b37a76c3fd..580975f922ca7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -30,8 +30,6 @@ import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieUpsertException; @@ -150,9 +148,7 @@ private String init(String fileId, Iterator> newRecordsItr) { // Load the new records in a map logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), - config.getSpillableMapBasePath(), new StringConverter(), - new HoodieRecordConverter(schema, config.getPayloadClass()), - new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); + config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java index a6c45c7370033..9e4be0db93ad9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java @@ -17,6 +17,7 @@ package com.uber.hoodie.common.model; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Optional; import org.apache.avro.Schema; @@ -29,10 +30,16 @@ */ public class HoodieAvroPayload implements HoodieRecordPayload { - private final Optional record; + // Store the GenericRecord converted to bytes - 1) Doesn't store schema hence memory efficient 2) Makes the payload + // java serializable + private final byte [] recordBytes; public HoodieAvroPayload(Optional record) { - this.record = record; + try { + this.recordBytes = HoodieAvroUtils.avroToBytes(record.get()); + } catch (IOException io) { + throw new HoodieIOException("Cannot convert record to bytes", io); + } } @Override @@ -48,6 +55,7 @@ public Optional combineAndGetUpdateValue(IndexedRecord currentVal @Override public Optional getInsertValue(Schema schema) throws IOException { + Optional record = Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema)); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java index ed976606958ef..6a12d5914687b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java @@ -23,8 +23,6 @@ import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; import com.uber.hoodie.common.util.HoodieTimer; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Iterator; @@ -71,7 +69,6 @@ public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, - new StringConverter(), new HoodieRecordConverter(readerSchema, getPayloadClassFQN()), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema)); // Do the scan and merge timer.startTimer(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java index 598025471d093..047b997969d6b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -16,8 +16,8 @@ package com.uber.hoodie.common.util.collection; +import com.uber.hoodie.common.util.SerializationUtils; import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; @@ -26,6 +26,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.RandomAccessFile; +import java.io.Serializable; import java.net.InetAddress; import java.util.AbstractMap; import java.util.Collection; @@ -45,15 +46,11 @@ * without any rollover support. It uses the following : 1) An in-memory map that tracks the key-> latest ValueMetadata. * 2) Current position in the file NOTE : Only String.class type supported for Key */ -public final class DiskBasedMap implements Map { +public final class DiskBasedMap implements Map { private static final Logger log = LogManager.getLogger(DiskBasedMap.class); // Stores the key and corresponding value's latest metadata spilled to disk private final Map valueMetadataMap; - // Key converter to convert key type to bytes - private final Converter keyConverter; - // Value converter to convert value type to bytes - private final Converter valueConverter; // Read only file access to be able to seek to random positions to readFromDisk values private RandomAccessFile readOnlyFileHandle; // Write only OutputStream to be able to ONLY append to the file @@ -67,8 +64,7 @@ public final class DiskBasedMap implements Map { private String filePath; - protected DiskBasedMap(String baseFilePath, - Converter keyConverter, Converter valueConverter) throws IOException { + protected DiskBasedMap(String baseFilePath) throws IOException { this.valueMetadataMap = new HashMap<>(); File writeOnlyFileHandle = new File(baseFilePath, UUID.randomUUID().toString()); this.filePath = writeOnlyFileHandle.getPath(); @@ -76,8 +72,6 @@ protected DiskBasedMap(String baseFilePath, this.fileOutputStream = new FileOutputStream(writeOnlyFileHandle, true); this.writeOnlyFileHandle = new SizeAwareDataOutputStream(fileOutputStream); this.filePosition = new AtomicLong(0L); - this.keyConverter = keyConverter; - this.valueConverter = valueConverter; } private void initFile(File writeOnlyFileHandle) throws IOException { @@ -125,7 +119,7 @@ public void run() { */ public Iterator iterator() { return new LazyFileIterable(readOnlyFileHandle, - valueMetadataMap, valueConverter).iterator(); + valueMetadataMap).iterator(); } /** @@ -162,7 +156,7 @@ public R get(Object key) { return null; } try { - return this.valueConverter.getData(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle, + return SerializationUtils.deserialize(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle, entry.getOffsetOfValue(), entry.getSizeOfValue())); } catch (IOException e) { throw new HoodieIOException("Unable to readFromDisk Hoodie Record from disk", e); @@ -172,12 +166,12 @@ public R get(Object key) { @Override public R put(T key, R value) { try { - byte[] val = this.valueConverter.getBytes(value); + byte[] val = SerializationUtils.serialize(value); Integer valueSize = val.length; Long timestamp = new Date().getTime(); this.valueMetadataMap.put(key, new DiskBasedMap.ValueMetadata(this.filePath, valueSize, filePosition.get(), timestamp)); - byte[] serializedKey = keyConverter.getBytes(key); + byte[] serializedKey = SerializationUtils.serialize(key); filePosition.set(SpillableMapUtils.spillToDisk(writeOnlyFileHandle, new FileEntry(SpillableMapUtils.generateChecksum(val), serializedKey.length, valueSize, serializedKey, val, timestamp))); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index 081a889e75a77..5ac94d7c7952e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -18,9 +18,9 @@ import com.twitter.common.objectsize.ObjectSizeCalculator; import com.uber.hoodie.common.util.SizeEstimator; -import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieNotSupportedException; import java.io.IOException; +import java.io.Serializable; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -39,7 +39,7 @@ * trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is available, resulting * in OOM. However, if the spill threshold is too low, we spill frequently and incur unnecessary disk writes. */ -public class ExternalSpillableMap implements Map { +public class ExternalSpillableMap implements Map { // Find the actual estimated payload size after inserting N records private static final int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100; @@ -53,10 +53,6 @@ public class ExternalSpillableMap implements Map { // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and // incorrect payload estimation private final Double sizingFactorForInMemoryMap = 0.8; - // Key converter to convert key type to bytes - private final Converter keyConverter; - // Value converter to convert value type to bytes - private final Converter valueConverter; // Size Estimator for key type private final SizeEstimator keySizeEstimator; // Size Estimator for key types @@ -69,15 +65,12 @@ public class ExternalSpillableMap implements Map { private boolean shouldEstimatePayloadSize = true; public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath, - Converter keyConverter, Converter valueConverter, SizeEstimator keySizeEstimator, SizeEstimator valueSizeEstimator) throws IOException { this.inMemoryMap = new HashMap<>(); - this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter); + this.diskBasedMap = new DiskBasedMap<>(baseFilePath); this.maxInMemorySizeInBytes = (long) Math .floor(maxInMemorySizeInBytes * sizingFactorForInMemoryMap); this.currentInMemoryMapSize = 0L; - this.keyConverter = keyConverter; - this.valueConverter = valueConverter; this.keySizeEstimator = keySizeEstimator; this.valueSizeEstimator = valueSizeEstimator; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java index 0d53e2659ffb6..c2d74e0bd4852 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java @@ -16,8 +16,8 @@ package com.uber.hoodie.common.util.collection; +import com.uber.hoodie.common.util.SerializationUtils; import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; @@ -37,20 +37,16 @@ public class LazyFileIterable implements Iterable { private final RandomAccessFile readOnlyFileHandle; // Stores the key and corresponding value's latest metadata spilled to disk private final Map inMemoryMetadataOfSpilledData; - private final Converter valueConverter; - public LazyFileIterable(RandomAccessFile file, Map map, - Converter valueConverter) { + public LazyFileIterable(RandomAccessFile file, Map map) { this.readOnlyFileHandle = file; this.inMemoryMetadataOfSpilledData = map; - this.valueConverter = valueConverter; } @Override public Iterator iterator() { try { - return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData, - valueConverter); + return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData); } catch (IOException io) { throw new HoodieException("Unable to initialize iterator for file on disk", io); } @@ -61,14 +57,11 @@ public Iterator iterator() { */ public class LazyFileIterator implements Iterator { - private final Converter valueConverter; private RandomAccessFile readOnlyFileHandle; private Iterator> metadataIterator; - public LazyFileIterator(RandomAccessFile file, Map map, - Converter valueConverter) throws IOException { + public LazyFileIterator(RandomAccessFile file, Map map) throws IOException { this.readOnlyFileHandle = file; - this.valueConverter = valueConverter; // sort the map in increasing order of offset of value so disk seek is only in one(forward) direction this.metadataIterator = map .entrySet() @@ -88,7 +81,7 @@ public boolean hasNext() { public R next() { Map.Entry entry = this.metadataIterator.next(); try { - return valueConverter.getData(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle, + return SerializationUtils.deserialize(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle, entry.getValue().getOffsetOfValue(), entry.getValue().getSizeOfValue())); } catch (IOException e) { throw new HoodieIOException("Unable to read hoodie record from value spilled to disk", e); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java deleted file mode 100644 index 55168baa92589..0000000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.common.util.collection.converter; - -/** - * A converter interface to getBytes or deserialize a payload. This is used in {@link - * com.uber.hoodie.common.util.collection.ExternalSpillableMap} to spillToDisk - */ -public interface Converter { - - /** - * This method is used to convert a payload to bytes - */ - byte[] getBytes(T t); - - /** - * This method is used to convert the serialized payload (in bytes) to the actual payload instance - */ - T getData(byte[] bytes); -} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java deleted file mode 100644 index 1f5ad6b1f9f58..0000000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.common.util.collection.converter; - -import com.uber.hoodie.common.model.HoodieKey; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieRecordLocation; -import com.uber.hoodie.common.model.HoodieRecordPayload; -import com.uber.hoodie.common.util.HoodieAvroUtils; -import com.uber.hoodie.common.util.ReflectionUtils; -import com.uber.hoodie.common.util.SerializationUtils; -import com.uber.hoodie.common.util.collection.Pair; -import com.uber.hoodie.common.util.collection.Triple; -import com.uber.hoodie.exception.HoodieSerializationException; -import java.io.IOException; -import java.util.Optional; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -/** - * A default converter implementation for HoodieRecord - */ -public class HoodieRecordConverter implements - Converter> { - - // Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa - private final Schema schema; - // The client implementation of HoodieRecordPayload used to re-create HoodieRecord from bytes - private final String payloadClazz; - - private static Logger log = LogManager.getLogger(HoodieRecordConverter.class); - - public HoodieRecordConverter(Schema schema, String payloadClazz) { - this.schema = schema; - this.payloadClazz = payloadClazz; - } - - @Override - public byte[] getBytes(HoodieRecord hoodieRecord) { - try { - // Need to initialize this to 0 bytes since deletes are handled by putting an empty record in HoodieRecord - byte[] val = new byte[0]; - if (hoodieRecord.getData().getInsertValue(schema).isPresent()) { - val = HoodieAvroUtils - .avroToBytes((GenericRecord) hoodieRecord.getData().getInsertValue(schema).get()); - } - byte [] currentLocation = hoodieRecord.getCurrentLocation() != null ? SerializationUtils.serialize(hoodieRecord - .getCurrentLocation()) : new byte[0]; - byte [] newLocation = hoodieRecord.getNewLocation().isPresent() ? SerializationUtils.serialize( - (HoodieRecordLocation) hoodieRecord.getNewLocation().get()) : new byte[0]; - - // Triple, Pair, data> - Triple, Pair, byte[]> data = - Triple.of(Pair.of(hoodieRecord.getKey().getRecordKey(), - hoodieRecord.getKey().getPartitionPath()), Pair.of(currentLocation, newLocation), val); - return SerializationUtils.serialize(data); - } catch (IOException io) { - throw new HoodieSerializationException("Cannot serialize value to bytes", io); - } - } - - @Override - public HoodieRecord getData(byte[] bytes) { - try { - Triple, Pair, byte[]> data = SerializationUtils.deserialize(bytes); - Optional payload = Optional.empty(); - HoodieRecordLocation currentLocation = null; - HoodieRecordLocation newLocation = null; - if (data.getRight().length > 0) { - // This can happen if the record is deleted, the payload is optional with 0 bytes - payload = Optional.of(HoodieAvroUtils.bytesToAvro(data.getRight(), schema)); - } - // Get the currentLocation for the HoodieRecord - if (data.getMiddle().getLeft().length > 0) { - currentLocation = SerializationUtils.deserialize(data.getMiddle().getLeft()); - } - // Get the newLocation for the HoodieRecord - if (data.getMiddle().getRight().length > 0) { - newLocation = SerializationUtils.deserialize(data.getMiddle().getRight()); - } - HoodieRecord hoodieRecord = new HoodieRecord<>( - new HoodieKey(data.getLeft().getKey(), data.getLeft().getValue()), - ReflectionUtils - .loadPayload(payloadClazz, - new Object[]{payload}, Optional.class)); - hoodieRecord.setCurrentLocation(currentLocation); - hoodieRecord.setNewLocation(newLocation); - return hoodieRecord; - } catch (IOException io) { - throw new HoodieSerializationException("Cannot de-serialize value from bytes", io); - } - } -} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java deleted file mode 100644 index 7855484dbec7a..0000000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.common.util.collection.converter; - -import java.nio.charset.StandardCharsets; - -/** - * A default converter implementation for String type of payload key - */ -public class StringConverter implements Converter { - - @Override - public byte[] getBytes(String s) { - return s.getBytes(StandardCharsets.UTF_8); - } - - @Override - public String getData(byte[] bytes) { - return new String(bytes); - } -} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java deleted file mode 100644 index 83d20bd2baaa7..0000000000000 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * 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 com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; -import com.uber.hoodie.common.util.collection.converter.StringConverter; -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, - new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); - List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); - List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); - - // make sure records have spilled to disk - assertTrue(records.sizeOfFileOnDiskInBytes() > 0); - Iterator> itr = records.iterator(); - List oRecords = new ArrayList<>(); - while (itr.hasNext()) { - HoodieRecord 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, - new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); - List hoodieRecords = SchemaTestUtil - .generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); - Set 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> itr = records.iterator(); - List oRecords = new ArrayList<>(); - while (itr.hasNext()) { - HoodieRecord 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, - new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); - List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - - // perform some inserts - List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); - - long fileSize = records.sizeOfFileOnDiskInBytes(); - // make sure records have spilled to disk - assertTrue(fileSize > 0); - - // generate updates from inserts - List 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> itr = records.iterator(); - while (itr.hasNext()) { - HoodieRecord 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 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 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 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); - } -} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index c85e869ddb67b..240684c1a9e0e 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -30,8 +30,6 @@ 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.collection.converter.HoodieRecordConverter; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -67,8 +65,7 @@ public void simpleInsertTest() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -90,8 +87,7 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -129,8 +125,7 @@ public void testAllMapOperations() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -186,8 +181,7 @@ public void simpleTestWithException() throws IOException, URISyntaxException { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz), + new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -206,8 +200,7 @@ public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOExcept String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List recordKeys = new ArrayList<>(); @@ -260,8 +253,7 @@ public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISy String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz), + new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List recordKeys = new ArrayList<>(); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java index 191b2d236e22c..c2a05c9a6da09 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java @@ -18,7 +18,10 @@ package com.uber.hoodie; +import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; import java.io.Serializable; import org.apache.avro.generic.GenericRecord; @@ -29,9 +32,14 @@ public abstract class BaseAvroPayload implements Serializable { /** - * Avro data extracted from the source + * Avro data extracted from the source converted to bytes */ - protected final GenericRecord record; + protected final byte [] recordBytes; + + /** + * The schema of the Avro data + */ + protected final String schemaStr; /** * For purposes of preCombining @@ -43,7 +51,12 @@ public abstract class BaseAvroPayload implements Serializable { * @param orderingVal */ public BaseAvroPayload(GenericRecord record, Comparable orderingVal) { - this.record = record; + try { + this.recordBytes = HoodieAvroUtils.avroToBytes(record); + this.schemaStr = record.getSchema().toString(); + } catch (IOException io) { + throw new HoodieIOException("Cannot convert GenericRecord to bytes", io); + } this.orderingVal = orderingVal; if (orderingVal == null) { throw new HoodieException("Ordering value is null for record: " + record); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java index 48396f08c9486..0b454f1ef0c87 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java @@ -66,6 +66,7 @@ public Optional combineAndGetUpdateValue(IndexedRecord currentVal @Override public Optional getInsertValue(Schema schema) throws IOException { - return Optional.of(HoodieAvroUtils.rewriteRecord(record, schema)); + return Optional.of(HoodieAvroUtils.rewriteRecord(HoodieAvroUtils.bytesToAvro(recordBytes, Schema.parse(schemaStr)), + schema)); } } diff --git a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala index 734cc7f5366cf..e136545da6d4e 100644 --- a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala @@ -107,11 +107,11 @@ class DataSourceDefaultsTest extends AssertionsForJUnit { // it will provide the record with greatest combine value val combinedPayload12 = overWritePayload1.preCombine(overWritePayload2) val combinedGR12 = combinedPayload12.getInsertValue(schema).get().asInstanceOf[GenericRecord] - assertEquals("field2", combinedGR12.get("field1")) + assertEquals("field2", combinedGR12.get("field1").toString) // and it will be deterministic, to order of processing. val combinedPayload21 = overWritePayload2.preCombine(overWritePayload1) val combinedGR21 = combinedPayload21.getInsertValue(schema).get().asInstanceOf[GenericRecord] - assertEquals("field2", combinedGR21.get("field1")) + assertEquals("field2", combinedGR21.get("field1").toString) } } From 86207e6305b096f297728fd97e2f71309d688e7a Mon Sep 17 00:00:00 2001 From: "jiale.tan" Date: Thu, 11 Oct 2018 17:40:32 -0700 Subject: [PATCH 132/374] feat(SparkDataSource): add structured streaming --- .../com/uber/hoodie/AvroConversionUtils.scala | 5 +- .../com/uber/hoodie/DataSourceOptions.scala | 22 ++ .../scala/com/uber/hoodie/DefaultSource.scala | 232 ++------------- .../uber/hoodie/HoodieSparkSqlWriter.scala | 266 +++++++++++++++++ .../com/uber/hoodie/HoodieStreamingSink.scala | 117 ++++++++ .../src/test/java/HoodieJavaStreamingApp.java | 279 ++++++++++++++++++ .../src/test/scala/DataSourceTest.scala | 96 +++++- 7 files changed, 799 insertions(+), 218 deletions(-) create mode 100644 hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala create mode 100644 hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala create mode 100644 hoodie-spark/src/test/java/HoodieJavaStreamingApp.java diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index e92043490690d..0dd03ee15f3e2 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -26,6 +26,7 @@ import org.apache.avro.generic.GenericData.Record import org.apache.avro.generic.GenericRecord import org.apache.avro.{Schema, SchemaBuilder} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row} @@ -34,7 +35,9 @@ object AvroConversionUtils { def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = { val dataType = df.schema - df.rdd.mapPartitions { records => + val encoder = RowEncoder.apply(dataType).resolveAndBind() + df.queryExecution.toRdd.map(encoder.fromRow) + .mapPartitions { records => if (records.isEmpty) Iterator.empty else { val convertor = createConverterToAvro(dataType, structName, recordNamespace) diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala index 63e1d19c5ed0e..7e4f8f03efc01 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala @@ -152,6 +152,28 @@ object DataSourceWriteOptions { val INSERT_DROP_DUPS_OPT_KEY = "hoodie.datasource.write.insert.drop.duplicates" val DEFAULT_INSERT_DROP_DUPS_OPT_VAL = "false" + /** + * Flag to indicate how many times streaming job should retry for a failed microbatch + * By default 3 + */ + val STREAMING_RETRY_CNT_OPT_KEY = "hoodie.datasource.write.streaming.retry.count" + val DEFAULT_STREAMING_RETRY_CNT_OPT_VAL = "3" + + /** + * Flag to indicate how long (by millisecond) before a retry should issued for failed microbatch + * By default 2000 and it will be doubled by every retry + */ + val STREAMING_RETRY_INTERVAL_MS_OPT_KEY = "hoodie.datasource.write.streaming.retry.interval.ms" + val DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL = "2000" + + /** + * Flag to indicate whether to ignore any non exception error (e.g. writestatus error) + * within a streaming microbatch + * By default true (in favor of streaming progressing over data integrity) + */ + val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = "hoodie.datasource.write.streaming.ignore.failed.batch" + val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = "true" + // HIVE SYNC SPECIFIC CONFIGS //NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes // unexpected issues with config getting reset diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 608974baf7333..dcf7628bd989e 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -18,30 +18,20 @@ package com.uber.hoodie -import java.util -import java.util.Optional import java.util.concurrent.ConcurrentHashMap import com.uber.hoodie.DataSourceReadOptions._ -import com.uber.hoodie.DataSourceWriteOptions._ -import com.uber.hoodie.common.table.HoodieTableMetaClient -import com.uber.hoodie.common.util.{FSUtils, TypedProperties} -import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException -import com.uber.hoodie.hive.{HiveSyncConfig, HiveSyncTool} -import org.apache.avro.generic.GenericRecord -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hive.conf.HiveConf import org.apache.log4j.LogManager -import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.sources._ +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} import scala.collection.JavaConversions._ -import scala.collection.mutable.ListBuffer +import scala.collection.mutable /** * Hoodie Spark Datasource, for reading and writing hoodie datasets @@ -51,6 +41,7 @@ class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider with DataSourceRegister + with StreamSinkProvider with Serializable { private val log = LogManager.getLogger(classOf[DefaultSource]) @@ -66,7 +57,7 @@ class DefaultSource extends RelationProvider * @param parameters * @return */ - def parametersWithReadDefaults(parameters: Map[String, String]) = { + def parametersWithReadDefaults(parameters: Map[String, String]): mutable.Map[String, String] = { val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters)) defaultsMap.putIfAbsent(VIEW_TYPE_OPT_KEY, DEFAULT_VIEW_TYPE_OPT_VAL) mapAsScalaMap(defaultsMap) @@ -106,216 +97,27 @@ class DefaultSource extends RelationProvider } } - /** - * Add default options for unspecified write options keys. - * - * @param parameters - * @return - */ - def parametersWithWriteDefaults(parameters: Map[String, String]) = { - val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters)) - defaultsMap.putIfAbsent(OPERATION_OPT_KEY, DEFAULT_OPERATION_OPT_VAL) - defaultsMap.putIfAbsent(STORAGE_TYPE_OPT_KEY, DEFAULT_STORAGE_TYPE_OPT_VAL) - defaultsMap.putIfAbsent(PRECOMBINE_FIELD_OPT_KEY, DEFAULT_PRECOMBINE_FIELD_OPT_VAL) - defaultsMap.putIfAbsent(PAYLOAD_CLASS_OPT_KEY, DEFAULT_PAYLOAD_OPT_VAL) - defaultsMap.putIfAbsent(RECORDKEY_FIELD_OPT_KEY, DEFAULT_RECORDKEY_FIELD_OPT_VAL) - defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL) - defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) - defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL) - defaultsMap.putIfAbsent(INSERT_DROP_DUPS_OPT_KEY, DEFAULT_INSERT_DROP_DUPS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_USER_OPT_KEY, DEFAULT_HIVE_USER_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_PASS_OPT_KEY, DEFAULT_HIVE_PASS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_URL_OPT_KEY, DEFAULT_HIVE_URL_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_PARTITION_FIELDS_OPT_KEY, DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_ASSUME_DATE_PARTITION_OPT_KEY, DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL) - mapAsScalaMap(defaultsMap) - } - - def toProperties(params: Map[String, String]): TypedProperties = { - val props = new TypedProperties() - params.foreach(kv => props.setProperty(kv._1, kv._2)) - props - } - - override def createRelation(sqlContext: SQLContext, mode: SaveMode, optParams: Map[String, String], df: DataFrame): BaseRelation = { - val parameters = parametersWithWriteDefaults(optParams).toMap - val sparkContext = sqlContext.sparkContext - val path = parameters.get("path") - val tblName = parameters.get(HoodieWriteConfig.TABLE_NAME) - if (path.isEmpty || tblName.isEmpty) { - throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.") - } - val serializer = sparkContext.getConf.get("spark.serializer") - if (!serializer.equals("org.apache.spark.serializer.KryoSerializer")) { - throw new HoodieException(s"${serializer} serialization is not supported by hoodie. Please use kryo.") - } - - val storageType = parameters(STORAGE_TYPE_OPT_KEY) - val operation = - // It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true - // Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly - // or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) . - if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean && - parameters(OPERATION_OPT_KEY) == UPSERT_OPERATION_OPT_VAL) { - - log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " + - s"when $INSERT_DROP_DUPS_OPT_KEY is set to be true, " + - s"overriding the $OPERATION_OPT_KEY to be $INSERT_OPERATION_OPT_VAL") - - INSERT_OPERATION_OPT_VAL - } else { - parameters(OPERATION_OPT_KEY) - } - - // register classes & schemas - val structName = s"${tblName.get}_record" - val nameSpace = s"hoodie.${tblName.get}" - sparkContext.getConf.registerKryoClasses( - Array(classOf[org.apache.avro.generic.GenericData], - classOf[org.apache.avro.Schema])) - val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) - sparkContext.getConf.registerAvroSchemas(schema) - log.info(s"Registered avro schema : ${schema.toString(true)}"); - - // Convert to RDD[HoodieRecord] - val keyGenerator = DataSourceUtils.createKeyGenerator( - parameters(KEYGENERATOR_CLASS_OPT_KEY), - toProperties(parameters) - ) - val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) - val hoodieAllIncomingRecords = genericRecords.map(gr => { - val orderingVal = DataSourceUtils.getNestedFieldValAsString( - gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]] - DataSourceUtils.createHoodieRecord(gr, - orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY)) - }).toJavaRDD(); - - val jsc = new JavaSparkContext(sparkContext) - - val hoodieRecords = - if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) { - DataSourceUtils.dropDuplicates( - jsc, - hoodieAllIncomingRecords, - mapAsJavaMap(parameters)) - } else { - hoodieAllIncomingRecords - } - - if (!hoodieRecords.isEmpty()) { - val basePath = new Path(parameters.get("path").get) - val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) - var exists = fs.exists(basePath) - - // Handle various save modes - if (mode == SaveMode.ErrorIfExists && exists) { - throw new HoodieException(s"basePath ${basePath} already exists.") - } - if (mode == SaveMode.Ignore && exists) { - log.warn(s" basePath ${basePath} already exists. Ignoring & not performing actual writes.") - return createRelation(sqlContext, parameters, df.schema) - } - if (mode == SaveMode.Overwrite && exists) { - log.warn(s" basePath ${basePath} already exists. Deleting existing data & overwriting with new data.") - fs.delete(basePath, true) - exists = false - } - - // Create the dataset if not present (APPEND mode) - if (!exists) { - HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType, - tblName.get, "archived") - } - - // Create a HoodieWriteClient & issue the write. - val client = DataSourceUtils.createHoodieClient(jsc, - schema.toString, - path.get, - tblName.get, - mapAsJavaMap(parameters) - ) - val commitTime = client.startCommit(); - - val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, commitTime, operation) - // Check for errors and commit the write. - val errorCount = writeStatuses.rdd.filter(ws => ws.hasErrors).count() - if (errorCount == 0) { - log.info("No errors. Proceeding to commit the write."); - val metaMap = parameters.filter(kv => - kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY))) - val success = if (metaMap.isEmpty) { - client.commit(commitTime, writeStatuses) - } else { - client.commit(commitTime, writeStatuses, - Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) - } - - if (success) { - log.info("Commit " + commitTime + " successful!") - } - else { - log.info("Commit " + commitTime + " failed!") - } - - val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false) - if (hiveSyncEnabled) { - log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")") - val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration) - syncHive(basePath, fs, parameters) - } - client.close - } else { - log.error(s"$operation failed with ${errorCount} errors :"); - if (log.isTraceEnabled) { - log.trace("Printing out the top 100 errors") - writeStatuses.rdd.filter(ws => ws.hasErrors) - .take(100) - .foreach(ws => { - log.trace("Global error :", ws.getGlobalError) - if (ws.getErrors.size() > 0) { - ws.getErrors.foreach(kt => - log.trace(s"Error for key: ${kt._1}", kt._2)) - } - }) - } - } - } else { - log.info("new batch has no new records, skipping...") - } + val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams).toMap + HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df) createRelation(sqlContext, parameters, df.schema) } - private def syncHive(basePath: Path, fs: FileSystem, parameters: Map[String, String]): Boolean = { - val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, parameters) - val hiveConf: HiveConf = new HiveConf() - hiveConf.addResource(fs.getConf) - new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable() - true + override def createSink(sqlContext: SQLContext, + optParams: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { + val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams).toMap + new HoodieStreamingSink( + sqlContext, + parameters, + partitionColumns, + outputMode) } - private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = { - val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig() - hiveSyncConfig.basePath = basePath.toString - hiveSyncConfig.assumeDatePartitioning = - parameters.get(HIVE_ASSUME_DATE_PARTITION_OPT_KEY).exists(r => r.toBoolean) - hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY) - hiveSyncConfig.tableName = parameters(HIVE_TABLE_OPT_KEY) - hiveSyncConfig.hiveUser = parameters(HIVE_USER_OPT_KEY) - hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY) - hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY) - hiveSyncConfig.partitionFields = - ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).toList: _*) - hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) - hiveSyncConfig - } override def shortName(): String = "hoodie" } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala new file mode 100644 index 0000000000000..93ad54878c86f --- /dev/null +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala @@ -0,0 +1,266 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ +package com.uber.hoodie + +import java.util +import java.util.concurrent.ConcurrentHashMap +import java.util.Optional + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ListBuffer + +import com.uber.hoodie.DataSourceWriteOptions._ +import com.uber.hoodie.common.table.HoodieTableMetaClient +import com.uber.hoodie.common.util.{FSUtils, TypedProperties} +import com.uber.hoodie.config.HoodieWriteConfig +import com.uber.hoodie.exception.HoodieException +import com.uber.hoodie.hive.{HiveSyncConfig, HiveSyncTool} +import org.apache.avro.generic.GenericRecord +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.log4j.LogManager +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} + +import scala.collection.mutable + +private[hoodie] object HoodieSparkSqlWriter { + + private val log = LogManager.getLogger("HoodieSparkSQLWriter") + + def write(sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + df: DataFrame): (Boolean, Option[String]) = { + + val sparkContext = sqlContext.sparkContext + val path = parameters.get("path") + val tblName = parameters.get(HoodieWriteConfig.TABLE_NAME) + if (path.isEmpty || tblName.isEmpty) { + throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.") + } + val serializer = sparkContext.getConf.get("spark.serializer") + if (!serializer.equals("org.apache.spark.serializer.KryoSerializer")) { + throw new HoodieException(s"${serializer} serialization is not supported by hoodie. Please use kryo.") + } + + val storageType = parameters(STORAGE_TYPE_OPT_KEY) + val operation = + // It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true + // Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly + // or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) . + if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean && + parameters(OPERATION_OPT_KEY) == UPSERT_OPERATION_OPT_VAL) { + + log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " + + s"when $INSERT_DROP_DUPS_OPT_KEY is set to be true, " + + s"overriding the $OPERATION_OPT_KEY to be $INSERT_OPERATION_OPT_VAL") + + INSERT_OPERATION_OPT_VAL + } else { + parameters(OPERATION_OPT_KEY) + } + + // register classes & schemas + val structName = s"${tblName.get}_record" + val nameSpace = s"hoodie.${tblName.get}" + sparkContext.getConf.registerKryoClasses( + Array(classOf[org.apache.avro.generic.GenericData], + classOf[org.apache.avro.Schema])) + val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) + sparkContext.getConf.registerAvroSchemas(schema) + log.info(s"Registered avro schema : ${schema.toString(true)}") + + // Convert to RDD[HoodieRecord] + val keyGenerator = DataSourceUtils.createKeyGenerator( + parameters(KEYGENERATOR_CLASS_OPT_KEY), + toProperties(parameters) + ) + val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) + val hoodieAllIncomingRecords = genericRecords.map(gr => { + val orderingVal = DataSourceUtils.getNestedFieldValAsString( + gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]] + DataSourceUtils.createHoodieRecord(gr, + orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY)) + }).toJavaRDD(); + + val jsc = new JavaSparkContext(sparkContext) + + val hoodieRecords = + if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) { + DataSourceUtils.dropDuplicates( + jsc, + hoodieAllIncomingRecords, + mapAsJavaMap(parameters)) + } else { + hoodieAllIncomingRecords + } + + if (hoodieRecords.isEmpty()) { + log.info("new batch has no new records, skipping...") + return (true, None) + } + + val basePath = new Path(parameters.get("path").get) + val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) + var exists = fs.exists(basePath) + + // Handle various save modes + if (mode == SaveMode.ErrorIfExists && exists) { + throw new HoodieException(s"basePath ${basePath} already exists.") + } + if (mode == SaveMode.Ignore && exists) { + log.warn(s" basePath ${basePath} already exists. Ignoring & not performing actual writes.") + return (true, None) + } + if (mode == SaveMode.Overwrite && exists) { + log.warn(s" basePath ${basePath} already exists. Deleting existing data & overwriting with new data.") + fs.delete(basePath, true) + exists = false + } + + // Create the dataset if not present (APPEND mode) + if (!exists) { + HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType, + tblName.get, "archived") + } + + // Create a HoodieWriteClient & issue the write. + val client = DataSourceUtils.createHoodieClient(jsc, + schema.toString, + path.get, + tblName.get, + mapAsJavaMap(parameters) + ) + val commitTime = client.startCommit() + + val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, commitTime, operation) + // Check for errors and commit the write. + val errorCount = writeStatuses.rdd.filter(ws => ws.hasErrors).count() + val writeSuccessful = + if (errorCount == 0) { + log.info("No errors. Proceeding to commit the write.") + val metaMap = parameters.filter(kv => + kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY))) + val commitSuccess = if (metaMap.isEmpty) { + client.commit(commitTime, writeStatuses) + } else { + client.commit(commitTime, writeStatuses, + Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) + } + + if (commitSuccess) { + log.info("Commit " + commitTime + " successful!") + } + else { + log.info("Commit " + commitTime + " failed!") + } + + val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false) + val syncHiveSucess = if (hiveSyncEnabled) { + log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")") + val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration) + syncHive(basePath, fs, parameters) + } else { + true + } + client.close() + commitSuccess && syncHiveSucess + } else { + log.error(s"$operation failed with ${errorCount} errors :"); + if (log.isTraceEnabled) { + log.trace("Printing out the top 100 errors") + writeStatuses.rdd.filter(ws => ws.hasErrors) + .take(100) + .foreach(ws => { + log.trace("Global error :", ws.getGlobalError) + if (ws.getErrors.size() > 0) { + ws.getErrors.foreach(kt => + log.trace(s"Error for key: ${kt._1}", kt._2)) + } + }) + } + false + } + (writeSuccessful, Some(commitTime)) + } + + /** + * Add default options for unspecified write options keys. + * + * @param parameters + * @return + */ + def parametersWithWriteDefaults(parameters: Map[String, String]): mutable.Map[String, String] = { + val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters)) + defaultsMap.putIfAbsent(OPERATION_OPT_KEY, DEFAULT_OPERATION_OPT_VAL) + defaultsMap.putIfAbsent(STORAGE_TYPE_OPT_KEY, DEFAULT_STORAGE_TYPE_OPT_VAL) + defaultsMap.putIfAbsent(PRECOMBINE_FIELD_OPT_KEY, DEFAULT_PRECOMBINE_FIELD_OPT_VAL) + defaultsMap.putIfAbsent(PAYLOAD_CLASS_OPT_KEY, DEFAULT_PAYLOAD_OPT_VAL) + defaultsMap.putIfAbsent(RECORDKEY_FIELD_OPT_KEY, DEFAULT_RECORDKEY_FIELD_OPT_VAL) + defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL) + defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) + defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL) + defaultsMap.putIfAbsent(INSERT_DROP_DUPS_OPT_KEY, DEFAULT_INSERT_DROP_DUPS_OPT_VAL) + defaultsMap.putIfAbsent(STREAMING_RETRY_CNT_OPT_KEY, DEFAULT_STREAMING_RETRY_CNT_OPT_VAL) + defaultsMap.putIfAbsent(STREAMING_RETRY_INTERVAL_MS_OPT_KEY, DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL) + defaultsMap.putIfAbsent(STREAMING_IGNORE_FAILED_BATCH_OPT_KEY, DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_USER_OPT_KEY, DEFAULT_HIVE_USER_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_PASS_OPT_KEY, DEFAULT_HIVE_PASS_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_URL_OPT_KEY, DEFAULT_HIVE_URL_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_PARTITION_FIELDS_OPT_KEY, DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL) + defaultsMap.putIfAbsent(HIVE_ASSUME_DATE_PARTITION_OPT_KEY, DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL) + mapAsScalaMap(defaultsMap) + } + + def toProperties(params: Map[String, String]): TypedProperties = { + val props = new TypedProperties() + params.foreach(kv => props.setProperty(kv._1, kv._2)) + props + } + + private def syncHive(basePath: Path, fs: FileSystem, parameters: Map[String, String]): Boolean = { + val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, parameters) + val hiveConf: HiveConf = new HiveConf() + hiveConf.addResource(fs.getConf) + new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable() + true + } + + private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = { + val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig() + hiveSyncConfig.basePath = basePath.toString + hiveSyncConfig.assumeDatePartitioning = + parameters.get(HIVE_ASSUME_DATE_PARTITION_OPT_KEY).exists(r => r.toBoolean) + hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY) + hiveSyncConfig.tableName = parameters(HIVE_TABLE_OPT_KEY) + hiveSyncConfig.hiveUser = parameters(HIVE_USER_OPT_KEY) + hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY) + hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY) + hiveSyncConfig.partitionFields = + ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).toList: _*) + hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) + hiveSyncConfig + } + +} \ No newline at end of file diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala new file mode 100644 index 0000000000000..f8109ccd2044e --- /dev/null +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala @@ -0,0 +1,117 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ +package com.uber.hoodie + +import com.uber.hoodie.exception.HoodieCorruptedDataException +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.spark.sql.execution.streaming.Sink +import org.apache.spark.sql.streaming.OutputMode +import org.apache.log4j.LogManager + +import scala.util.{Failure, Success, Try} + +class HoodieStreamingSink(sqlContext: SQLContext, + options: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode) + extends Sink + with Serializable { + @volatile private var latestBatchId = -1L + + private val log = LogManager.getLogger(classOf[HoodieStreamingSink]) + + private val retryCnt = options(DataSourceWriteOptions.STREAMING_RETRY_CNT_OPT_KEY).toInt + private val retryIntervalMs = options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY).toLong + private val ignoreFailedBatch = options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY).toBoolean + + private val mode = + if (outputMode == OutputMode.Append()) { + SaveMode.Append + } else { + SaveMode.Overwrite + } + + override def addBatch(batchId: Long, data: DataFrame): Unit = { + retry(retryCnt, retryIntervalMs)( + Try( + HoodieSparkSqlWriter.write( + sqlContext, + mode, + options, + data) + ) match { + case Success((true, commitOps)) => + log.info(s"Micro batch id=$batchId succeeded" + + commitOps.map(commit => s" for commit=$commit").getOrElse(" with no new commits")) + Success((true, commitOps)) + case Failure(e) => + // clean up persist rdds in the write process + data.sparkSession.sparkContext.getPersistentRDDs + .foreach { + case (id, rdd) => + rdd.unpersist() + } + log.error(s"Micro batch id=$batchId threw following expection: ", e) + if (ignoreFailedBatch) { + log.info(s"Ignore the exception and move on streaming as per " + + s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration") + Success((true, None)) + } else { + if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...") + Failure(e) + } + case Success((false, commitOps)) => + log.error(s"Micro batch id=$batchId ended up with errors" + + commitOps.map(commit => s" for commit=$commit").getOrElse("")) + if (ignoreFailedBatch) { + log.info(s"Ignore the errors and move on streaming as per " + + s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration") + Success((true, None)) + } else { + if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...") + Failure(new HoodieCorruptedDataException(s"Micro batch id=$batchId ended up with errors")) + } + } + ) match { + case Failure(e) => + if (!ignoreFailedBatch) { + log.error(s"Micro batch id=$batchId threw following expections," + + s"aborting streaming app to avoid data loss: ", e) + // spark sometimes hangs upon exceptions and keep on hold of the executors + // this is to force exit upon errors / exceptions and release all executors + // will require redeployment / supervise mode to restart the streaming + System.exit(1) + } + case Success(_) => + log.info(s"Micro batch id=$batchId succeeded") + } + } + + override def toString: String = s"HoodieStreamingSink[${options("path")}]" + + @annotation.tailrec + private def retry[T](n: Int, waitInMillis: Long)(fn: => Try[T]): Try[T] = { + fn match { + case x: util.Success[T] => x + case _ if n > 1 => + Thread.sleep(waitInMillis) + retry(n - 1, waitInMillis * 2)(fn) + case f => f + } + } +} \ No newline at end of file diff --git a/hoodie-spark/src/test/java/HoodieJavaStreamingApp.java b/hoodie-spark/src/test/java/HoodieJavaStreamingApp.java new file mode 100644 index 0000000000000..dd5586373d202 --- /dev/null +++ b/hoodie-spark/src/test/java/HoodieJavaStreamingApp.java @@ -0,0 +1,279 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.uber.hoodie.DataSourceReadOptions; +import com.uber.hoodie.DataSourceWriteOptions; +import com.uber.hoodie.HoodieDataSourceHelpers; +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.hive.MultiPartKeysValueExtractor; + +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +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; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.DataStreamWriter; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.streaming.ProcessingTime; + +/** + * Sample program that writes & reads hoodie datasets via the Spark datasource streaming + */ +public class HoodieJavaStreamingApp { + + @Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table") + private String tablePath = "file:///tmp/hoodie/streaming/sample-table"; + + @Parameter(names = {"--streaming-source-path", "-ssp"}, description = "path for streaming source file folder") + private String streamingSourcePath = "file:///tmp/hoodie/streaming/source"; + + @Parameter(names = {"--streaming-checkpointing-path", "-scp"}, + description = "path for streaming checking pointing folder") + private String streamingCheckpointingPath = "file:///tmp/hoodie/streaming/checkpoint"; + + @Parameter(names = {"--streaming-duration-in-ms", "-sdm"}, + description = "time in millisecond for the streaming duration") + private Long streamingDurationInMs = 15000L; + + @Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table") + private String tableName = "hoodie_test"; + + @Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ") + private String tableType = HoodieTableType.MERGE_ON_READ.name(); + + @Parameter(names = {"--hive-sync", "-hv"}, description = "Enable syncing to hive") + private Boolean enableHiveSync = false; + + @Parameter(names = {"--hive-db", "-hd"}, description = "hive database") + private String hiveDB = "default"; + + @Parameter(names = {"--hive-table", "-ht"}, description = "hive table") + private String hiveTable = "hoodie_sample_test"; + + @Parameter(names = {"--hive-user", "-hu"}, description = "hive username") + private String hiveUser = "hive"; + + @Parameter(names = {"--hive-password", "-hp"}, description = "hive password") + private String hivePass = "hive"; + + @Parameter(names = {"--hive-url", "-hl"}, description = "hive JDBC URL") + private String hiveJdbcUrl = "jdbc:hive2://localhost:10000"; + + @Parameter(names = {"--use-multi-partition-keys", "-mp"}, description = "Use Multiple Partition Keys") + private Boolean useMultiPartitionKeys = false; + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + + private static Logger logger = LogManager.getLogger(HoodieJavaStreamingApp.class); + + public static void main(String[] args) throws Exception { + HoodieJavaStreamingApp cli = new HoodieJavaStreamingApp(); + JCommander cmd = new JCommander(cli, args); + + if (cli.help) { + cmd.usage(); + System.exit(1); + } + cli.run(); + } + + /** + * + * @throws Exception + */ + public void run() throws Exception { + // Spark session setup.. + SparkSession spark = SparkSession.builder().appName("Hoodie Spark Streaming APP") + .config("spark.serializer", + "org.apache.spark.serializer.KryoSerializer").master("local[1]") + .getOrCreate(); + JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext()); + + // folder path clean up and creation, preparing the environment + FileSystem fs = FileSystem.get(jssc.hadoopConfiguration()); + fs.delete(new Path(streamingSourcePath), true); + fs.delete(new Path(streamingCheckpointingPath), true); + fs.delete(new Path(tablePath), true); + fs.mkdirs(new Path(streamingSourcePath)); + + // Generator of some records to be loaded in. + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + + List records1 = DataSourceTestUtils.convertToStringList( + dataGen.generateInserts("001", 100)); + Dataset inputDF1 = spark.read().json(jssc.parallelize(records1, 2)); + + List records2 = DataSourceTestUtils.convertToStringList( + dataGen.generateUpdates("002", 100)); + + Dataset inputDF2 = spark.read().json(jssc.parallelize(records2, 2)); + + // setup the input for streaming + Dataset streamingInput = spark.readStream().schema(inputDF1.schema()) + .json(streamingSourcePath); + + + // start streaming and showing + ExecutorService executor = Executors.newFixedThreadPool(2); + + // thread for spark strucutured streaming + Future streamFuture = executor.submit(new Callable() { + public Void call() throws Exception { + logger.info("===== Streaming Starting ====="); + stream(streamingInput); + logger.info("===== Streaming Ends ====="); + return null; + } + }); + + // thread for adding data to the streaming source and showing results over time + Future showFuture = executor.submit(new Callable() { + public Void call() throws Exception { + logger.info("===== Showing Starting ====="); + show(spark, fs, inputDF1, inputDF2); + logger.info("===== Showing Ends ====="); + return null; + } + }); + + // let the threads run + streamFuture.get(); + showFuture.get(); + + executor.shutdown(); + } + + /** + * Adding data to the streaming source and showing results over time + * @param spark + * @param fs + * @param inputDF1 + * @param inputDF2 + * @throws Exception + */ + public void show(SparkSession spark, + FileSystem fs, + Dataset inputDF1, + Dataset inputDF2) throws Exception { + inputDF1.write().mode(SaveMode.Append).json(streamingSourcePath); + // wait for spark streaming to process one microbatch + Thread.sleep(3000); + String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); + logger.info("First commit at instant time :" + commitInstantTime1); + + inputDF2.write().mode(SaveMode.Append).json(streamingSourcePath); + // wait for spark streaming to process one microbatch + Thread.sleep(3000); + String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); + logger.info("Second commit at instant time :" + commitInstantTime1); + + /** + * Read & do some queries + */ + Dataset hoodieROViewDF = spark.read().format("com.uber.hoodie") + // pass any path glob, can include hoodie & non-hoodie + // datasets + .load(tablePath + "/*/*/*/*"); + hoodieROViewDF.registerTempTable("hoodie_ro"); + spark.sql("describe hoodie_ro").show(); + // all trips whose fare was greater than 2. + spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 2.0") + .show(); + + if (tableType.equals(HoodieTableType.COPY_ON_WRITE.name())) { + /** + * Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE + */ + Dataset hoodieIncViewDF = spark.read().format("com.uber.hoodie") + .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), + DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), + commitInstantTime1) // Only changes in write 2 above + .load( + tablePath); // For incremental view, pass in the root/base path of dataset + + logger.info("You will only see records from : " + commitInstantTime2); + hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show(); + } + } + + /** + * Hoodie spark streaming job + * @param streamingInput + * @throws Exception + */ + public void stream(Dataset streamingInput) throws Exception { + + DataStreamWriter writer = streamingInput + .writeStream() + .format("com.uber.hoodie") + .option("hoodie.insert.shuffle.parallelism", "2") + .option("hoodie.upsert.shuffle.parallelism", "2") + .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) + .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") + .option(HoodieWriteConfig.TABLE_NAME, tableName) + .option("checkpointLocation", streamingCheckpointingPath) + .outputMode(OutputMode.Append()); + + updateHiveSyncConfig(writer); + writer + .trigger(new ProcessingTime(500)) + .start(tablePath) + .awaitTermination(streamingDurationInMs); + } + + /** + * Setup configs for syncing to hive + * @param writer + * @return + */ + private DataStreamWriter updateHiveSyncConfig(DataStreamWriter writer) { + if (enableHiveSync) { + logger.info("Enabling Hive sync to " + hiveJdbcUrl); + writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable) + .option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB) + .option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl) + .option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser) + .option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass) + .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true"); + if (useMultiPartitionKeys) { + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day") + .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(), + MultiPartKeysValueExtractor.class.getCanonicalName()); + } else { + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr"); + } + } + return writer; + } +} diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index 1ad42e7777ca5..2f34beb4ed535 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -20,14 +20,18 @@ import com.uber.hoodie.common.HoodieTestDataGenerator import com.uber.hoodie.common.util.FSUtils import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} -import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql._ +import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime} import org.junit.Assert._ import org.junit.rules.TemporaryFolder import org.junit.{Before, Test} import org.scalatest.junit.AssertionsForJUnit import scala.collection.JavaConversions._ +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future} +import scala.concurrent.ExecutionContext.Implicits.global /** * Basic tests on the spark datasource @@ -62,7 +66,7 @@ class DataSourceTest extends AssertionsForJUnit { @Test def testCopyOnWriteStorage() { // Insert Operation - val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001", 100)).toList + val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("000", 100)).toList val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("com.uber.hoodie") .options(commonOpts) @@ -182,4 +186,92 @@ class DataSourceTest extends AssertionsForJUnit { .load(basePath) assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt) } + + @Test def testStructuredStreaming(): Unit = { + fs.delete(new Path(basePath), true) + val sourcePath = basePath + "/source" + val destPath = basePath + "/dest" + fs.mkdirs(new Path(sourcePath)) + + // First chunk of data + val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("000", 100)).toList + val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + + // Second chunk of data + val records2 = DataSourceTestUtils.convertToStringList(dataGen.generateUpdates("001", 100)).toList + val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() + + // define the source of streaming + val streamingInput = + spark.readStream + .schema(inputDF1.schema) + .json(sourcePath) + + val f1 = Future { + println("streaming starting") + //'writeStream' can be called only on streaming Dataset/DataFrame + streamingInput + .writeStream + .format("com.uber.hoodie") + .options(commonOpts) + .trigger(new ProcessingTime(100)) + .option("checkpointLocation", basePath + "/checkpoint") + .outputMode(OutputMode.Append) + .start(destPath) + .awaitTermination(10000) + println("streaming ends") + } + + val f2 = Future { + inputDF1.write.mode(SaveMode.Append).json(sourcePath) + // wait for spark streaming to process one microbatch + Thread.sleep(3000) + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000")) + val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, destPath) + // Read RO View + val hoodieROViewDF1 = spark.read.format("com.uber.hoodie") + .load(destPath + "/*/*/*/*") + assert(hoodieROViewDF1.count() == 100) + + inputDF2.write.mode(SaveMode.Append).json(sourcePath) + // wait for spark streaming to process one microbatch + Thread.sleep(3000) + val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, destPath) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size()) + // Read RO View + val hoodieROViewDF2 = spark.read.format("com.uber.hoodie") + .load(destPath + "/*/*/*/*") + assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated + + + // Read Incremental View + // we have 2 commits, try pulling the first commit (which is not the latest) + val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0) + val hoodieIncViewDF1 = spark.read.format("com.uber.hoodie") + .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit) + .load(destPath) + assertEquals(100, hoodieIncViewDF1.count()) + // 100 initial inserts must be pulled + var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect() + assertEquals(1, countsPerCommit.length) + assertEquals(firstCommit, countsPerCommit(0).get(0)) + + // pull the latest commit + val hoodieIncViewDF2 = spark.read.format("com.uber.hoodie") + .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .load(destPath) + + assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled + countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect() + assertEquals(1, countsPerCommit.length) + assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + } + + Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf) + + } } From 27789b9a8cfc8871b737d637b5939cdbe2b630ab Mon Sep 17 00:00:00 2001 From: xubo245 Date: Tue, 11 Dec 2018 09:16:37 +0800 Subject: [PATCH 133/374] fix some spell errorin Hudi --- .../bloom/HoodieBloomIndexCheckFunction.java | 20 +++++++++---------- .../TestHoodieCompactionStrategy.java | 4 ++-- .../common/model/HoodieRecordLocation.java | 2 +- .../com/uber/hoodie/hive/util/SchemaUtil.java | 4 ++-- .../test/scala/DataSourceDefaultsTest.scala | 4 ++-- 5 files changed, 17 insertions(+), 17 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index 9c2519cb0de1f..0c2ffa6a94234 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -38,7 +38,7 @@ import scala.Tuple2; /** - * Function performing actual checking of RDD parition containing (fileId, hoodieKeys) against the + * Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the * actual files */ public class HoodieBloomIndexCheckFunction implements @@ -82,9 +82,9 @@ public static List checkCandidatesAgainstFile(Configuration configuratio @Override public Iterator> call(Integer partition, - Iterator>> fileParitionRecordKeyTripletItr) + Iterator>> filePartitionRecordKeyTripletItr) throws Exception { - return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); + return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr); } class LazyKeyCheckIterator extends @@ -96,15 +96,15 @@ class LazyKeyCheckIterator extends private String currentFile; - private String currentParitionPath; + private String currentPartitionPath; LazyKeyCheckIterator( - Iterator>> fileParitionRecordKeyTripletItr) { - super(fileParitionRecordKeyTripletItr); + Iterator>> filePartitionRecordKeyTripletItr) { + super(filePartitionRecordKeyTripletItr); currentFile = null; candidateRecordKeys = new ArrayList<>(); bloomFilter = null; - currentParitionPath = null; + currentPartitionPath = null; } @Override @@ -118,7 +118,7 @@ private void initState(String fileName, String partitionPath) throws HoodieIndex .readBloomFilterFromParquetMetadata(metaClient.getHadoopConf(), filePath); candidateRecordKeys = new ArrayList<>(); currentFile = fileName; - currentParitionPath = partitionPath; + currentPartitionPath = partitionPath; } catch (Exception e) { throw new HoodieIndexException("Error checking candidate keys against file.", e); } @@ -154,7 +154,7 @@ protected List computeNext() { } } else { // do the actual checking of file & break out - Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile); + Path filePath = new Path(basePath + "/" + currentPartitionPath + "/" + currentFile); logger.info( "#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys .size() + " for " + filePath); @@ -178,7 +178,7 @@ protected List computeNext() { // handle case, where we ran out of input, close pending work, update return val if (!inputItr.hasNext()) { - Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile); + Path filePath = new Path(basePath + "/" + currentPartitionPath + "/" + currentFile); logger.info( "#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys .size() + " for " + filePath); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index b041546cf2f8c..e3d96b292e72f 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -143,10 +143,10 @@ public void testPartitionAwareCompactionSimple() { private List createCompactionOperations(HoodieWriteConfig config, Map> sizesMap) { - Map keyToParitionMap = sizesMap.entrySet().stream().map(e -> + Map keyToPartitionMap = sizesMap.entrySet().stream().map(e -> Pair.of(e.getKey(), partitionPaths[new Random().nextInt(partitionPaths.length - 1)])) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - return createCompactionOperations(config, sizesMap, keyToParitionMap); + return createCompactionOperations(config, sizesMap, keyToPartitionMap); } private List createCompactionOperations(HoodieWriteConfig config, diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java index 51edf49513b24..04249a9aa1eb8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java @@ -20,7 +20,7 @@ import java.io.Serializable; /** - * Location of a HoodieRecord within the parition it belongs to. Ultimately, this points to an + * Location of a HoodieRecord within the partition it belongs to. Ultimately, this points to an * actual file on disk */ public class HoodieRecordLocation implements Serializable { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index c410273bd7bf1..00337c5b93925 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -410,12 +410,12 @@ public static String generateCreateDDL(MessageType storageSchema, HiveSyncConfig .append(getPartitionKeyType(hiveSchema, partitionKey)).toString()); } - String paritionsStr = partitionFields.stream().collect(Collectors.joining(",")); + String partitionsStr = partitionFields.stream().collect(Collectors.joining(",")); StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS "); sb = sb.append(config.databaseName).append(".").append(config.tableName); sb = sb.append("( ").append(columns).append(")"); if (!config.partitionFields.isEmpty()) { - sb = sb.append(" PARTITIONED BY (").append(paritionsStr).append(")"); + sb = sb.append(" PARTITIONED BY (").append(partitionsStr).append(")"); } sb = sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'"); sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'"); diff --git a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala index e136545da6d4e..2abf50c0e2a22 100644 --- a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala @@ -38,10 +38,10 @@ class DataSourceDefaultsTest extends AssertionsForJUnit { } - private def getKeyConfig(recordKeyFieldName: String, paritionPathField: String): TypedProperties = { + private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String): TypedProperties = { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName) - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, paritionPathField) + props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionPathField) props } From cf2269b43c329c905be6e37a77ff3ca6b33ac0e7 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 5 Dec 2018 11:42:38 -0800 Subject: [PATCH 134/374] Ensure Hoodie works for non-partitioned Hive table --- .../uber/hoodie/io/HoodieCreateHandle.java | 2 +- .../com/uber/hoodie/io/HoodieIOHandle.java | 2 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 4 +- .../table/view/HoodieTableFileSystemView.java | 2 +- .../com/uber/hoodie/common/util/FSUtils.java | 10 ++++- .../com/uber/hoodie/hive/HiveSyncConfig.java | 2 +- .../uber/hoodie/hive/HoodieHiveClient.java | 18 ++++---- .../hoodie/hive/NonPartitionedExtractor.java | 31 ++++++++++++++ .../com/uber/hoodie/hive/util/SchemaUtil.java | 3 +- .../uber/hoodie/integ/ITTestHoodieSanity.java | 36 +++++++++++++--- .../hoodie/NonpartitionedKeyGenerator.java | 41 +++++++++++++++++++ .../uber/hoodie/HoodieSparkSqlWriter.scala | 3 +- hoodie-spark/src/test/java/HoodieJavaApp.java | 29 +++++++++++-- 13 files changed, 157 insertions(+), 26 deletions(-) create mode 100644 hoodie-hive/src/main/java/com/uber/hoodie/hive/NonPartitionedExtractor.java create mode 100644 hoodie-spark/src/main/java/com/uber/hoodie/NonpartitionedKeyGenerator.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index c2688b8f0362f..c82f8a2f2fb42 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -68,7 +68,7 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTab try { HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, - new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath)); + new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath)); partitionMetadata.trySave(TaskContext.getPartitionId()); this.storageWriter = HoodieStorageWriterFactory .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 3e3318161ceb4..07bc368e44332 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -85,7 +85,7 @@ public static Schema createHoodieWriteSchema(HoodieWriteConfig config) { } public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) { - Path path = new Path(config.getBasePath(), partitionPath); + Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath); try { fs.mkdirs(path); // create a new partition as needed. } catch (IOException e) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 580975f922ca7..bf575d416913d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -101,12 +101,12 @@ private void init(String fileId, String partitionPath, Optional writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, - new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath)); + new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath)); partitionMetadata.trySave(TaskContext.getPartitionId()); oldFilePath = new Path( config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath); - String relativePath = new Path(partitionPath + "/" + FSUtils + String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/") + FSUtils .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); newFilePath = new Path(config.getBasePath(), relativePath); if (config.shouldUseTempFolderForCopyOnWriteForMerge()) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index 7e7792d3dd1f1..91520b7252ba5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -426,7 +426,7 @@ public Stream getAllFileGroups(String partitionPathStr) { try { // Create the path if it does not exist already - Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); + Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr); FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); List fileGroups = addFilesToView(statuses); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index f71f6b4cc84c6..ba9cef9edd567 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -147,7 +147,9 @@ public static List getAllFoldersThreeLevelsDown(FileSystem fs, String ba public static String getRelativePartitionPath(Path basePath, Path partitionPath) { String partitionFullPath = partitionPath.toString(); int partitionStartIndex = partitionFullPath.lastIndexOf(basePath.getName()); - return partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1); + // Partition-Path could be empty for non-partitioned tables + return partitionStartIndex + basePath.getName().length() == partitionFullPath.length() ? "" : + partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1); } /** @@ -396,4 +398,10 @@ public static void createPathIfNotExists(FileSystem fs, Path partitionPath) thro public static Long getSizeInMB(long sizeInBytes) { return sizeInBytes / (1024 * 1024); } + + public static Path getPartitionPath(String basePath, String partitionPath) { + // FOr non-partitioned table, return only base-path + return ((partitionPath == null) || (partitionPath.isEmpty())) ? new Path(basePath) : + new Path(basePath, partitionPath); + } } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java index d040a5e79abdf..14b01ffecc6cb 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java @@ -49,7 +49,7 @@ public class HiveSyncConfig implements Serializable { public String basePath; @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by", - required = true) + required = false) public List partitionFields = new ArrayList<>(); @Parameter(names = "-partition-value-extractor", description = "Class which implements " diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index de7c7aed35f15..e3f9cb0feed76 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -154,7 +154,7 @@ private String constructAddPartitions(List partitions) { .append(" ADD IF NOT EXISTS "); for (String partition : partitions) { String partitionClause = getPartitionClause(partition); - String fullPartitionPath = new Path(syncConfig.basePath, partition).toString(); + String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString(); alterSQL.append(" PARTITION (").append(partitionClause).append(") LOCATION '") .append(fullPartitionPath).append("' "); } @@ -185,7 +185,7 @@ private List constructChangePartitions(List partitions) { String alterTable = "ALTER TABLE " + syncConfig.databaseName + "." + syncConfig.tableName; for (String partition : partitions) { String partitionClause = getPartitionClause(partition); - String fullPartitionPath = new Path(syncConfig.basePath, partition).toString(); + String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString(); String changePartition = alterTable + " PARTITION (" + partitionClause + ") SET LOCATION '" + fullPartitionPath + "'"; changePartitions.add(changePartition); @@ -210,16 +210,18 @@ List getPartitionEvents(List tablePartitions, List events = Lists.newArrayList(); for (String storagePartition : partitionStoragePartitions) { - String fullStoragePartitionPath = new Path(syncConfig.basePath, storagePartition).toString(); + String fullStoragePartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, storagePartition).toString(); // Check if the partition values or if hdfs path is the same List storagePartitionValues = partitionValueExtractor .extractPartitionValuesInPath(storagePartition); Collections.sort(storagePartitionValues); - String storageValue = String.join(", ", storagePartitionValues); - if (!paths.containsKey(storageValue)) { - events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); - } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { - events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + if (!storagePartitionValues.isEmpty()) { + String storageValue = String.join(", ", storagePartitionValues); + if (!paths.containsKey(storageValue)) { + events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); + } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { + events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + } } } return events; diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/NonPartitionedExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/NonPartitionedExtractor.java new file mode 100644 index 0000000000000..ce610360d6c2e --- /dev/null +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/NonPartitionedExtractor.java @@ -0,0 +1,31 @@ +/* + * 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.hive; + +import java.util.ArrayList; +import java.util.List; + +/** + * Extractor for Non-partitioned hive tables + */ +public class NonPartitionedExtractor implements PartitionValueExtractor { + + @Override + public List extractPartitionValuesInPath(String partitionPath) { + return new ArrayList<>(); + } +} diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index 00337c5b93925..6ed51b732f85d 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -406,8 +406,9 @@ public static String generateCreateDDL(MessageType storageSchema, HiveSyncConfig List partitionFields = new ArrayList<>(); for (String partitionKey : config.partitionFields) { + String partitionKeyWithTicks = tickSurround(partitionKey); partitionFields.add(new StringBuilder().append(partitionKey).append(" ") - .append(getPartitionKeyType(hiveSchema, partitionKey)).toString()); + .append(getPartitionKeyType(hiveSchema, partitionKeyWithTicks)).toString()); } String partitionsStr = partitionFields.stream().collect(Collectors.joining(",")); diff --git a/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java index 9a1b694ff33ac..6dc48515fa7e3 100644 --- a/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java +++ b/hoodie-integ-test/src/test/java/com/uber/hoodie/integ/ITTestHoodieSanity.java @@ -25,6 +25,12 @@ */ public class ITTestHoodieSanity extends ITTestBase { + enum PartitionType { + SINGLE_KEY_PARTITIONED, + MULTI_KEYS_PARTITIONED, + NON_PARTITIONED, + } + @Test public void testRunEcho() throws Exception { String[] cmd = new String[]{"echo", "Happy Testing"}; @@ -44,7 +50,7 @@ public void testRunEcho() throws Exception { */ public void testRunHoodieJavaAppOnSinglePartitionKeyCOWTable() throws Exception { String hiveTableName = "docker_hoodie_single_partition_key_cow_test"; - testRunHoodieJavaAppOnCOWTable(hiveTableName, true); + testRunHoodieJavaAppOnCOWTable(hiveTableName, PartitionType.SINGLE_KEY_PARTITIONED); } @Test @@ -55,7 +61,18 @@ public void testRunHoodieJavaAppOnSinglePartitionKeyCOWTable() throws Exception */ public void testRunHoodieJavaAppOnMultiPartitionKeysCOWTable() throws Exception { String hiveTableName = "docker_hoodie_multi_partition_key_cow_test"; - testRunHoodieJavaAppOnCOWTable(hiveTableName, false); + testRunHoodieJavaAppOnCOWTable(hiveTableName, PartitionType.MULTI_KEYS_PARTITIONED); + } + + @Test + /** + * A basic integration test that runs HoodieJavaApp to create a sample non-partitioned COW Hoodie + * data-set and performs upserts on it. Hive integration and upsert functionality is checked by running a count + * query in hive console. + */ + public void testRunHoodieJavaAppOnNonPartitionedCOWTable() throws Exception { + String hiveTableName = "docker_hoodie_non_partition_key_cow_test"; + testRunHoodieJavaAppOnCOWTable(hiveTableName, PartitionType.NON_PARTITIONED); } /** @@ -64,7 +81,7 @@ public void testRunHoodieJavaAppOnMultiPartitionKeysCOWTable() throws Exception * query in hive console. * TODO: Add spark-shell test-case */ - public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singlePartitionKey) throws Exception { + public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, PartitionType partitionType) throws Exception { String hdfsPath = "/" + hiveTableName; String hdfsUrl = "hdfs://namenode" + hdfsPath; @@ -90,7 +107,7 @@ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singleP // Run Hoodie Java App { String[] cmd = null; - if (singlePartitionKey) { + if (partitionType == PartitionType.SINGLE_KEY_PARTITIONED) { cmd = new String[]{ HOODIE_JAVA_APP, "--hive-sync", @@ -98,7 +115,7 @@ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singleP "--hive-url", HIVE_SERVER_JDBC_URL, "--hive-table", hiveTableName }; - } else { + } else if (partitionType == PartitionType.MULTI_KEYS_PARTITIONED) { cmd = new String[]{ HOODIE_JAVA_APP, "--hive-sync", @@ -107,6 +124,15 @@ public void testRunHoodieJavaAppOnCOWTable(String hiveTableName, boolean singleP "--use-multi-partition-keys", "--hive-table", hiveTableName }; + } else { + cmd = new String[]{ + HOODIE_JAVA_APP, + "--hive-sync", + "--table-path", hdfsUrl, + "--hive-url", HIVE_SERVER_JDBC_URL, + "--non-partitioned", + "--hive-table", hiveTableName + }; } TestExecStartResultCallback callback = executeCommandInDocker(ADHOC_1_CONTAINER, cmd, true); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/NonpartitionedKeyGenerator.java b/hoodie-spark/src/main/java/com/uber/hoodie/NonpartitionedKeyGenerator.java new file mode 100644 index 0000000000000..59ca5d453404b --- /dev/null +++ b/hoodie-spark/src/main/java/com/uber/hoodie/NonpartitionedKeyGenerator.java @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie; + +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.util.TypedProperties; +import org.apache.avro.generic.GenericRecord; + +/** + * Simple Key generator for unpartitioned Hive Tables + */ +public class NonpartitionedKeyGenerator extends SimpleKeyGenerator { + + private static final String EMPTY_PARTITION = ""; + + public NonpartitionedKeyGenerator(TypedProperties props) { + super(props); + } + + @Override + public HoodieKey getKey(GenericRecord record) { + String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField); + return new HoodieKey(recordKey, EMPTY_PARTITION); + } +} diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala index 93ad54878c86f..fe3507e081269 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala @@ -258,9 +258,8 @@ private[hoodie] object HoodieSparkSqlWriter { hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY) hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY) hiveSyncConfig.partitionFields = - ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).toList: _*) + ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*) hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) hiveSyncConfig } - } \ No newline at end of file diff --git a/hoodie-spark/src/test/java/HoodieJavaApp.java b/hoodie-spark/src/test/java/HoodieJavaApp.java index 3cdc55136c5a1..f6f17f73fabc8 100644 --- a/hoodie-spark/src/test/java/HoodieJavaApp.java +++ b/hoodie-spark/src/test/java/HoodieJavaApp.java @@ -21,10 +21,13 @@ import com.uber.hoodie.DataSourceReadOptions; import com.uber.hoodie.DataSourceWriteOptions; import com.uber.hoodie.HoodieDataSourceHelpers; +import com.uber.hoodie.NonpartitionedKeyGenerator; +import com.uber.hoodie.SimpleKeyGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.hive.MultiPartKeysValueExtractor; +import com.uber.hoodie.hive.NonPartitionedExtractor; import java.util.List; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; @@ -68,6 +71,9 @@ public class HoodieJavaApp { @Parameter(names = {"--hive-url", "-hl"}, description = "hive JDBC URL") private String hiveJdbcUrl = "jdbc:hive2://localhost:10000"; + @Parameter(names = {"--non-partitioned", "-np"}, description = "Use non-partitioned Table") + private Boolean nonPartitionedTable = false; + @Parameter(names = {"--use-multi-partition-keys", "-mp"}, description = "Use Multiple Partition Keys") private Boolean useMultiPartitionKeys = false; @@ -98,7 +104,13 @@ public void run() throws Exception { FileSystem fs = FileSystem.get(jssc.hadoopConfiguration()); // Generator of some records to be loaded in. - HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + HoodieTestDataGenerator dataGen = null; + if (nonPartitionedTable) { + // All data goes to base-path + dataGen = new HoodieTestDataGenerator(new String[]{""}); + } else { + dataGen = new HoodieTestDataGenerator(); + } /** * Commit with only inserts @@ -124,6 +136,9 @@ public void run() throws Exception { .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") // use to combine duplicate records in input/with disk val .option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries + .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), + nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName() : + SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor .mode( SaveMode.Overwrite); // This will remove any existing data at path below, and create a @@ -145,7 +160,11 @@ public void run() throws Exception { .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") + .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), + nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName() : + SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor .option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append); + updateHiveSyncConfig(writer); writer.save(tablePath); String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); @@ -157,7 +176,7 @@ public void run() throws Exception { Dataset hoodieROViewDF = spark.read().format("com.uber.hoodie") // pass any path glob, can include hoodie & non-hoodie // datasets - .load(tablePath + "/*/*/*/*"); + .load(tablePath + (nonPartitionedTable ? "/*" : "/*/*/*/*")); hoodieROViewDF.registerTempTable("hoodie_ro"); spark.sql("describe hoodie_ro").show(); // all trips whose fare was greater than 2. @@ -195,7 +214,11 @@ private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { .option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser) .option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass) .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true"); - if (useMultiPartitionKeys) { + if (nonPartitionedTable) { + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(), + NonPartitionedExtractor.class.getCanonicalName()) + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), ""); + } else if (useMultiPartitionKeys) { writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day") .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(), MultiPartKeysValueExtractor.class.getCanonicalName()); From e13dd8c68451d60b84eb9e045d22310abe26f491 Mon Sep 17 00:00:00 2001 From: arukavytsia Date: Wed, 12 Dec 2018 03:19:43 +0200 Subject: [PATCH 135/374] General enhancements --- .../hoodie/cli/commands/StatsCommand.java | 5 +- .../com/uber/hoodie/cli/DedupeSparkJob.scala | 42 ++-- .../com/uber/hoodie/cli/SparkHelpers.scala | 14 +- .../uber/hoodie/CompactionAdminClient.java | 15 +- .../com/uber/hoodie/HoodieWriteClient.java | 20 +- .../hoodie/config/HoodieMemoryConfig.java | 5 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 10 +- .../index/bloom/HoodieGlobalBloomIndex.java | 2 +- .../com/uber/hoodie/io/ConsistencyCheck.java | 2 +- .../uber/hoodie/io/HoodieAppendHandle.java | 4 +- .../com/uber/hoodie/io/HoodieCleanHelper.java | 7 +- .../hoodie/io/HoodieCommitArchiveLog.java | 11 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 4 +- .../compact/HoodieRealtimeTableCompactor.java | 5 +- .../compact/strategy/CompactionStrategy.java | 2 +- .../io/storage/HoodieWrapperFileSystem.java | 9 +- .../uber/hoodie/metrics/HoodieMetrics.java | 9 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 10 +- .../hoodie/table/HoodieMergeOnReadTable.java | 203 ++++++++---------- .../hoodie/common/model/HoodieFileGroup.java | 16 +- .../hoodie/common/model/HoodieLogFile.java | 2 +- .../hoodie/common/storage/StorageSchemes.java | 5 +- .../common/table/HoodieTableConfig.java | 13 +- .../table/log/block/HoodieCorruptBlock.java | 2 +- .../table/view/HoodieTableFileSystemView.java | 77 +++---- .../uber/hoodie/common/util/ParquetUtils.java | 18 +- .../hoodie/common/util/ReflectionUtils.java | 2 +- .../AbstractRealtimeRecordReader.java | 6 +- .../com/uber/hoodie/AvroConversionUtils.scala | 2 +- .../com/uber/hoodie/DataSourceOptions.scala | 2 +- .../com/uber/hoodie/IncrementalRelation.scala | 4 +- .../utilities/HoodieSnapshotCopier.java | 9 +- .../uber/hoodie/utilities/UtilHelpers.java | 8 +- .../keygen/TimestampBasedKeyGenerator.java | 3 +- .../schema/FilebasedSchemaProvider.java | 5 +- .../schema/SchemaRegistryProvider.java | 5 +- .../utilities/sources/AvroConvertor.java | 2 +- .../utilities/sources/AvroKafkaSource.java | 3 +- .../hoodie/utilities/sources/DFSSource.java | 18 +- .../utilities/sources/HiveIncrPullSource.java | 4 +- .../utilities/sources/JsonDFSSource.java | 2 +- .../utilities/sources/JsonKafkaSource.java | 2 +- .../hoodie/utilities/sources/KafkaSource.java | 54 ++--- .../utilities/sources/TestDataSource.java | 2 +- pom.xml | 28 ++- style/scalastyle-config.xml | 98 +++++++++ 46 files changed, 400 insertions(+), 371 deletions(-) create mode 100644 style/scalastyle-config.xml diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java index 212e2d2773720..cc139315d048d 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java @@ -143,9 +143,8 @@ public String fileSizeStats( Snapshot s = globalHistogram.getSnapshot(); rows.add(printFileSizeHistogram("ALL", s)); - Function converterFunction = entry -> { - return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString()))); - }; + Function converterFunction = entry -> + NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString()))); Map> fieldNameToConverterMap = new HashMap<>(); fieldNameToConverterMap.put("Min", converterFunction); fieldNameToConverterMap.put("10th", converterFunction); diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala index 0058043ed6388..1e6f003dba566 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala @@ -77,7 +77,7 @@ class DedupeSparkJob(basePath: String, val metadata = new HoodieTableMetaClient(fs.getConf, basePath) - val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}")) + val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]()) val filteredStatuses = latestFiles.map(f => f.getPath) @@ -92,8 +92,8 @@ class DedupeSparkJob(basePath: String, val dupeDataSql = s""" SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time` - FROM ${tmpTableName} h - JOIN ${dedupeTblName} d + FROM $tmpTableName h + JOIN $dedupeTblName d ON h.`_hoodie_record_key` = d.dupe_key """ val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0)) @@ -101,8 +101,7 @@ class DedupeSparkJob(basePath: String, // Mark all files except the one with latest commits for deletion dupeMap.foreach(rt => { - val key = rt._1 - val rows = rt._2 + val (key, rows) = rt var maxCommit = -1L rows.foreach(r => { @@ -129,7 +128,7 @@ class DedupeSparkJob(basePath: String, def fixDuplicates(dryRun: Boolean = true) = { val metadata = new HoodieTableMetaClient(fs.getConf, basePath) - val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}")) + val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]()) @@ -138,30 +137,28 @@ class DedupeSparkJob(basePath: String, val dupeFixPlan = planDuplicateFix() // 1. Copy all latest files into the temp fix path - fileNameToPathMap.foreach { case (fileName, filePath) => { + fileNameToPathMap.foreach { case (fileName, filePath) => val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else "" - val dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}") - LOG.info(s"Copying from ${filePath} to ${dstPath}") + val dstPath = new Path(s"$repairOutputPath/${filePath.getName}$badSuffix") + LOG.info(s"Copying from $filePath to $dstPath") FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf) } - } // 2. Remove duplicates from the bad files - dupeFixPlan.foreach { case (fileName, keysToSkip) => { + dupeFixPlan.foreach { case (fileName, keysToSkip) => val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName) - val badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad") - val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}") + val badFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}.bad") + val newFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}") LOG.info(" Skipping and writing new file for : " + fileName) SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName)) fs.delete(badFilePath, false) } - } // 3. Check that there are no duplicates anymore. - val df = sqlContext.read.parquet(s"${repairOutputPath}/*.parquet") + val df = sqlContext.read.parquet(s"$repairOutputPath/*.parquet") df.registerTempTable("fixedTbl") val dupeKeyDF = getDupeKeyDF("fixedTbl") - val dupeCnt = dupeKeyDF.count(); + val dupeCnt = dupeKeyDF.count() if (dupeCnt != 0) { dupeKeyDF.show() throw new HoodieException("Still found some duplicates!!.. Inspect output") @@ -169,7 +166,7 @@ class DedupeSparkJob(basePath: String, // 4. Additionally ensure no record keys are left behind. val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList) - val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"${repairOutputPath}/${t._2.getName}").toList) + val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"$repairOutputPath/${t._2.getName}").toList) val missedRecordKeysDF = sourceDF.except(fixedDF) val missedCnt = missedRecordKeysDF.count() if (missedCnt != 0) { @@ -180,17 +177,16 @@ class DedupeSparkJob(basePath: String, println("No duplicates found & counts are in check!!!! ") // 4. Prepare to copy the fixed files back. - fileNameToPathMap.foreach { case (fileName, filePath) => { - val srcPath = new Path(s"${repairOutputPath}/${filePath.getName}") - val dstPath = new Path(s"${basePath}/${duplicatedPartitionPath}/${filePath.getName}") + fileNameToPathMap.foreach { case (_, filePath) => + val srcPath = new Path(s"$repairOutputPath/${filePath.getName}") + val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}") if (dryRun) { - LOG.info(s"[JUST KIDDING!!!] Copying from ${srcPath} to ${dstPath}") + LOG.info(s"[JUST KIDDING!!!] Copying from $srcPath to $dstPath") } else { // for real - LOG.info(s"[FOR REAL!!!] Copying from ${srcPath} to ${dstPath}") + LOG.info(s"[FOR REAL!!!] Copying from $srcPath to $dstPath") FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf) } } - } } } diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala index f383e5fba4fae..81db77053274d 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala @@ -94,12 +94,9 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) { * @return */ def isFileContainsKey(rowKey: String, file: String): Boolean = { - println(s"Checking ${file} for key ${rowKey}") - val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '${rowKey}'") - if (ff.count() > 0) - return true - else - return false + println(s"Checking $file for key $rowKey") + val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '$rowKey'") + if (ff.count() > 0) true else false } /** @@ -109,7 +106,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) { * @param sqlContext */ def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = { - println(getRowKeyDF(file).collect().size) + println(getRowKeyDF(file).collect().length) } @@ -128,8 +125,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) { val bf = new com.uber.hoodie.common.BloomFilter(bfStr) val foundCount = sqlContext.parquetFile(file) .select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") - .collect(). - filter(r => !bf.mightContain(r.getString(0))).size + .collect().count(r => !bf.mightContain(r.getString(0))) val totalCount = getKeyCount(file, sqlContext) s"totalCount: ${totalCount}, foundCount: ${foundCount}" totalCount == foundCount diff --git a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java index 2f6495f0540bf..6a43c9c88da53 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java @@ -238,8 +238,8 @@ protected static List> getRenamingActionsToAl fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp()) .filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get(); final int maxVersion = - op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf))) - .reduce((x, y) -> x > y ? x : y).map(x -> x).orElse(0); + op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf))) + .reduce((x, y) -> x > y ? x : y).orElse(0); List logFilesToBeMoved = merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList()); return logFilesToBeMoved.stream().map(lf -> { @@ -322,8 +322,7 @@ private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient met Set diff = logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf)) .collect(Collectors.toSet()); - Preconditions.checkArgument(diff.stream() - .filter(lf -> !lf.getBaseCommitTime().equals(compactionInstant)).count() == 0, + Preconditions.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)), "There are some log-files which are neither specified in compaction plan " + "nor present after compaction request instant. Some of these :" + diff); } else { @@ -438,14 +437,14 @@ public List> getRenamingActionsForUnschedulin fileSystemView.getLatestMergedFileSlicesBeforeOrOn(operation.getPartitionPath(), lastInstant.getTimestamp()) .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get(); List logFilesToRepair = - merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant)) - .collect(Collectors.toList()); - logFilesToRepair.sort(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed()); + merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant)) + .sorted(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed()) + .collect(Collectors.toList()); FileSlice fileSliceForCompaction = fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime()) .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get(); int maxUsedVersion = - fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getLogVersion()) + fileSliceForCompaction.getLogFiles().findFirst().map(HoodieLogFile::getLogVersion) .orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1); String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension()) .orElse(HoodieLogFile.DELTA_EXTENSION); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 7389cf6e0b3de..3693b55bf929a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -440,7 +440,7 @@ private JavaRDD upsertRecordsInternal(JavaRDD> prep } else { return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner); } - }, true).flatMap(writeStatuses -> writeStatuses.iterator()); + }, true).flatMap(List::iterator); return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime); } @@ -469,7 +469,7 @@ private JavaRDD> partition(JavaRDD> dedupedRecor Partitioner partitioner) { return dedupedRecords.mapToPair(record -> new Tuple2<>( new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record)) - .partitionBy(partitioner).map(tuple -> tuple._2()); + .partitionBy(partitioner).map(Tuple2::_2); } /** @@ -499,7 +499,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - List stats = writeStatuses.map(status -> status.getStat()).collect(); + List stats = writeStatuses.map(WriteStatus::getStat).collect(); updateMetadataAndRollingStats(actionType, metadata, stats); @@ -522,7 +522,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, // add in extra metadata if (extraMetadata.isPresent()) { - extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v)); + extraMetadata.get().forEach(metadata::addMetadata); } try { @@ -806,7 +806,7 @@ private void rollback(List commits) { }); List pendingCompactionToRollback = - commits.stream().filter(c -> pendingCompactions.contains(c)).collect(Collectors.toList()); + commits.stream().filter(pendingCompactions::contains).collect(Collectors.toList()); List commitsToRollback = commits.stream().filter(c -> !pendingCompactions.contains(c)).collect(Collectors.toList()); @@ -837,12 +837,12 @@ private void rollback(List commits) { } // Remove interleaving pending compactions before rolling back commits - pendingCompactionToRollback.stream().forEach(this::deletePendingCompaction); + pendingCompactionToRollback.forEach(this::deletePendingCompaction); List stats = table.rollback(jsc, commitsToRollback); // cleanup index entries - commitsToRollback.stream().forEach(s -> { + commitsToRollback.forEach(s -> { if (!index.rollbackCommit(s)) { throw new HoodieRollbackException("Rollback index changes failed, for time :" + s); } @@ -1076,7 +1076,7 @@ JavaRDD> deduplicateRecords(JavaRDD> records, // everything // so pick it from one of the records. return new HoodieRecord(rec1.getKey(), reducedData); - }, parallelism).map(recordTuple -> recordTuple._2()); + }, parallelism).map(Tuple2::_2); } /** @@ -1099,7 +1099,7 @@ private HoodieTable getTableAndInitCtx() { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { + if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) { writeContext = metrics.getCommitCtx(); } else { writeContext = metrics.getDeltaCommitCtx(); @@ -1214,7 +1214,7 @@ private void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTab private HoodieCommitMetadata doCompactionCommit(JavaRDD writeStatuses, HoodieTableMetaClient metaClient, String compactionCommitTime, Optional> extraMetadata) { - List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) + List updateStatusMap = writeStatuses.map(WriteStatus::getStat) .collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java index 6efa70b0d4ab6..5746e8d167253 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java @@ -117,10 +117,9 @@ private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) { if (SparkEnv.get() != null) { // 1 GB is the default conf used by Spark, look at SparkContext.scala - long executorMemoryInBytes = Long.valueOf( - Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP, + long executorMemoryInBytes = Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 - * 1024L); + * 1024L; // 0.6 is the default value used by Spark, // look at {@link // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 050958159035a..fcbd90f0b3d22 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -187,13 +187,11 @@ private int autoComputeParallelism(final Map recordsPerPartition, for (String partitionPath : recordsPerPartition.keySet()) { long numRecords = recordsPerPartition.get(partitionPath); long numFiles = - filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) - : 1L; + filesPerPartition.getOrDefault(partitionPath, 1L); totalComparisons += numFiles * numRecords; totalFiles += - filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) - : 0L; + filesPerPartition.getOrDefault(partitionPath, 0L); totalRecords += numRecords; } logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles @@ -340,7 +338,7 @@ JavaPairRDD> explodeRecordRDDWithFileCompariso } } return recordComparisons; - }).flatMapToPair(t -> t.iterator()); + }).flatMapToPair(List::iterator); } /** @@ -369,7 +367,7 @@ JavaPairRDD findMatchingFilesForRecordKeys( return fileSortedTripletRDD.mapPartitionsWithIndex( new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true) - .flatMap(indexLookupResults -> indexLookupResults.iterator()) + .flatMap(List::iterator) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { List> vals = new ArrayList<>(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java index 49c5fd3fc1053..db1f76f328676 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java @@ -104,7 +104,7 @@ JavaPairRDD> explodeRecordRDDWithFileCompariso } } return recordComparisons; - }).flatMapToPair(t -> t.iterator()); + }).flatMapToPair(List::iterator); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java b/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java index 43913089e73d5..0171e07b83a1a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/ConsistencyCheck.java @@ -97,7 +97,7 @@ public List check(int maxAttempts, long initalDelayMs) { .filter(p -> !fileNames.contains(new Path(basePath, p).getName())) .collect(Collectors.toList()); }) - .flatMap(itr -> itr.iterator()).collect(); + .flatMap(List::iterator).collect(); if (remainingPaths.size() == 0) { break; // we are done. } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 5434c576630d6..452d111eb56ea 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -65,9 +65,9 @@ public class HoodieAppendHandle extends HoodieIOH private final WriteStatus writeStatus; private final String fileId; // Buffer for holding records in memory before they are flushed to disk - List recordList = new ArrayList<>(); + private List recordList = new ArrayList<>(); // Buffer for holding records (to be deleted) in memory before they are flushed to disk - List keysToDelete = new ArrayList<>(); + private List keysToDelete = new ArrayList<>(); private TableFileSystemView.RealtimeView fileSystemView; private String partitionPath; private Iterator> recordItr; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index 0d991642d7b4b..537bbedaf89be 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -168,8 +168,7 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) // The window of commit retain == max query run time. So a query could be running which // still // uses this file. - if (fileCommitTime.equals(lastVersion) || (lastVersionBeforeEarliestCommitToRetain != null - && fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) { + if (fileCommitTime.equals(lastVersion) || (fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) { // move on to the next file continue; } @@ -180,9 +179,7 @@ private List getFilesToCleanKeepingLatestCommits(String partitionPath) .compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) { // this is a commit, that should be cleaned. - if (aFile.isPresent()) { - deletePaths.add(aFile.get().getFileStatus().getPath().toString()); - } + aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getFileStatus().getPath().toString())); if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { // If merge on read, then clean the log files for the commits as well deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString()) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index a5dff6b90c145..f8647e5cbd68a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -195,12 +195,11 @@ private boolean deleteArchivedInstants(List archivedInstants) thr // Remove older meta-data from auxiliary path too Optional latestCommitted = - archivedInstants.stream() - .filter(i -> { - return i.isCompleted() - && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); - }) - .sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()).findFirst(); + archivedInstants.stream() + .filter(i -> { + return i.isCompleted() + && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); + }).max(Comparator.comparing(HoodieInstant::getTimestamp)); if (latestCommitted.isPresent()) { success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index bf575d416913d..210f311bcc466 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -254,9 +254,7 @@ public void write(GenericRecord oldRecord) { public WriteStatus close() { try { // write out any pending records (this can happen when inserts are turned into updates) - Iterator pendingRecordsItr = keyToNewRecords.keySet().iterator(); - while (pendingRecordsItr.hasNext()) { - String key = pendingRecordsItr.next(); + for (String key : keyToNewRecords.keySet()) { if (!writtenRecordKeys.contains(key)) { HoodieRecord hoodieRecord = keyToNewRecords.get(key); writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema)); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 4d47e76055a1e..ae74a58f36b23 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -90,7 +90,7 @@ public JavaRDD compact(JavaSparkContext jsc, log.info("Compactor compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) .map(s -> compact(table, metaClient, config, s, compactionInstantTime)) - .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); + .flatMap(List::iterator); } private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient, @@ -141,7 +141,7 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, } Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) - .map(s -> { + .peek(s -> { s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog()); s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); @@ -154,7 +154,6 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks()); s.getStat().setRuntimeStats(runtimeStats); - return s; }).collect(toList()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index fa5db25278b96..d7444818ca3af 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -74,7 +74,7 @@ public Map captureMetrics(HoodieWriteConfig writeConfig, Optiona metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue()); metrics.put(TOTAL_IO_MB, totalIO.doubleValue()); metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); - metrics.put(TOTAL_LOG_FILES, Double.valueOf(logFiles.size())); + metrics.put(TOTAL_LOG_FILES, (double) logFiles.size()); return metrics; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java index 57c32311b381f..5ada7c4cb01b7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java @@ -139,13 +139,8 @@ private FSDataOutputStream wrapOutputStream(final Path path, return fsDataOutputStream; } - SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream, - new Runnable() { - @Override - public void run() { - openStreams.remove(path.getName()); - } - }); + SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream( + fsDataOutputStream, () -> openStreams.remove(path.getName())); openStreams.put(path.getName(), os); return os; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index aa9afa6fc3b06..5adf45fe06e63 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -145,7 +145,7 @@ public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) { if (config.isMetricsOn()) { logger.info(String - .format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", durationInMs, + .format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs, numFilesDeleted)); registerGauge(getMetricsName("rollback", "duration"), durationInMs); registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted); @@ -180,12 +180,7 @@ String getMetricsName(String action, String metric) { void registerGauge(String metricName, final long value) { try { MetricRegistry registry = Metrics.getInstance().getRegistry(); - registry.register(metricName, new Gauge() { - @Override - public Long getValue() { - return value; - } - }); + registry.register(metricName, (Gauge) () -> value); } catch (Exception e) { // Here we catch all exception, so the major upsert pipeline will not be affected if the // metrics system diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 0a7eeaecab6e7..b9d7c18d31059 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -201,17 +201,15 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups "Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId); } else { AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); - ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) - .withConf(getHadoopConf()).build(); BoundedInMemoryExecutor wrapper = null; - try { + try (ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) + .withConf(getHadoopConf()).build()) { wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader), - new UpdateHandler(upsertHandle), x -> x); + new UpdateHandler(upsertHandle), x -> x); wrapper.execute(); } catch (Exception e) { throw new HoodieException(e); } finally { - reader.close(); upsertHandle.close(); if (null != wrapper) { wrapper.shutdownNow(); @@ -480,7 +478,7 @@ private List cleanPartitionPaths(List partitionsToClean .merge(e2)).collect(); Map partitionCleanStatsMap = partitionCleanStats.stream() - .collect(Collectors.toMap(e -> e._1(), e -> e._2())); + .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config); // Return PartitionCleanStat for each partition passed. diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index ad065fbfcadf5..849dbae45caf1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -52,16 +52,7 @@ import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -185,105 +176,103 @@ public List rollback(JavaSparkContext jsc, List comm .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants() .filter(i -> commits.contains(i.getTimestamp())) - .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); + .collect(Collectors.toMap(HoodieInstant::getTimestamp, i -> i)); // Atomically un-publish all non-inflight commits - commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue()) + commitsAndCompactions.entrySet().stream().map(Map.Entry::getValue) .filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight); logger.info("Unpublished " + commits); Long startTime = System.currentTimeMillis(); List allRollbackStats = jsc.parallelize(FSUtils .getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) - .map((Function>) partitionPath -> { - return commits.stream().map(commit -> { - HoodieInstant instant = commitsAndCompactions.get(commit); - HoodieRollbackStat hoodieRollbackStats = null; - // Need to put the path filter here since Filter is not serializable - // PathFilter to get all parquet files and log files that need to be deleted - PathFilter filter = (path) -> { - if (path.toString().contains(".parquet")) { - String fileCommitTime = FSUtils.getCommitTime(path.getName()); - return commits.contains(fileCommitTime); - } else if (path.toString().contains(".log")) { - // Since the baseCommitTime is the only commit for new log files, it's okay here - String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path); - return commits.contains(fileCommitTime); + .map((Function>) partitionPath -> commits.stream().map(commit -> { + HoodieInstant instant = commitsAndCompactions.get(commit); + HoodieRollbackStat hoodieRollbackStats = null; + // Need to put the path filter here since Filter is not serializable + // PathFilter to get all parquet files and log files that need to be deleted + PathFilter filter = (path) -> { + if (path.toString().contains(".parquet")) { + String fileCommitTime = FSUtils.getCommitTime(path.getName()); + return commits.contains(fileCommitTime); + } else if (path.toString().contains(".log")) { + // Since the baseCommitTime is the only commit for new log files, it's okay here + String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path); + return commits.contains(fileCommitTime); + } + return false; + }; + + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.COMPACTION_ACTION: + try { + Map results = super + .deleteCleanedFiles(partitionPath, Collections.singletonList(commit)); + hoodieRollbackStats = HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath).withDeletedFileResults(results).build(); + break; + } catch (IOException io) { + throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } - return false; - }; - - switch (instant.getAction()) { - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.COMPACTION_ACTION: - try { - Map results = super - .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); - hoodieRollbackStats = HoodieRollbackStat.newBuilder() - .withPartitionPath(partitionPath).withDeletedFileResults(results).build(); - break; - } catch (IOException io) { - throw new UncheckedIOException("Failed to rollback for commit " + commit, io); + case HoodieTimeline.DELTA_COMMIT_ACTION: + // -------------------------------------------------------------------------------------------------- + // (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal + // -------------------------------------------------------------------------------------------------- + // (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In + // this scenario we would want to delete these log files. + // (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario, + // HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks. + // (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is + // being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and + // and hence will end up deleting these log files. This is done so there are no orphan log files + // lying around. + // (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions + // taken in this scenario is a combination of (A.2) and (A.3) + // --------------------------------------------------------------------------------------------------- + // (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal + // --------------------------------------------------------------------------------------------------- + // (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries. + // In this scenario, we delete all the parquet files written for the failed commit. + // (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In + // this scenario, perform (A.1) and for updates written to log files, write rollback blocks. + // (B.3) Rollback triggered for first commit - Same as (B.1) + // (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files + // as well if the base parquet file gets deleted. + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + metaClient.getCommitTimeline().getInstantDetails( + new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) + .get(), HoodieCommitMetadata.class); + + // read commit file and (either append delete blocks or delete file) + final Map filesToDeletedStatus = new HashMap<>(); + Map filesToNumBlocksRollback = new HashMap<>(); + + // In case all data was inserts and the commit failed, delete the file belonging to that commit + // We do not know fileIds for inserts (first inserts are either log files or parquet files), + // delete all files for the corresponding failed commit, if present (same as COW) + super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); + final Set deletedFiles = filesToDeletedStatus.entrySet().stream() + .map(entry -> { + Path filePath = entry.getKey().getPath(); + return FSUtils.getFileIdFromFilePath(filePath); + }).collect(Collectors.toSet()); + + // append rollback blocks for updates + if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { + hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus, + filesToNumBlocksRollback, deletedFiles); } - case HoodieTimeline.DELTA_COMMIT_ACTION: - // -------------------------------------------------------------------------------------------------- - // (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal - // -------------------------------------------------------------------------------------------------- - // (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In - // this scenario we would want to delete these log files. - // (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario, - // HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks. - // (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is - // being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and - // and hence will end up deleting these log files. This is done so there are no orphan log files - // lying around. - // (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions - // taken in this scenario is a combination of (A.2) and (A.3) - // --------------------------------------------------------------------------------------------------- - // (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal - // --------------------------------------------------------------------------------------------------- - // (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries. - // In this scenario, we delete all the parquet files written for the failed commit. - // (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In - // this scenario, perform (A.1) and for updates written to log files, write rollback blocks. - // (B.3) Rollback triggered for first commit - Same as (B.1) - // (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files - // as well if the base parquet file gets deleted. - try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - metaClient.getCommitTimeline().getInstantDetails( - new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) - .get(), HoodieCommitMetadata.class); - - // read commit file and (either append delete blocks or delete file) - final Map filesToDeletedStatus = new HashMap<>(); - Map filesToNumBlocksRollback = new HashMap<>(); - - // In case all data was inserts and the commit failed, delete the file belonging to that commit - // We do not know fileIds for inserts (first inserts are either log files or parquet files), - // delete all files for the corresponding failed commit, if present (same as COW) - super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter); - final Set deletedFiles = filesToDeletedStatus.entrySet().stream() - .map(entry -> { - Path filePath = entry.getKey().getPath(); - return FSUtils.getFileIdFromFilePath(filePath); - }).collect(Collectors.toSet()); - - // append rollback blocks for updates - if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { - hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus, - filesToNumBlocksRollback, deletedFiles); - } - break; - } catch (IOException io) { - throw new UncheckedIOException("Failed to rollback for commit " + commit, io); - } - default: break; - } - return hoodieRollbackStats; - }).collect(Collectors.toList()); - }).flatMap(x -> x.iterator()).filter(x -> x != null).collect(); + } catch (IOException io) { + throw new UncheckedIOException("Failed to rollback for commit " + commit, io); + } + default: + break; + } + return hoodieRollbackStats; + }).collect(Collectors.toList())).flatMap(List::iterator).filter(Objects::nonNull).collect(); commitsAndCompactions.entrySet().stream().map( entry -> new HoodieInstant(true, entry.getValue().getAction(), @@ -312,9 +301,8 @@ protected HoodieRollingStatMetadata getRollingStats() { Optional lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata() .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); if (lastRollingStat.isPresent()) { - HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata + return HoodieCommitMetadata .fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class); - return rollingStatMetadata; } } return null; @@ -411,8 +399,7 @@ private long getTotalFileSize(String partitionPath, FileSlice fileSlice) { HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId()); if (rollingStatForFile != null) { long inserts = rollingStatForFile.getInserts(); - long totalSize = averageRecordSize * inserts; - return totalSize; + return averageRecordSize * inserts; } } } @@ -427,10 +414,7 @@ private long getTotalFileSize(String partitionPath, FileSlice fileSlice) { private boolean isSmallFile(String partitionPath, FileSlice fileSlice) { long totalSize = getTotalFileSize(partitionPath, fileSlice); - if (totalSize < config.getParquetMaxFileSize()) { - return true; - } - return false; + return totalSize < config.getParquetMaxFileSize(); } // TODO (NA) : Make this static part of utility @@ -470,11 +454,8 @@ private HoodieRollbackStat rollback(HoodieIndex hoodieIndex, String partitionPat commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { // Filter out stats without prevCommit since they are all inserts - if (wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null - && !deletedFiles.contains(wStat.getFileId())) { - return true; - } - return false; + return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null + && !deletedFiles.contains(wStat.getFileId()); }).forEach(wStat -> { HoodieLogFormat.Writer writer = null; String baseCommitTime = wStat.getPrevCommit(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java index 1f30cfad4611f..01aac41272d54 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java @@ -21,10 +21,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import java.io.Serializable; -import java.util.Comparator; -import java.util.List; -import java.util.Optional; -import java.util.TreeMap; +import java.util.*; import java.util.stream.Stream; /** @@ -33,10 +30,7 @@ public class HoodieFileGroup implements Serializable { public static Comparator getReverseCommitTimeComparator() { - return (o1, o2) -> { - // reverse the order - return o2.compareTo(o1); - }; + return Comparator.reverseOrder(); } /** @@ -127,7 +121,7 @@ private boolean isFileSliceCommitted(FileSlice slice) { * Get all the the file slices including in-flight ones as seen in underlying file-system */ public Stream getAllFileSlicesIncludingInflight() { - return fileSlices.entrySet().stream().map(sliceEntry -> sliceEntry.getValue()); + return fileSlices.entrySet().stream().map(Map.Entry::getValue); } /** @@ -143,8 +137,8 @@ public Optional getLatestFileSlicesIncludingInflight() { public Stream getAllFileSlices() { if (!timeline.empty()) { return fileSlices.entrySet().stream() - .map(sliceEntry -> sliceEntry.getValue()) - .filter(slice -> isFileSliceCommitted(slice)); + .map(Map.Entry::getValue) + .filter(this::isFileSliceCommitted); } return Stream.empty(); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java index 9f159d6cae51f..88acc4ae6b68a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java @@ -103,7 +103,7 @@ public static Comparator getBaseInstantAndLogVersionComparator() return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion()); } // reverse the order by base-commits - return new Integer(baseInstantTime2.compareTo(baseInstantTime1)); + return baseInstantTime2.compareTo(baseInstantTime1); }; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java index f42238ed6721a..346c31903bbd4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/StorageSchemes.java @@ -53,14 +53,13 @@ public boolean supportsAppend() { } public static boolean isSchemeSupported(String scheme) { - return Arrays.stream(values()).filter(s -> s.getScheme().equals(scheme)).count() > 0; + return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme)); } public static boolean isAppendSupported(String scheme) { if (!isSchemeSupported(scheme)) { throw new IllegalArgumentException("Unsupported scheme :" + scheme); } - return Arrays.stream(StorageSchemes.values()) - .filter(s -> s.supportsAppend() && s.scheme.equals(scheme)).count() > 0; + return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsAppend() && s.scheme.equals(scheme)); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 007a0f825ed41..345f2906e6110 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -95,27 +95,24 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, fs.mkdirs(metadataFolder); } Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE); - FSDataOutputStream outputStream = fs.create(propertyPath); - try { + try (FSDataOutputStream outputStream = fs.create(propertyPath)) { if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) { throw new IllegalArgumentException( - HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified"); + HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified"); } if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) { properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name()); } if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME) == HoodieTableType.MERGE_ON_READ - .name() - && !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) { + .name() + && !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) { properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS); } if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) { properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); } properties - .store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); - } finally { - outputStream.close(); + .store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java index fc0402f08d5f8..e6d87ca5c6a90 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java @@ -57,7 +57,7 @@ public static HoodieLogBlock getBlock(HoodieLogFile logFile, long blockSize, long blockEndPos, Map header, - Map footer) throws IOException { + Map footer) { return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily, Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index 91520b7252ba5..cf5cb9eecd7be 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -91,10 +91,10 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, List pendingCompactionInstants = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); this.fileIdToPendingCompaction = ImmutableMap.copyOf( - CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream().map(entry -> { - return Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(), - CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue()))); - }).collect(Collectors.toMap(Pair::getKey, Pair::getValue))); + CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream() + .map(entry -> Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(), + CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue())))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue))); } /** @@ -152,10 +152,10 @@ private List addFilesToView(FileStatus[] statuses) { String fileId = pair.getValue(); HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, visibleActiveTimeline); if (dataFiles.containsKey(pair)) { - dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); + dataFiles.get(pair).forEach(group::addDataFile); } if (logFiles.containsKey(pair)) { - logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); + logFiles.get(pair).forEach(group::addLogFile); } if (fileIdToPendingCompaction.containsKey(fileId)) { // If there is no delta-commit after compaction request, this step would ensure a new file-slice appears @@ -219,9 +219,7 @@ public Stream getLatestDataFiles(final String partitionPath) { @Override public Stream getLatestDataFiles() { return fileGroupMap.values().stream() - .map(fileGroup -> { - return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst(); - }) + .map(fileGroup -> fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst()) .filter(Optional::isPresent) .map(Optional::get); } @@ -230,15 +228,13 @@ public Stream getLatestDataFiles() { public Stream getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) { return getAllFileGroups(partitionPath) - .map(fileGroup -> { - return fileGroup.getAllDataFiles() - .filter(dataFile -> - HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), - maxCommitTime, - HoodieTimeline.LESSER_OR_EQUAL)) - .filter(df -> !isDataFileDueToPendingCompaction(df)) - .findFirst(); - }) + .map(fileGroup -> fileGroup.getAllDataFiles() + .filter(dataFile -> + HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), + maxCommitTime, + HoodieTimeline.LESSER_OR_EQUAL)) + .filter(df -> !isDataFileDueToPendingCompaction(df)) + .findFirst()) .filter(Optional::isPresent) .map(Optional::get); } @@ -246,12 +242,10 @@ public Stream getLatestDataFilesBeforeOrOn(String partitionPath, @Override public Stream getLatestDataFilesInRange(List commitsToReturn) { return fileGroupMap.values().stream() - .map(fileGroup -> { - return fileGroup.getAllDataFiles() - .filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime()) - && !isDataFileDueToPendingCompaction(dataFile)) - .findFirst(); - }) + .map(fileGroup -> fileGroup.getAllDataFiles() + .filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime()) + && !isDataFileDueToPendingCompaction(dataFile)) + .findFirst()) .filter(Optional::isPresent) .map(Optional::get); } @@ -259,15 +253,13 @@ public Stream getLatestDataFilesInRange(List commitsToRe @Override public Stream getLatestDataFilesOn(String partitionPath, String instantTime) { return getAllFileGroups(partitionPath) - .map(fileGroup -> { - return fileGroup.getAllDataFiles() - .filter(dataFile -> - HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), - instantTime, - HoodieTimeline.EQUAL)) - .filter(df -> !isDataFileDueToPendingCompaction(df)) - .findFirst(); - }) + .map(fileGroup -> fileGroup.getAllDataFiles() + .filter(dataFile -> + HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), + instantTime, + HoodieTimeline.EQUAL)) + .filter(df -> !isDataFileDueToPendingCompaction(df)) + .findFirst()) .filter(Optional::isPresent) .map(Optional::get); } @@ -275,7 +267,7 @@ public Stream getLatestDataFilesOn(String partitionPath, String @Override public Stream getAllDataFiles(String partitionPath) { return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getAllDataFiles()) + .map(HoodieFileGroup::getAllDataFiles) .flatMap(dataFileList -> dataFileList) .filter(df -> !isDataFileDueToPendingCompaction(df)); } @@ -283,7 +275,7 @@ public Stream getAllDataFiles(String partitionPath) { @Override public Stream getLatestFileSlices(String partitionPath) { return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestFileSlice()) + .map(HoodieFileGroup::getLatestFileSlice) .filter(Optional::isPresent) .map(Optional::get) .map(this::filterDataFileAfterPendingCompaction); @@ -312,11 +304,8 @@ public Stream getLatestUnCompactedFileSlices(String partitionPath) { */ private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) { Pair compactionWithInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()); - if ((null != compactionWithInstantTime) - && fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey())) { - return true; - } - return false; + return (null != compactionWithInstantTime) + && fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey()); } /** @@ -330,7 +319,7 @@ private FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) { // Data file is filtered out of the file-slice as the corresponding compaction // instant not completed yet. FileSlice transformed = new FileSlice(fileSlice.getBaseInstantTime(), fileSlice.getFileId()); - fileSlice.getLogFiles().forEach(lf -> transformed.addLogFile(lf)); + fileSlice.getLogFiles().forEach(transformed::addLogFile); return transformed; } return fileSlice; @@ -358,8 +347,8 @@ private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, F merged.setDataFile(penultimateSlice.getDataFile().get()); } // Add Log files from penultimate and last slices - penultimateSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf)); - lastSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf)); + penultimateSlice.getLogFiles().forEach(merged::addLogFile); + lastSlice.getLogFiles().forEach(merged::addLogFile); return merged; } @@ -409,7 +398,7 @@ public Stream getLatestFileSliceInRange(List commitsToReturn) @Override public Stream getAllFileSlices(String partitionPath) { return getAllFileGroups(partitionPath) - .map(group -> group.getAllFileSlices()) + .map(HoodieFileGroup::getAllFileSlices) .flatMap(sliceList -> sliceList); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index 636bec583ec9c..a112a6de29f0d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -80,10 +80,8 @@ public static Set filterParquetRowKeys(Configuration configuration, Path Schema readSchema = HoodieAvroUtils.getRecordKeySchema(); AvroReadSupport.setAvroReadSchema(conf, readSchema); AvroReadSupport.setRequestedProjection(conf, readSchema); - ParquetReader reader = null; Set rowKeys = new HashSet<>(); - try { - reader = AvroParquetReader.builder(filePath).withConf(conf).build(); + try (ParquetReader reader = AvroParquetReader.builder(filePath).withConf(conf).build()) { Object obj = reader.read(); while (obj != null) { if (obj instanceof GenericRecord) { @@ -97,15 +95,8 @@ public static Set filterParquetRowKeys(Configuration configuration, Path } catch (IOException e) { throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e); - } finally { - if (reader != null) { - try { - reader.close(); - } catch (IOException e) { - // ignore - } - } } + // ignore return rowKeys; } @@ -212,10 +203,7 @@ static class RecordKeysFilterFunction implements Function { @Override public Boolean apply(String recordKey) { - if (candidateKeys.contains(recordKey)) { - return true; - } - return false; + return candidateKeys.contains(recordKey); } } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java index 54348d5f56c72..9beaa24dc17a2 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java @@ -83,7 +83,7 @@ public static Object loadClass(String clazz, Class[] constructorArgTypes, Obj */ public static Object loadClass(String clazz, Object... constructorArgs) { Class[] constructorArgTypes = Arrays.stream(constructorArgs) - .map(arg -> arg.getClass()).toArray(Class[]::new); + .map(Object::getClass).toArray(Class[]::new); return loadClass(clazz, constructorArgTypes, constructorArgs); } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java index e2bd10b8ddbe1..b48f891467c6d 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -131,9 +131,9 @@ protected static String arrayWritableToString(ArrayWritable writable) { builder.append(String.format("(Size: %s)[", values.length)); for (Writable w : values) { if (w instanceof ArrayWritable) { - builder.append(arrayWritableToString((ArrayWritable) w) + " "); + builder.append(arrayWritableToString((ArrayWritable) w)).append(" "); } else { - builder.append(w + " "); + builder.append(w).append(" "); } } builder.append("]"); @@ -186,7 +186,7 @@ public static Schema generateProjectionSchema(Schema writeSchema, List f if (field == null) { throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! " + "Derived Schema Fields: " - + schemaFieldsMap.keySet().stream().collect(Collectors.toList())); + + new ArrayList<>(schemaFieldsMap.keySet())); } projectedFields .add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index 0dd03ee15f3e2..75d13e7cfbd74 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -58,7 +58,7 @@ object AvroConversionUtils { def createConverterToAvro(dataType: DataType, structName: String, - recordNamespace: String): (Any) => Any = { + recordNamespace: String): Any => Any = { dataType match { case BinaryType => (item: Any) => item match { diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala index 7e4f8f03efc01..5df7118bd4dcf 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala @@ -79,7 +79,7 @@ object DataSourceWriteOptions { val BULK_INSERT_OPERATION_OPT_VAL = "bulk_insert" val INSERT_OPERATION_OPT_VAL = "insert" val UPSERT_OPERATION_OPT_VAL = "upsert" - val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL; + val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL /** * The storage type for the underlying data, for this write. diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index 019a76e235e18..1f7b47682abc3 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -56,7 +56,7 @@ class IncrementalRelation(val sqlContext: SQLContext, // TODO : Figure out a valid HoodieWriteConfig val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), sqlContext.sparkContext) - val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants(); + val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants() if (commitTimeline.empty()) { throw new HoodieException("No instants to incrementally pull") } @@ -90,7 +90,7 @@ class IncrementalRelation(val sqlContext: SQLContext, } // unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view // 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")) sqlContext.read.options(sOpts) .schema(latestSchema) // avoid AnalysisException for empty input diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index a91e8f396bd55..189c97a7cf9a8 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -99,7 +99,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi Path outputPath = new Path(outputDir); if (fs.exists(outputPath)) { logger.warn( - String.format("The output path %targetBasePath already exists, deleting", outputPath)); + String.format("The output path %s targetBasePath already exists, deleting", outputPath)); fs.delete(new Path(outputDir), true); } @@ -155,7 +155,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi } if (fs.exists(targetFilePath)) { logger.error(String.format( - "The target output commit file (%targetBasePath) already exists.", targetFilePath)); + "The target output commit file (%s targetBasePath) already exists.", targetFilePath)); } FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf()); } @@ -166,7 +166,8 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi // Create the _SUCCESS tag Path successTagPath = new Path(outputDir + "/_SUCCESS"); if (!fs.exists(successTagPath)) { - logger.info("Creating _SUCCESS under targetBasePath: " + outputDir); + logger.info(String.format( + "Creating _SUCCESS under targetBasePath: $s", outputDir)); fs.createNewFile(successTagPath); } } @@ -175,7 +176,7 @@ public static void main(String[] args) throws IOException { // Take input configs final Config cfg = new Config(); new JCommander(cfg, args); - logger.info(String.format("Snapshot hoodie table from %targetBasePath to %targetBasePath", + logger.info(String.format("Snapshot hoodie table from %s targetBasePath to %stargetBasePath", cfg.basePath, cfg.outputPath)); // Create a spark job to do the snapshot copy diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index 7cd040dfed98d..b5008cf31aabc 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -95,14 +95,8 @@ public static String parseSchema(FileSystem fs, String schemaFile) throws Except } long len = fs.getFileStatus(p).getLen(); ByteBuffer buf = ByteBuffer.allocate((int) len); - FSDataInputStream inputStream = null; - try { - inputStream = fs.open(p); + try (FSDataInputStream inputStream = fs.open(p)) { inputStream.readFully(0, buf.array(), 0, buf.array().length); - } finally { - if (inputStream != null) { - inputStream.close(); - } } return new String(buf.array()); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java index 3228fdeb5e3e4..f6a0c90e3855e 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java @@ -28,6 +28,7 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Arrays; +import java.util.Collections; import java.util.Date; import java.util.TimeZone; import org.apache.avro.generic.GenericRecord; @@ -73,7 +74,7 @@ public TimestampBasedKeyGenerator(TypedProperties config) { if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) { DataSourceUtils - .checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP)); + .checkRequiredProperties(config, Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP)); this.inputDateFormat = new SimpleDateFormat( config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP)); this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT")); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java index d4df829c43609..4629d4b40aaf3 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java @@ -23,7 +23,8 @@ import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; + import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,7 +53,7 @@ public static class Config { public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { super(props, jssc); - DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP)); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP)); this.fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), jssc.hadoopConfiguration()); try { this.sourceSchema = new Schema.Parser().parse( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java index e77a67bed3358..3312db5aae8f4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java @@ -25,7 +25,8 @@ import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.net.URL; -import java.util.Arrays; +import java.util.Collections; + import org.apache.avro.Schema; import org.apache.spark.api.java.JavaSparkContext; @@ -55,7 +56,7 @@ private String fetchSchemaFromRegistry(String registryUrl) throws IOException { public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) { super(props, jssc); - DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.SCHEMA_REGISTRY_URL_PROP)); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SCHEMA_REGISTRY_URL_PROP)); String registryUrl = props.getString(Config.SCHEMA_REGISTRY_URL_PROP); try { this.schema = new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl)); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java index e8abfba791802..feb06d5b362a3 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java @@ -87,7 +87,7 @@ public Schema getSchema() { } - public GenericRecord fromAvroBinary(byte[] avroBinary) throws IOException { + public GenericRecord fromAvroBinary(byte[] avroBinary) { initSchema(); initInjection(); return recordInjection.invert(avroBinary).get(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java index a4d01d1b9c75f..4e14714134bf7 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java @@ -39,9 +39,8 @@ public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, Sch @Override protected JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { - JavaRDD recordRDD = KafkaUtils + return KafkaUtils .createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, kafkaParams, offsetRanges).values().map(obj -> (GenericRecord) obj); - return recordRDD; } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java index 2d4d07851b2de..6d962b276d57d 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java @@ -26,10 +26,7 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.utilities.schema.SchemaProvider; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; +import java.util.*; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FileStatus; @@ -58,7 +55,7 @@ static class Config { public DFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { 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.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), sparkContext.hadoopConfiguration()); } @@ -75,16 +72,14 @@ public Pair>, String> fetchNewData( new Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), true); while (fitr.hasNext()) { LocatedFileStatus fileStatus = fitr.next(); - if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream().filter( - pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) { + if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream() + .anyMatch(pfx -> fileStatus.getPath().getName().startsWith(pfx))) { continue; } eligibleFiles.add(fileStatus); } // sort them by modification time. - eligibleFiles.sort((FileStatus f1, FileStatus f2) -> Long.valueOf(f1.getModificationTime()) - .compareTo(Long.valueOf( - f2.getModificationTime()))); + eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime)); // Filter based on checkpoint & input size, if needed long currentBytes = 0; @@ -110,8 +105,7 @@ public Pair>, String> fetchNewData( // no data to read if (filteredFiles.size() == 0) { return new ImmutablePair<>(Optional.empty(), - lastCheckpointStr.isPresent() ? lastCheckpointStr.get() - : String.valueOf(Long.MIN_VALUE)); + lastCheckpointStr.orElseGet(() -> String.valueOf(Long.MIN_VALUE))); } // read the files out. diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index faf6d0966785b..3d6af40fab588 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -76,7 +76,7 @@ static class Config { public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { 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.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP); this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration()); } @@ -121,7 +121,7 @@ public Pair>, String> fetchNewData( if (!commitToPull.isPresent()) { return new ImmutablePair<>(Optional.empty(), - lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); + lastCheckpointStr.orElse("")); } // read the files out. diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java index 3167dbe592ce1..6b1018e152b98 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java @@ -35,6 +35,6 @@ public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, Schem @Override protected JavaRDD fromFiles(AvroConvertor convertor, String pathStr) { - return sparkContext.textFile(pathStr).map((String j) -> convertor.fromJson(j)); + return sparkContext.textFile(pathStr).map(convertor::fromJson); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java index 6dd111654b21a..b271e370494a9 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java @@ -40,6 +40,6 @@ public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext protected JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, offsetRanges) - .values().map(jsonStr -> avroConvertor.fromJson(jsonStr)); + .values().map(avroConvertor::fromJson); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java index 5e0328016fa7d..4699fcaf32cc5 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java @@ -25,11 +25,8 @@ import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; import com.uber.hoodie.utilities.schema.SchemaProvider; -import java.util.Arrays; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Optional; + +import java.util.*; import java.util.stream.Collectors; import kafka.common.TopicAndPartition; import org.apache.avro.generic.GenericRecord; @@ -85,7 +82,7 @@ public static HashMap strToOffsets */ public static String offsetsToStr(OffsetRange[] ranges) { StringBuilder sb = new StringBuilder(); - // atleast 1 partition will be present. + // at least 1 partition will be present. sb.append(ranges[0].topic() + ","); sb.append(Arrays.stream(ranges) .map(r -> String.format("%s:%d", r.partition(), r.untilOffset())) @@ -106,8 +103,7 @@ public static OffsetRange[] computeOffsetRanges( HashMap toOffsetMap, long numEvents) { - Comparator byPartition = (OffsetRange o1, OffsetRange o2) -> - Integer.valueOf(o1.partition()).compareTo(Integer.valueOf(o2.partition())); + Comparator byPartition = Comparator.comparing(OffsetRange::partition); // Create initial offset ranges for each 'to' partition, with from = to offsets. OffsetRange[] ranges = new OffsetRange[toOffsetMap.size()]; @@ -144,7 +140,7 @@ public static OffsetRange[] computeOffsetRanges( } public static long totalNewMessages(OffsetRange[] ranges) { - return Arrays.asList(ranges).stream().mapToLong(r -> r.count()).sum(); + return Arrays.stream(ranges).mapToLong(OffsetRange::count).sum(); } } @@ -166,13 +162,20 @@ public static java.util.Map toJavaMap(Map m) { } } + /** + * Kafka reset offset strategies + */ + enum KafkaResetOffsetStrategies { + LARGEST, + SMALLEST + } /** * Configs to be passed for this source. All standard Kafka consumer configs are also respected */ static class Config { private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; - private static final String DEFAULT_AUTO_RESET_OFFSET = "largest"; + private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LARGEST; } @@ -187,7 +190,7 @@ public KafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaP for (Object prop : props.keySet()) { kafkaParams.put(prop.toString(), props.getString(prop.toString())); } - DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.KAFKA_TOPIC_NAME)); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.KAFKA_TOPIC_NAME)); topicName = props.getString(Config.KAFKA_TOPIC_NAME); } @@ -200,7 +203,7 @@ public Pair>, String> fetchNewData( // Obtain current metadata for the topic KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); Either, Set> either = cluster.getPartitions( - ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName)))); + ScalaHelpers.toScalaSet(new HashSet<>(Collections.singletonList(topicName)))); if (either.isLeft()) { // log errors. and bail out. throw new HoodieDeltaStreamerException("Error obtaining partition metadata", @@ -213,17 +216,20 @@ public Pair>, String> fetchNewData( if (lastCheckpointStr.isPresent()) { fromOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get()); } else { - String autoResetValue = props - .getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET); - if (autoResetValue.equals("smallest")) { - fromOffsets = new HashMap(ScalaHelpers.toJavaMap( - cluster.getEarliestLeaderOffsets(topicPartitions).right().get())); - } else if (autoResetValue.equals("largest")) { - fromOffsets = new HashMap( - ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); - } else { - throw new HoodieNotSupportedException( - "Auto reset value must be one of 'smallest' or 'largest' "); + KafkaResetOffsetStrategies autoResetValue = KafkaResetOffsetStrategies.valueOf( + props.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET.toString()).toUpperCase()); + switch (autoResetValue) { + case SMALLEST: + fromOffsets = new HashMap(ScalaHelpers.toJavaMap( + cluster.getEarliestLeaderOffsets(topicPartitions).right().get())); + break; + case LARGEST: + fromOffsets = new HashMap( + ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); + break; + default: + throw new HoodieNotSupportedException( + "Auto reset value must be one of 'smallest' or 'largest' "); } } @@ -236,7 +242,7 @@ public Pair>, String> fetchNewData( OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); if (totalNewMsgs <= 0) { - return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); + return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.orElse("")); } else { log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName); } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java index f4c589a00269f..57369de33cbf3 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java @@ -71,7 +71,7 @@ private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) { public Pair>, String> fetchNewData(Optional lastCheckpointStr, long sourceLimit) { - int nextCommitNum = lastCheckpointStr.isPresent() ? Integer.parseInt(lastCheckpointStr.get()) + 1 : 0; + int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0); String commitTime = String.format("%05d", nextCommitNum); // No new data. if (sourceLimit <= 0) { diff --git a/pom.xml b/pom.xml index f4255a0782ae8..88196084c9a47 100644 --- a/pom.xml +++ b/pom.xml @@ -197,6 +197,29 @@ + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + false + true + true + false + ${project.basedir}/src/main/scala + ${project.basedir}/src/test/scala + style/scalastyle-config.xml + UTF-8 + + + + compile + + check + + + + org.apache.maven.plugins maven-compiler-plugin @@ -634,11 +657,6 @@ 1.9.13 - - junit - junit - ${junit.version} - org.apache.hadoop hadoop-hdfs diff --git a/style/scalastyle-config.xml b/style/scalastyle-config.xml new file mode 100644 index 0000000000000..e7e41a123ae92 --- /dev/null +++ b/style/scalastyle-config.xml @@ -0,0 +1,98 @@ + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From 55b73beb187791f900e7cd331731ce550446158e Mon Sep 17 00:00:00 2001 From: Manu Sridharan Date: Sun, 16 Dec 2018 09:27:27 -0800 Subject: [PATCH 136/374] Add m2 directory to Travis cache --- .travis.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.travis.yml b/.travis.yml index d3f5d7b8d6dff..e60067e4f6246 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,3 +6,6 @@ env: - HUDI_QUIETER_LOGGING=1 services: - docker +cache: + directories: + - $HOME/.m2 From cbe5893bd32ada6f07eb71ce6aae15e5f3219f0c Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 21 Dec 2018 14:51:44 -0800 Subject: [PATCH 137/374] Enabling hard deletes for MergeOnRead table type --- .../uber/hoodie/io/HoodieAppendHandle.java | 7 +-- .../hoodie/table/TestMergeOnReadTable.java | 4 +- .../common/model/HoodieAvroPayload.java | 9 +++- .../log/AbstractHoodieLogRecordScanner.java | 5 +- .../log/HoodieMergedLogRecordScanner.java | 13 ++--- .../log/HoodieUnMergedLogRecordScanner.java | 3 +- .../table/log/block/HoodieDeleteBlock.java | 15 +++--- .../hoodie/common/util/SpillableMapUtils.java | 11 +++++ .../common/table/log/HoodieLogFormatTest.java | 47 +++++++++++++++---- .../RealtimeCompactedRecordReader.java | 28 ++++++++--- 10 files changed, 103 insertions(+), 39 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 452d111eb56ea..04f715bab8bb8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -20,6 +20,7 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDeltaWriteStat; +import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; @@ -67,7 +68,7 @@ public class HoodieAppendHandle extends HoodieIOH // Buffer for holding records in memory before they are flushed to disk private List recordList = new ArrayList<>(); // Buffer for holding records (to be deleted) in memory before they are flushed to disk - private List keysToDelete = new ArrayList<>(); + private List keysToDelete = new ArrayList<>(); private TableFileSystemView.RealtimeView fileSystemView; private String partitionPath; private Iterator> recordItr; @@ -209,7 +210,7 @@ private void doAppend(Map header) { } if (keysToDelete.size() > 0) { writer = writer.appendBlock( - new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), header)); + new HoodieDeleteBlock(keysToDelete.stream().toArray(HoodieKey[]::new), header)); keysToDelete.clear(); } } catch (Exception e) { @@ -286,7 +287,7 @@ private void writeToBuffer(HoodieRecord record) { if (indexedRecord.isPresent()) { recordList.add(indexedRecord.get()); } else { - keysToDelete.add(record.getRecordKey()); + keysToDelete.add(record.getKey()); } numberOfRecords++; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index ed68e5857d741..1b86180fad0ea 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -326,8 +326,8 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); - //Wrote 40 records and deleted 20 records, so remaining 40-20 = 20 - assertEquals("Must contain 20 records", 20, recordsRead.size()); + //Wrote 20 records and deleted 20 records, so remaining 20-20 = 0 + assertEquals("Must contain 0 records", 0, recordsRead.size()); } @Test diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java index 9e4be0db93ad9..692079014404b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java @@ -36,7 +36,11 @@ public class HoodieAvroPayload implements HoodieRecordPayload public HoodieAvroPayload(Optional record) { try { - this.recordBytes = HoodieAvroUtils.avroToBytes(record.get()); + if (record.isPresent()) { + this.recordBytes = HoodieAvroUtils.avroToBytes(record.get()); + } else { + this.recordBytes = new byte[0]; + } } catch (IOException io) { throw new HoodieIOException("Cannot convert record to bytes", io); } @@ -55,6 +59,9 @@ public Optional combineAndGetUpdateValue(IndexedRecord currentVal @Override public Optional getInsertValue(Schema schema) throws IOException { + if (recordBytes.length == 0) { + return Optional.empty(); + } Optional record = Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema)); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index 56271904da727..5a4bb24511aea 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -19,6 +19,7 @@ import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; +import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -63,7 +64,7 @@ public abstract class AbstractHoodieLogRecordScanner { private static final Logger log = LogManager.getLogger(AbstractHoodieLogRecordScanner.class); // Reader schema for the records - private final Schema readerSchema; + protected final Schema readerSchema; // Latest valid instant time // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark. private final String latestInstantTime; @@ -291,7 +292,7 @@ protected abstract void processNextRecord(HoodieRecord hoodieRecord) { + protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { String key = hoodieRecord.getRecordKey(); if (records.containsKey(key)) { - // Merge and store the merged record + // Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be + // done when a delete (empty payload) is encountered before or after an insert/update. HoodieRecordPayload combinedValue = records.get(key).getData().preCombine(hoodieRecord.getData()); records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); } else { @@ -115,10 +117,9 @@ protected void processNextRecord(HoodieRecord hoo } @Override - protected void processNextDeletedKey(String key) { - // TODO : If delete is the only block written and/or records are present in parquet file - // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry - records.remove(key); + protected void processNextDeletedKey(HoodieKey hoodieKey) { + records.put(hoodieKey.getRecordKey(), SpillableMapUtils.generateEmptyPayload(hoodieKey.getRecordKey(), + hoodieKey.getPartitionPath(), getPayloadClassFQN())); } public long getTotalTimeTakenToReadAndMergeBlocks() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java index 98264352b8a94..f8673f4582418 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -18,6 +18,7 @@ package com.uber.hoodie.common.table.log; +import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import java.util.List; @@ -43,7 +44,7 @@ protected void processNextRecord(HoodieRecord hoo } @Override - protected void processNextDeletedKey(String key) { + protected void processNextDeletedKey(HoodieKey key) { throw new IllegalStateException("Not expected to see delete records in this log-scan mode. Check Job Config"); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index eddae761d0366..9ee4f296b6414 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -16,16 +16,16 @@ package com.uber.hoodie.common.table.log.block; +import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.storage.SizeAwareDataInputStream; -import com.uber.hoodie.common.util.StringUtils; +import com.uber.hoodie.common.util.SerializationUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.charset.Charset; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -36,9 +36,9 @@ */ public class HoodieDeleteBlock extends HoodieLogBlock { - private String[] keysToDelete; + private HoodieKey[] keysToDelete; - public HoodieDeleteBlock(String[] keysToDelete, + public HoodieDeleteBlock(HoodieKey[] keysToDelete, Map header) { this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>()); this.keysToDelete = keysToDelete; @@ -64,15 +64,14 @@ public byte[] getContentBytes() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); - byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ",") - .getBytes(Charset.forName("utf-8")); + byte[] bytesToWrite = SerializationUtils.serialize(getKeysToDelete()); output.writeInt(HoodieLogBlock.version); output.writeInt(bytesToWrite.length); output.write(bytesToWrite); return baos.toByteArray(); } - public String[] getKeysToDelete() { + public HoodieKey[] getKeysToDelete() { try { if (keysToDelete == null) { if (!getContent().isPresent() && readBlockLazily) { @@ -86,7 +85,7 @@ public String[] getKeysToDelete() { int dataLength = dis.readInt(); byte[] data = new byte[dataLength]; dis.readFully(data); - this.keysToDelete = new String(data).split(","); + this.keysToDelete = SerializationUtils.deserialize(data); deflate(); } return keysToDelete; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index 79aa90ee65be9..cb0aab35f0c83 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -117,4 +117,15 @@ public static R convertToHoodieRecordPayload(GenericRecord rec, String paylo .loadPayload(payloadClazz, new Object[]{Optional.of(rec)}, Optional.class)); return (R) hoodieRecord; } + + /** + * Utility method to convert bytes to HoodieRecord using schema and payload class + */ + public static R generateEmptyPayload(String recKey, String partitionPath, String payloadClazz) { + HoodieRecord hoodieRecord = new HoodieRecord<>( + new HoodieKey(recKey, partitionPath), + ReflectionUtils + .loadPayload(payloadClazz, new Object[]{Optional.empty()}, Optional.class)); + return (R) hoodieRecord; + } } \ No newline at end of file diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 3c57b55f25758..cc795ffcbf688 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Maps; import com.uber.hoodie.common.minicluster.MiniClusterUtil; import com.uber.hoodie.common.model.HoodieArchivedLogFile; +import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; @@ -43,6 +44,7 @@ import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.exception.CorruptedLogFileException; import java.io.IOException; +import java.io.UncheckedIOException; import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Arrays; @@ -714,10 +716,13 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys - List deletedKeys = originalKeys.subList(0, 50); + List deletedKeys = copyOfRecords1.stream().map( + s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(), + ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) + .collect(Collectors.toList()).subList(0, 50); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer = writer.appendBlock(deleteBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, @@ -727,8 +732,19 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); + final List emptyPayloads = new ArrayList<>(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals("Stream collect should return all 150 records", 150, readKeys.size()); + scanner.forEach(s -> { + try { + if (!s.getData().getInsertValue(schema).isPresent()) { + emptyPayloads.add(true); + } + } catch (IOException io) { + throw new UncheckedIOException(io); + } + }); + assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); + assertEquals("Stream collect should return all 50 records with empty payloads", 50, emptyPayloads.size()); originalKeys.removeAll(deletedKeys); Collections.sort(originalKeys); Collections.sort(readKeys); @@ -782,8 +798,13 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys - List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); + // Delete 50 keys + List deletedKeys = copyOfRecords1.stream().map( + s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(), + ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) + .collect(Collectors.toList()).subList(0, 50); + + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer = writer.appendBlock(deleteBlock); // Attempt 1 : Write rollback block for a failed write @@ -839,8 +860,12 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys - List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); + // Delete 50 keys + List deletedKeys = copyOfRecords1.stream().map( + s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(), + ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) + .collect(Collectors.toList()).subList(0, 50); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer = writer.appendBlock(deleteBlock); // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write @@ -921,8 +946,12 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys - List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); + // Delete 50 keys + List deletedKeys = copyOfRecords1.stream().map( + s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(), + ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) + .collect(Collectors.toList()).subList(0, 50); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer = writer.appendBlock(deleteBlock); // Write 1 rollback block for a failed write diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java index 266e0d64c3975..b8ef2680f5690 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -24,7 +24,9 @@ import com.uber.hoodie.common.util.FSUtils; import java.io.IOException; import java.util.HashMap; +import java.util.Optional; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; @@ -60,13 +62,20 @@ private void readAndCompactLog() throws IOException { // but can return records for completed commits > the commit we are trying to read (if using // readCommit() API) for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { - GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(getReaderSchema()).get(); + Optional recordOptional = hoodieRecord.getData().getInsertValue(getReaderSchema()); + ArrayWritable aWritable; String key = hoodieRecord.getRecordKey(); - // we assume, a later safe record in the log, is newer than what we have in the map & - // replace it. - // TODO : handle deletes here - ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema()); - deltaRecordMap.put(key, aWritable); + if (recordOptional.isPresent()) { + GenericRecord rec = (GenericRecord) recordOptional.get(); + // we assume, a later safe record in the log, is newer than what we have in the map & + // replace it. + // TODO : handle deletes here + aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema()); + deltaRecordMap.put(key, aWritable); + } else { + aWritable = new ArrayWritable(Writable.class, new Writable[0]); + deltaRecordMap.put(key, aWritable); + } if (LOG.isDebugEnabled()) { LOG.debug("Log record : " + arrayWritableToString(aWritable)); } @@ -92,8 +101,13 @@ public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); } if (deltaRecordMap.containsKey(key)) { - // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? + // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro. This is required since the + // deltaRecord may not be a full record and needs values of columns from the parquet Writable[] replaceValue = deltaRecordMap.get(key).get(); + if (replaceValue.length < 1) { + // This record has been deleted, move to the next record + return next(aVoid, arrayWritable); + } Writable[] originalValue = arrayWritable.get(); try { System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); From 8eba0e515be0276c529bea83fb671d3fa933c0af Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 4 Jan 2019 15:01:49 -0800 Subject: [PATCH 138/374] Passing a path filter to avoid including folders under .hoodie directory as partition paths --- .../java/com/uber/hoodie/common/util/FSUtils.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index ba9cef9edd567..a18c6539f66ef 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidHoodiePathException; @@ -38,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.log4j.LogManager; @@ -135,7 +137,14 @@ public static String getFileId(String fullFileName) { public static List getAllFoldersThreeLevelsDown(FileSystem fs, String basePath) throws IOException { List datePartitions = new ArrayList<>(); - FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*")); + // Avoid listing and including any folders under the metafolder + PathFilter filter = (path) -> { + if (path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)) { + return false; + } + return true; + }; + FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"), filter); for (FileStatus status : folders) { Path path = status.getPath(); datePartitions.add(String.format("%s/%s/%s", path.getParent().getParent().getName(), From c2f85a5c94a7f73db376d35aca104ede7151122b Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Fri, 28 Dec 2018 10:24:23 -0800 Subject: [PATCH 139/374] Filtering partition paths before performing a list status on all partitions --- .../io/compact/HoodieRealtimeTableCompactor.java | 3 +++ .../io/compact/strategy/CompactionStrategy.java | 12 +++++++++++- .../compact/strategy/DayBasedCompactionStrategy.java | 8 ++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index ae74a58f36b23..549034e3bb6cc 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -181,6 +181,9 @@ public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, .getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); + // filter the partition paths if needed to reduce list status + partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths); + TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); List operations = diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index d7444818ca3af..3efc532a48c18 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -105,9 +105,19 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieWriteConfig writeConfig * @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan * @return list of compactions to perform in this run */ - protected List orderAndFilter(HoodieWriteConfig writeConfig, + public List orderAndFilter(HoodieWriteConfig writeConfig, List operations, List pendingCompactionPlans) { return operations; } + + /** + * Filter the partition paths based on compaction strategy + * @param writeConfig + * @param allPartitionPaths + * @return + */ + public List filterPartitionPaths(HoodieWriteConfig writeConfig, List allPartitionPaths) { + return allPartitionPaths; + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java index 3360feb783c56..ee4d9a5ac3b61 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java @@ -70,4 +70,12 @@ public List orderAndFilter(HoodieWriteConfig writeCon .collect(Collectors.toList()); return filteredList; } + + @Override + public List filterPartitionPaths(HoodieWriteConfig writeConfig, List allPartitionPaths) { + List filteredPartitionPaths = allPartitionPaths.stream().map(partition -> partition.replace("/", "-")) + .sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/")) + .collect(Collectors.toList()).subList(0, writeConfig.getTargetPartitionsPerDayBasedCompaction()); + return filteredPartitionPaths; + } } \ No newline at end of file From 46b6c814fa0f2d3d1cc4f36cb4e2ac1ae0594447 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 15 Jan 2019 13:51:47 -0800 Subject: [PATCH 140/374] Adding compaction to HoodieClient example --- hoodie-client/src/test/java/HoodieClientExample.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index ab81e2aca1c56..a438d5390b559 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -17,6 +17,7 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieAvroPayload; import com.uber.hoodie.common.model.HoodieRecord; @@ -28,6 +29,7 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex.IndexType; import java.util.List; +import java.util.Optional; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -110,5 +112,15 @@ public void run() throws Exception { records.addAll(dataGen.generateUpdates(newCommitTime, 100)); writeRecords = jsc.parallelize(records, 1); client.upsert(writeRecords, newCommitTime); + + /** + * Schedule a compaction and also perform compaction on a MOR dataset + */ + if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) { + Optional instant = client.scheduleCompaction(Optional.empty()); + JavaRDD writeStatues = client.compact(instant.get()); + client.commitCompaction(instant.get(), writeStatues, Optional.empty()); + } } + } From b9401b46d4f4e7cde63edd1a0d72c42735b8f671 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Wed, 26 Dec 2018 13:37:22 -0800 Subject: [PATCH 141/374] cleaner should now use commit timeline and not include deltacomits --- .../java/com/uber/hoodie/HoodieWriteClient.java | 10 +++++----- .../uber/hoodie/io/HoodieCommitArchiveLog.java | 2 +- .../java/com/uber/hoodie/io/HoodieIOHandle.java | 2 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 2 +- .../hoodie/table/HoodieMergeOnReadTable.java | 2 +- .../java/com/uber/hoodie/table/HoodieTable.java | 13 ++++++++++--- .../com/uber/hoodie/TestAsyncCompaction.java | 2 +- .../test/java/com/uber/hoodie/TestCleaner.java | 6 +++--- .../uber/hoodie/table/TestMergeOnReadTable.java | 16 ++++++++-------- 9 files changed, 31 insertions(+), 24 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 3693b55bf929a..c879c06ccb560 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -584,11 +584,11 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, public boolean savepoint(String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - if (table.getCompletedCommitTimeline().empty()) { + if (table.getCompletedCommitsTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } - String latestCommit = table.getCompletedCommitTimeline().lastInstant().get().getTimestamp(); + String latestCommit = table.getCompletedCommitsTimeline().lastInstant().get().getTimestamp(); logger.info("Savepointing latest commit " + latestCommit); return savepoint(latestCommit, user, comment); } @@ -615,7 +615,7 @@ public boolean savepoint(String commitTime, String user, String comment) { HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); - if (!table.getCompletedCommitTimeline().containsInstant(commitInstant)) { + if (!table.getCompletedCommitsTimeline().containsInstant(commitInstant)) { throw new HoodieSavepointException( "Could not savepoint non-existing commit " + commitInstant); } @@ -628,7 +628,7 @@ public boolean savepoint(String commitTime, String user, String comment) { table.getActiveTimeline().getInstantDetails(cleanInstant.get()).get()); lastCommitRetained = cleanMetadata.getEarliestCommitToRetain(); } else { - lastCommitRetained = table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); + lastCommitRetained = table.getCompletedCommitsTimeline().firstInstant().get().getTimestamp(); } // Cannot allow savepoint time on a commit that could have been cleaned @@ -792,7 +792,7 @@ private void rollback(List commits) { table.getActiveTimeline().filterPendingCompactionTimeline().getInstants() .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline(); - HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); + HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); // Check if any of the commits is a savepoint - do not allow rollback on those commits List savepoints = table.getCompletedSavepointTimeline().getInstants() diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index f8647e5cbd68a..eb836d0d41425 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -150,7 +150,7 @@ private Stream getInstantsToArchive(JavaSparkContext jsc) { //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify // with logic above to avoid Stream.concats - HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); + HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); Optional oldestPendingCompactionInstant = table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 07bc368e44332..67aa6b2ef853c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -53,7 +53,7 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable getSmallFiles(String partitionPath) { // smallFiles only for partitionPath List smallFileLocations = new ArrayList<>(); - HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + HoodieTimeline commitTimeline = getCompletedCommitsTimeline(); if (!commitTimeline.empty()) { // if we have some commits HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 849dbae45caf1..de1e563e9331e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -327,7 +327,7 @@ protected List getSmallFiles(String partitionPath) { List smallFileLocations = new ArrayList<>(); // Init here since this class (and member variables) might not have been initialized - HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + HoodieTimeline commitTimeline = getCompletedCommitsTimeline(); // Find out all eligible small file slices if (!commitTimeline.empty()) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 10ef94ed25f3d..2f51cbc8c77e1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -111,14 +111,14 @@ public Configuration getHadoopConf() { * Get the view of the file system for this table */ public TableFileSystemView getFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline()); + return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline()); } /** * Get the read optimized view of the file system for this table */ public TableFileSystemView.ReadOptimizedView getROFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline()); + return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline()); } /** @@ -136,11 +136,18 @@ public TableFileSystemView getCompletedFileSystemView() { return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline()); } + /** + * Get only the completed (no-inflights) commit + deltacommit timeline + */ + public HoodieTimeline getCompletedCommitsTimeline() { + return metaClient.getCommitsTimeline().filterCompletedInstants(); + } + /** * Get only the completed (no-inflights) commit timeline */ public HoodieTimeline getCompletedCommitTimeline() { - return metaClient.getCommitsTimeline().filterCompletedInstants(); + return metaClient.getCommitTimeline().filterCompletedInstants(); } /** diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java index c13c0c43e6364..dca6d9d69660a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -493,7 +493,7 @@ private List createNextDeltaCommit(String instantTime, List getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(table.getMetaClient().getFs(), cfg.getBasePath()); HoodieTableFileSystemView - view = new HoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitTimeline(), allFiles); + view = new HoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitsTimeline(), allFiles); List dataFilesToRead = view.getLatestDataFiles().collect(Collectors.toList()); return dataFilesToRead; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 03f895b27e09a..f3a9616dd5d91 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -126,8 +126,8 @@ private String insertFirstBigBatchForClientCleanerTest( // Should have 100 records in table (check using Index), all in locations marked at commit HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); - assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); + assertFalse(table.getCompletedCommitsTimeline().empty()); + String commitTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp(); assertFalse(table.getCompletedCleanTimeline().empty()); assertEquals("The clean instant should be the same as the commit instant", commitTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); @@ -380,7 +380,7 @@ private void testInsertAndCleanByCommits( HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc); - HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); + HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline(); Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); if (earliestRetainedCommit.isPresent()) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 1b86180fad0ea..f651b8ef1859a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -176,7 +176,7 @@ public void testSimpleInsertAndUpdate() throws Exception { Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -210,7 +210,7 @@ public void testSimpleInsertAndUpdate() throws Exception { client.compact(compactionCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); @@ -283,7 +283,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -320,7 +320,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { assertFalse(commit.isPresent()); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); @@ -380,7 +380,7 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCompletedCommitTimeline(), allFiles); + hoodieTable.getCompletedCommitsTimeline(), allFiles); final String absentCommit = newCommitTime; assertFalse(roView.getLatestDataFiles().filter(file -> { @@ -430,7 +430,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -504,7 +504,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // check that the number of records read is still correct after rollback operation @@ -599,7 +599,7 @@ public void testUpsertPartitioner() throws Exception { Map parquetFileIdToSize = dataFilesToRead.collect( Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); List dataFilesList = dataFilesToRead.collect(Collectors.toList()); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", From e3890787d23b7cab1f6ab0ca04a2c9bd706b1dc7 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Tue, 22 Jan 2019 13:04:15 -0800 Subject: [PATCH 142/374] Fixing behavior of Merge/CreateHandle for invalid/wrong schema records --- .../func/CopyOnWriteLazyInsertIterable.java | 44 +++++++++++-------- .../func/MergeOnReadLazyInsertIterable.java | 11 ++--- .../uber/hoodie/io/HoodieAppendHandle.java | 4 -- .../uber/hoodie/io/HoodieCreateHandle.java | 33 +++++++------- .../com/uber/hoodie/io/HoodieIOHandle.java | 17 +++++++ .../com/uber/hoodie/io/HoodieMergeHandle.java | 5 --- .../func/TestBoundedInMemoryExecutor.java | 7 +-- .../hoodie/func/TestBoundedInMemoryQueue.java | 38 ++++++++-------- 8 files changed, 86 insertions(+), 73 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java index 5311f07a2bfaf..657c7869de469 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java @@ -26,7 +26,6 @@ import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; -import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -38,7 +37,6 @@ import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.spark.TaskContext; -import scala.Tuple2; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -61,20 +59,30 @@ public CopyOnWriteLazyInsertIterable(Iterator> sortedRecordItr, this.hoodieTable = hoodieTable; } + // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread. + static class HoodieInsertValueGenResult { + public T record; + public Optional insertValue; + // It caches the exception seen while fetching insert value. + public Optional exception = Optional.empty(); + + public HoodieInsertValueGenResult(T record, Schema schema) { + this.record = record; + try { + this.insertValue = record.getData().getInsertValue(schema); + } catch (Exception e) { + this.exception = Optional.of(e); + } + } + } + /** * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some * expensive operations of transformation to the reader thread. */ static Function, - Tuple2, Optional>> getTransformFunction(Schema schema) { - return hoodieRecord -> { - try { - return new Tuple2, Optional>(hoodieRecord, - hoodieRecord.getData().getInsertValue(schema)); - } catch (IOException e) { - throw new HoodieException(e); - } - }; + HoodieInsertValueGenResult> getTransformFunction(Schema schema) { + return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema); } @Override @@ -85,7 +93,7 @@ protected void start() { protected List computeNext() { // Executor service used for launching writer thread. BoundedInMemoryExecutor, - Tuple2, Optional>, List> bufferedIteratorExecutor = null; + HoodieInsertValueGenResult, List> bufferedIteratorExecutor = null; try { final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); bufferedIteratorExecutor = @@ -117,14 +125,14 @@ protected CopyOnWriteInsertHandler getInsertHandler() { * writes to one or more create-handles */ protected class CopyOnWriteInsertHandler extends - BoundedInMemoryQueueConsumer, Optional>, List> { + BoundedInMemoryQueueConsumer, List> { protected final List statuses = new ArrayList<>(); protected HoodieIOHandle handle; @Override - protected void consumeOneRecord(Tuple2, Optional> payload) { - final HoodieRecord insertPayload = payload._1(); + protected void consumeOneRecord(HoodieInsertValueGenResult payload) { + final HoodieRecord insertPayload = payload.record; // clean up any partial failures if (!partitionsCleaned.contains(insertPayload.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with @@ -141,16 +149,16 @@ protected void consumeOneRecord(Tuple2, Optional> .randomUUID().toString()); } - if (handle.canWrite(payload._1())) { + if (handle.canWrite(payload.record)) { // write the payload, if the handle has capacity - handle.write(insertPayload, payload._2()); + handle.write(insertPayload, payload.insertValue, payload.exception); } else { // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID .randomUUID().toString()); - handle.write(insertPayload, payload._2()); // we should be able to write 1 payload. + handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload. } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java index a6b5889766a78..b4eea0f1b0df7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java @@ -25,9 +25,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Optional; -import org.apache.avro.generic.IndexedRecord; -import scala.Tuple2; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -49,8 +46,8 @@ protected CopyOnWriteInsertHandler getInsertHandler() { protected class MergeOnReadInsertHandler extends CopyOnWriteInsertHandler { @Override - protected void consumeOneRecord(Tuple2, Optional> payload) { - final HoodieRecord insertPayload = payload._1(); + protected void consumeOneRecord(HoodieInsertValueGenResult payload) { + final HoodieRecord insertPayload = payload.record; List statuses = new ArrayList<>(); // lazily initialize the handle, for the first time if (handle == null) { @@ -58,14 +55,14 @@ protected void consumeOneRecord(Tuple2, Optional> } if (handle.canWrite(insertPayload)) { // write the payload, if the handle has capacity - handle.write(insertPayload, payload._2); + handle.write(insertPayload, payload.insertValue, payload.exception); } else { // handle is full. handle.close(); statuses.add(handle.getWriteStatus()); // Need to handle the rejected payload & open new handle handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable); - handle.write(insertPayload, payload._2); // we should be able to write 1 payload. + handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload. } } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 04f715bab8bb8..516ab5b784da5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -33,7 +33,6 @@ import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.HoodieAvroUtils; -import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieAppendException; import com.uber.hoodie.exception.HoodieUpsertException; @@ -63,7 +62,6 @@ public class HoodieAppendHandle extends HoodieIOH private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); // This acts as the sequenceID for records written private static AtomicLong recordIndex = new AtomicLong(1); - private final WriteStatus writeStatus; private final String fileId; // Buffer for holding records in memory before they are flushed to disk private List recordList = new ArrayList<>(); @@ -98,9 +96,7 @@ public class HoodieAppendHandle extends HoodieIOH public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String fileId, Iterator> recordItr) { super(config, commitTime, hoodieTable); - WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieDeltaWriteStat()); - this.writeStatus = writeStatus; this.fileId = fileId; this.fileSystemView = hoodieTable.getRTFileSystemView(); this.recordItr = recordItr; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index c82f8a2f2fb42..c56d30c7e2789 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -24,7 +24,6 @@ import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.util.FSUtils; -import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieInsertException; import com.uber.hoodie.io.storage.HoodieStorageWriter; @@ -43,7 +42,6 @@ public class HoodieCreateHandle extends HoodieIOH private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class); - private final WriteStatus status; private final HoodieStorageWriter storageWriter; private final Path path; private Path tempPath = null; @@ -55,14 +53,13 @@ public class HoodieCreateHandle extends HoodieIOH public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String partitionPath, String fileId) { super(config, commitTime, hoodieTable); - this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName()); - status.setFileId(fileId); - status.setPartitionPath(partitionPath); + writeStatus.setFileId(fileId); + writeStatus.setPartitionPath(partitionPath); final int sparkPartitionId = TaskContext.getPartitionId(); - this.path = makeNewPath(partitionPath, sparkPartitionId, status.getFileId()); + this.path = makeNewPath(partitionPath, sparkPartitionId, writeStatus.getFileId()); if (config.shouldUseTempFolderForCopyOnWriteForCreate()) { - this.tempPath = makeTempPath(partitionPath, sparkPartitionId, status.getFileId(), + this.tempPath = makeTempPath(partitionPath, sparkPartitionId, writeStatus.getFileId(), TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); } @@ -87,7 +84,7 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTab @Override public boolean canWrite(HoodieRecord record) { - return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath()); + return storageWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath()); } /** @@ -99,13 +96,13 @@ public void write(HoodieRecord record, Optional avroRecord) { if (avroRecord.isPresent()) { storageWriter.writeAvroWithMetadata(avroRecord.get(), record); // update the new location of record, so we know where to find it next - record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId())); + record.setNewLocation(new HoodieRecordLocation(commitTime, writeStatus.getFileId())); recordsWritten++; insertRecordsWritten++; } else { recordsDeleted++; } - status.markSuccess(record, recordMetadata); + writeStatus.markSuccess(record, recordMetadata); // deflate record payload after recording success. This will help users access payload as a // part of marking // record successful. @@ -113,7 +110,7 @@ public void write(HoodieRecord record, Optional avroRecord) { } catch (Throwable t) { // Not throwing exception from here, since we don't want to fail the entire job // for a single record - status.markFailure(record, t, recordMetadata); + writeStatus.markFailure(record, t, recordMetadata); logger.error("Error writing record " + record, t); } } @@ -135,7 +132,7 @@ public void write() { @Override public WriteStatus getWriteStatus() { - return status; + return writeStatus; } /** @@ -143,27 +140,27 @@ public WriteStatus getWriteStatus() { */ @Override public WriteStatus close() { - logger.info("Closing the file " + status.getFileId() + " as we are done with all the records " + logger.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten); try { storageWriter.close(); HoodieWriteStat stat = new HoodieWriteStat(); - stat.setPartitionPath(status.getPartitionPath()); + stat.setPartitionPath(writeStatus.getPartitionPath()); stat.setNumWrites(recordsWritten); stat.setNumDeletes(recordsDeleted); stat.setNumInserts(insertRecordsWritten); stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); - stat.setFileId(status.getFileId()); + stat.setFileId(writeStatus.getFileId()); stat.setPaths(new Path(config.getBasePath()), path, tempPath); stat.setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath())); - stat.setTotalWriteErrors(status.getFailedRecords().size()); + stat.setTotalWriteErrors(writeStatus.getFailedRecords().size()); RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalCreateTime(timer.endTimer()); stat.setRuntimeStats(runtimeStats); - status.setStat(stat); + writeStatus.setStat(stat); - return status; + return writeStatus; } catch (IOException e) { throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 67aa6b2ef853c..af8574fd29c87 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -24,6 +24,7 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.HoodieTimer; +import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; @@ -47,6 +48,7 @@ public abstract class HoodieIOHandle { protected final Schema schema; protected HoodieTimeline hoodieTimeline; protected HoodieTimer timer; + protected final WriteStatus writeStatus; public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { this.commitTime = commitTime; @@ -56,6 +58,7 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable insertValue) { // NO_OP } + /** + * Perform the actual writing of the given record into the backing file. + */ + public void write(HoodieRecord record, Optional avroRecord, Optional exception) { + Optional recordMetadata = record.getData().getMetadata(); + if (exception.isPresent() && exception.get() instanceof Throwable) { + // Not throwing exception from here, since we don't want to fail the entire job for a single record + writeStatus.markFailure(record, exception.get(), recordMetadata); + logger.error("Error writing record " + record, exception.get()); + } else { + write(record, avroRecord); + } + } + public abstract WriteStatus close(); public abstract WriteStatus getWriteStatus(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 210f311bcc466..0700b4cbb71c8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -28,7 +28,6 @@ import com.uber.hoodie.common.util.DefaultSizeEstimator; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; -import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; @@ -54,7 +53,6 @@ public class HoodieMergeHandle extends HoodieIOHa private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class); - private WriteStatus writeStatus; private Map> keyToNewRecords; private Set writtenRecordKeys; private HoodieStorageWriter storageWriter; @@ -91,10 +89,7 @@ public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTabl */ private void init(String fileId, String partitionPath, Optional dataFileToBeMerged) { this.writtenRecordKeys = new HashSet<>(); - - WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieWriteStat()); - this.writeStatus = writeStatus; try { //TODO: dataFileToBeMerged must be optional. Will be fixed by Nishith's changes to support insert to log-files String latestValidFilePath = dataFileToBeMerged.get().getFileName(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java index b629b837fe53c..886156437629f 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult; import java.util.List; import java.util.Optional; import org.apache.avro.generic.IndexedRecord; @@ -55,13 +56,13 @@ public void testExecutor() throws Exception { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); - BoundedInMemoryQueueConsumer>, Integer> consumer = - new BoundedInMemoryQueueConsumer>, Integer>() { + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { private int count = 0; @Override - protected void consumeOneRecord(Tuple2> record) { + protected void consumeOneRecord(HoodieInsertValueGenResult record) { count++; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java index 4092f10a451a3..1add47af402a5 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer; import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -78,9 +79,8 @@ public void afterTest() { public void testRecordReading() throws Exception { final int numRecords = 128; final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - final BoundedInMemoryQueue>> queue = new BoundedInMemoryQueue(FileUtils.ONE_KB, - getTransformFunction(HoodieTestDataGenerator.avroSchema)); + final BoundedInMemoryQueue> queue = + new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema)); // Produce Future resFuture = executorService.submit(() -> { @@ -94,12 +94,12 @@ public void testRecordReading() throws Exception { final HoodieRecord originalRecord = originalRecordIterator.next(); final Optional originalInsertValue = originalRecord.getData() .getInsertValue(HoodieTestDataGenerator.avroSchema); - final Tuple2> payload = queue.iterator().next(); + final HoodieInsertValueGenResult payload = queue.iterator().next(); // Ensure that record ordering is guaranteed. - Assert.assertEquals(originalRecord, payload._1()); + Assert.assertEquals(originalRecord, payload.record); // cached insert value matches the expected insert value. Assert.assertEquals(originalInsertValue, - payload._1().getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); + payload.record.getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); recordsRead++; } Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext()); @@ -119,7 +119,7 @@ public void testCompositeProducerRecordReading() throws Exception { final int numProducers = 40; final List> recs = new ArrayList<>(); - final BoundedInMemoryQueue>> queue = + final BoundedInMemoryQueue> queue = new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema)); // Record Key to @@ -143,7 +143,7 @@ public void testCompositeProducerRecordReading() throws Exception { if (i % 2 == 0) { producers.add(new IteratorBasedQueueProducer<>(r.iterator())); } else { - producers.add(new FunctionBasedQueueProducer((buf) -> { + producers.add(new FunctionBasedQueueProducer<>((buf) -> { Iterator itr = r.iterator(); while (itr.hasNext()) { try { @@ -185,8 +185,8 @@ public void testCompositeProducerRecordReading() throws Exception { // Read recs and ensure we have covered all producer recs. while (queue.iterator().hasNext()) { - final Tuple2> payload = queue.iterator().next(); - final HoodieRecord rec = payload._1(); + final HoodieInsertValueGenResult payload = queue.iterator().next(); + final HoodieRecord rec = payload.record; Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); Integer lastSeenPos = lastSeenMap.get(producerPos._1()); countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1); @@ -212,12 +212,13 @@ public void testMemoryLimitForBuffering() throws Exception { final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); // maximum number of records to keep in memory. final int recordLimit = 5; - final SizeEstimator>> sizeEstimator = + final SizeEstimator> sizeEstimator = new DefaultSizeEstimator<>(); - final long objSize = sizeEstimator.sizeEstimate( - getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0))); + HoodieInsertValueGenResult payload = getTransformFunction(HoodieTestDataGenerator.avroSchema) + .apply(hoodieRecords.get(0)); + final long objSize = sizeEstimator.sizeEstimate(payload); final long memoryLimitInBytes = recordLimit * objSize; - final BoundedInMemoryQueue>> queue = + final BoundedInMemoryQueue> queue = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); @@ -236,8 +237,8 @@ public void testMemoryLimitForBuffering() throws Exception { Assert.assertEquals(recordLimit - 1, queue.samplingRecordCounter.get()); // try to read 2 records. - Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next()._1()); - Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next()._1()); + Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next().record); + Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next().record); // waiting for permits to expire. while (!isQueueFull(queue.rateLimiter)) { @@ -263,8 +264,9 @@ public void testException() throws Exception { final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit - final long objSize = sizeEstimator.sizeEstimate( - getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0))); + HoodieInsertValueGenResult payload = getTransformFunction(HoodieTestDataGenerator.avroSchema) + .apply(hoodieRecords.get(0)); + final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue)); final long memoryLimitInBytes = 4 * objSize; // first let us throw exception from queueIterator reader and test that queueing thread From 4ed53310e0768311d4ae214160543a517dc03d98 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 29 Jan 2019 10:01:25 +0800 Subject: [PATCH 143/374] Using immutable map instead of mutables to generate parameters --- hoodie-spark/pom.xml | 18 ++++++ .../scala/com/uber/hoodie/DefaultSource.scala | 29 +++------ .../uber/hoodie/HoodieSparkSqlWriter.scala | 62 +++++++++---------- .../hoodie/HoodieSparkSqlWriterSuite.scala | 41 ++++++++++++ 4 files changed, 95 insertions(+), 55 deletions(-) create mode 100644 hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 25d1fa7abbd74..c4869f3b22799 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -130,6 +130,24 @@ org.apache.rat apache-rat-plugin + + org.scalatest + scalatest-maven-plugin + 1.0 + + ${project.build.directory}/surefire-reports + . + TestSuite.txt + + + + test + + test + + + + diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index dcf7628bd989e..c6fbd82e03625 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -18,20 +18,15 @@ package com.uber.hoodie -import java.util.concurrent.ConcurrentHashMap - import com.uber.hoodie.DataSourceReadOptions._ import com.uber.hoodie.exception.HoodieException import org.apache.log4j.LogManager +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} - -import scala.collection.JavaConversions._ -import scala.collection.mutable /** * Hoodie Spark Datasource, for reading and writing hoodie datasets @@ -51,22 +46,12 @@ class DefaultSource extends RelationProvider createRelation(sqlContext, parameters, null) } - /** - * Add default options for unspecified read options keys. - * - * @param parameters - * @return - */ - def parametersWithReadDefaults(parameters: Map[String, String]): mutable.Map[String, String] = { - val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters)) - defaultsMap.putIfAbsent(VIEW_TYPE_OPT_KEY, DEFAULT_VIEW_TYPE_OPT_VAL) - mapAsScalaMap(defaultsMap) - } - override def createRelation(sqlContext: SQLContext, optParams: Map[String, String], schema: StructType): BaseRelation = { - val parameters = parametersWithReadDefaults(optParams) + // Add default options for unspecified read options keys. + val parameters = Map(VIEW_TYPE_OPT_KEY -> DEFAULT_VIEW_TYPE_OPT_VAL) ++: optParams + val path = parameters.get("path") if (path.isEmpty) { throw new HoodieException("'path' must be specified.") @@ -92,7 +77,7 @@ class DefaultSource extends RelationProvider sparkSession = sqlContext.sparkSession, userSpecifiedSchema = Option(schema), className = "parquet", - options = parameters.toMap) + options = parameters) .resolveRelation() } } @@ -102,7 +87,7 @@ class DefaultSource extends RelationProvider optParams: Map[String, String], df: DataFrame): BaseRelation = { - val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams).toMap + val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams) HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df) createRelation(sqlContext, parameters, df.schema) } @@ -111,7 +96,7 @@ class DefaultSource extends RelationProvider optParams: Map[String, String], partitionColumns: Seq[String], outputMode: OutputMode): Sink = { - val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams).toMap + val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams) new HoodieStreamingSink( sqlContext, parameters, diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala index fe3507e081269..4092fe3c13675 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala @@ -18,12 +18,8 @@ package com.uber.hoodie import java.util -import java.util.concurrent.ConcurrentHashMap import java.util.Optional -import scala.collection.JavaConversions._ -import scala.collection.mutable.ListBuffer - import com.uber.hoodie.DataSourceWriteOptions._ import com.uber.hoodie.common.table.HoodieTableMetaClient import com.uber.hoodie.common.util.{FSUtils, TypedProperties} @@ -36,9 +32,10 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.log4j.LogManager import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} -import scala.collection.mutable +import scala.collection.JavaConversions._ +import scala.collection.mutable.ListBuffer private[hoodie] object HoodieSparkSqlWriter { @@ -98,7 +95,7 @@ private[hoodie] object HoodieSparkSqlWriter { gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]] DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY)) - }).toJavaRDD(); + }).toJavaRDD() val jsc = new JavaSparkContext(sparkContext) @@ -117,7 +114,7 @@ private[hoodie] object HoodieSparkSqlWriter { return (true, None) } - val basePath = new Path(parameters.get("path").get) + val basePath = new Path(parameters("path")) val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) var exists = fs.exists(basePath) @@ -172,7 +169,7 @@ private[hoodie] object HoodieSparkSqlWriter { log.info("Commit " + commitTime + " failed!") } - val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false) + val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean) val syncHiveSucess = if (hiveSyncEnabled) { log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")") val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration) @@ -207,30 +204,29 @@ private[hoodie] object HoodieSparkSqlWriter { * @param parameters * @return */ - def parametersWithWriteDefaults(parameters: Map[String, String]): mutable.Map[String, String] = { - val defaultsMap = new ConcurrentHashMap[String, String](mapAsJavaMap(parameters)) - defaultsMap.putIfAbsent(OPERATION_OPT_KEY, DEFAULT_OPERATION_OPT_VAL) - defaultsMap.putIfAbsent(STORAGE_TYPE_OPT_KEY, DEFAULT_STORAGE_TYPE_OPT_VAL) - defaultsMap.putIfAbsent(PRECOMBINE_FIELD_OPT_KEY, DEFAULT_PRECOMBINE_FIELD_OPT_VAL) - defaultsMap.putIfAbsent(PAYLOAD_CLASS_OPT_KEY, DEFAULT_PAYLOAD_OPT_VAL) - defaultsMap.putIfAbsent(RECORDKEY_FIELD_OPT_KEY, DEFAULT_RECORDKEY_FIELD_OPT_VAL) - defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL) - defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) - defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL) - defaultsMap.putIfAbsent(INSERT_DROP_DUPS_OPT_KEY, DEFAULT_INSERT_DROP_DUPS_OPT_VAL) - defaultsMap.putIfAbsent(STREAMING_RETRY_CNT_OPT_KEY, DEFAULT_STREAMING_RETRY_CNT_OPT_VAL) - defaultsMap.putIfAbsent(STREAMING_RETRY_INTERVAL_MS_OPT_KEY, DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL) - defaultsMap.putIfAbsent(STREAMING_IGNORE_FAILED_BATCH_OPT_KEY, DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_USER_OPT_KEY, DEFAULT_HIVE_USER_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_PASS_OPT_KEY, DEFAULT_HIVE_PASS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_URL_OPT_KEY, DEFAULT_HIVE_URL_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_PARTITION_FIELDS_OPT_KEY, DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL) - defaultsMap.putIfAbsent(HIVE_ASSUME_DATE_PARTITION_OPT_KEY, DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL) - mapAsScalaMap(defaultsMap) + def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = { + Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL, + STORAGE_TYPE_OPT_KEY -> DEFAULT_STORAGE_TYPE_OPT_VAL, + PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL, + PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL, + RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL, + PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL, + KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, + COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL, + INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL, + STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL, + STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL, + STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL, + HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL, + HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL, + HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL, + HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL, + HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL, + HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL, + HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL, + HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL, + HIVE_ASSUME_DATE_PARTITION_OPT_KEY -> DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL + ) ++: parameters } def toProperties(params: Map[String, String]): TypedProperties = { diff --git a/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala b/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala new file mode 100644 index 0000000000000..cd3c21343aa95 --- /dev/null +++ b/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ +package com.uber.hoodie + +import org.scalatest.{FunSuite, Matchers} +import DataSourceWriteOptions._ + +class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { + + test("Parameters With Write Defaults") { + val originals = HoodieSparkSqlWriter.parametersWithWriteDefaults(Map.empty) + val rhsKey = "hoodie.right.hand.side.key" + val rhsVal = "hoodie.right.hand.side.val" + val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, STORAGE_TYPE_OPT_KEY -> MOR_STORAGE_TYPE_OPT_VAL, rhsKey -> rhsVal) + val modified = HoodieSparkSqlWriter.parametersWithWriteDefaults(modifier) + val matcher = (k: String, v: String) => modified(k) should be(v) + + originals foreach { + case (OPERATION_OPT_KEY, _) => matcher(OPERATION_OPT_KEY, INSERT_OPERATION_OPT_VAL) + case (STORAGE_TYPE_OPT_KEY, _) => matcher(STORAGE_TYPE_OPT_KEY, MOR_STORAGE_TYPE_OPT_VAL) + case (`rhsKey`, _) => matcher(rhsKey, rhsVal) + case (k, v) => matcher(k, v) + } + } + +} From eceddad78e9aefe7c23279f8ff4b08feccf5ad65 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 6 Feb 2019 13:50:16 -0800 Subject: [PATCH 144/374] Updating new slack signup link --- docs/community.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/community.md b/docs/community.md index 0eccac22dcf0e..89f50de01aa84 100644 --- a/docs/community.md +++ b/docs/community.md @@ -13,7 +13,7 @@ issues or pull requests against this repo. Before you do so, please sign the Also, be sure to write unit tests for your bug fix or feature to show that it works as expected. If the reviewer feels this contributions needs to be in the release notes, please add it to CHANGELOG.md as well. -If you want to participate in day-day conversations, please join our [slack group](https://hoodielib.slack.com/x-147852474016-157730502112/signup). +If you want to participate in day-day conversations, please join our [slack group](https://join.slack.com/t/apache-hudi/signup). If you are from select pre-listed email domains, you can self signup. Others, please add your email onto this [issue](https://github.com/uber/hoodie/issues/143) ## Becoming a Committer From 376259350e9236e1d21936db203360ca7488e3c8 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Wed, 10 Oct 2018 10:31:34 -0700 Subject: [PATCH 145/374] 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 --- docs/incremental_processing.md | 8 +- docs/quickstart.md | 4 +- .../cli/commands/ArchivedCommitsCommand.java | 88 +++++++ hoodie-client/pom.xml | 42 +--- .../uber/hoodie/metrics/HoodieMetrics.java | 16 +- .../java/com/uber/hoodie/metrics/Metrics.java | 16 ++ .../common/HoodieTestDataGenerator.java | 5 +- .../uber/hoodie/io/TestHoodieCompactor.java | 3 +- .../hoodie/metrics/TestHoodieMetrics.java | 3 +- .../hoodie/table/TestMergeOnReadTable.java | 4 +- .../src/main/avro/HoodieCommitMetadata.avsc | 42 ++-- .../hoodie/common/model/HoodieRecord.java | 10 + .../util/DFSPropertiesConfiguration.java | 18 +- .../hoodie/common/util/HoodieAvroUtils.java | 12 +- .../hoodie/common/util/TypedProperties.java | 10 + .../util/collection/TestDiskBasedMap.java | 209 +++++++++++++++ hoodie-hadoop-mr/pom.xml | 70 ++---- hoodie-hive/pom.xml | 84 ++----- .../uber/hoodie/hive/HoodieHiveClient.java | 26 +- .../hive/MultiPartKeysValueExtractor.java | 12 +- .../hoodie/hive/PartitionValueExtractor.java | 3 +- ...lashEncodedDayPartitionValueExtractor.java | 11 +- hoodie-spark/pom.xml | 87 ++----- .../java/com/uber/hoodie/DataSourceUtils.java | 40 +++ .../com/uber/hoodie/AvroConversionUtils.scala | 206 ++++++++++++++- .../com/uber/hoodie/DataSourceOptions.scala | 9 +- .../com/uber/hoodie/IncrementalRelation.scala | 37 ++- .../src/test/scala/DataSourceTest.scala | 1 - hoodie-utilities/pom.xml | 138 +++++----- .../uber/hoodie/utilities/HoodieCleaner.java | 115 +++++++++ .../uber/hoodie/utilities/UtilHelpers.java | 33 ++- .../deltastreamer/HoodieDeltaStreamer.java | 230 ++++++++++++++--- .../HoodieDeltaStreamerMetrics.java | 61 +++++ .../deltastreamer/SourceFormatAdapter.java | 112 +++++++++ .../schema/RowBasedSchemaProvider.java | 25 ++ .../schema/SchemaRegistryProvider.java | 28 ++- .../utilities/sources/AvroDFSSource.java | 26 +- .../utilities/sources/AvroKafkaSource.java | 42 +++- .../hoodie/utilities/sources/AvroSource.java | 36 +++ .../utilities/sources/HiveIncrPullSource.java | 25 +- .../utilities/sources/HoodieIncrSource.java | 144 +++++++++++ .../hoodie/utilities/sources/InputBatch.java | 54 ++++ .../utilities/sources/JsonDFSSource.java | 28 ++- .../utilities/sources/JsonKafkaSource.java | 37 ++- .../hoodie/utilities/sources/JsonSource.java | 35 +++ .../hoodie/utilities/sources/RowSource.java | 51 ++++ .../uber/hoodie/utilities/sources/Source.java | 55 +++- .../sources/{ => helpers}/AvroConvertor.java | 6 +- .../DFSPathSelector.java} | 33 +-- .../sources/helpers/IncrSourceHelper.java | 88 +++++++ .../KafkaOffsetGen.java} | 62 ++--- .../transform/IdentityTransformer.java | 37 +++ .../transform/SqlQueryBasedTransformer.java | 66 +++++ .../utilities/transform/Transformer.java | 43 ++++ .../utilities/TestHoodieDeltaStreamer.java | 238 +++++++++++++++++- .../hoodie/utilities/UtilitiesTestBase.java | 60 +++++ .../utilities/sources/TestDFSSource.java | 53 ++-- .../utilities/sources/TestDataSource.java | 16 +- .../utilities/sources/TestKafkaSource.java | 51 ++-- .../sql-transformer.properties | 19 ++ .../delta-streamer-config/target.avsc | 37 +++ packaging/hoodie-hadoop-mr-bundle/pom.xml | 154 ++++-------- packaging/hoodie-hive-bundle/pom.xml | 89 ++----- packaging/hoodie-spark-bundle/pom.xml | 89 ++----- pom.xml | 157 ++++-------- 65 files changed, 2745 insertions(+), 904 deletions(-) create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCleaner.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/RowBasedSchemaProvider.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroSource.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonSource.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java rename hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/{ => helpers}/AvroConvertor.java (93%) rename hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/{DFSSource.java => helpers/DFSPathSelector.java} (76%) create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java rename hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/{KafkaSource.java => helpers/KafkaOffsetGen.java} (84%) create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/IdentityTransformer.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/SqlQueryBasedTransformer.java create mode 100644 hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/Transformer.java create mode 100644 hoodie-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties create mode 100644 hoodie-utilities/src/test/resources/delta-streamer-config/target.avsc diff --git a/docs/incremental_processing.md b/docs/incremental_processing.md index 973875a0bb972..9a65ccfe16ba3 100644 --- a/docs/incremental_processing.md +++ b/docs/incremental_processing.md @@ -85,8 +85,12 @@ Usage:
    [options] exist first time around. If exists, expected to be a hoodie dataset) * --target-table 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) ``` diff --git a/docs/quickstart.md b/docs/quickstart.md index 41ec9a96d0f6b..bb7be6edda54b 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -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 | | ---- | ----- | ---- | ---- | -| 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.7.3 | Apache hive-1.2.1 | spark-2.[1-3].x | Use "mvn clean install -DskipTests" | @@ -1244,4 +1244,4 @@ cd docker [INFO] Finished at: 2018-09-10T17:47:37-07:00 [INFO] Final Memory: 236M/1848M [INFO] ------------------------------------------------------------------------ -``` \ No newline at end of file +``` diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 5f4d5de0d6a1d..49fcc591961a2 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -17,6 +17,7 @@ package com.uber.hoodie.cli.commands; 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.HoodiePrintHelper; import com.uber.hoodie.cli.TableHeader; @@ -32,6 +33,7 @@ import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.springframework.shell.core.CommandMarker; @@ -48,6 +50,92 @@ public boolean isShowArchivedCommitAvailable() { 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 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 readRecords = new ArrayList<>(); + //read the avro blocks + while (reader.hasNext()) { + HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); + List records = blk.getRecords(); + readRecords.addAll(records); + } + List 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 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") public String showCommits( @CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata", unspecifiedDefaultValue = "true") diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index eccfa802c58b8..c813ef24a499b 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -171,6 +171,13 @@ spark-sql_2.11 + + ${hive.groupid} + hive-exec + ${hive.version} + test + + org.apache.hbase hbase-client @@ -218,39 +225,4 @@ - - - hive12 - - - !hive11 - - - - - ${hive12.groupid} - hive-exec - ${hive12.version} - test - - - - - hive11 - - - hive11 - - - - - ${hive11.groupid} - hive-exec - ${hive11.version} - test - - - - - diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index 5adf45fe06e63..625e73313848e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -16,8 +16,8 @@ package com.uber.hoodie.metrics; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.MetricRegistry; +import static com.uber.hoodie.metrics.Metrics.registerGauge; + import com.codahale.metrics.Timer; import com.google.common.annotations.VisibleForTesting; import com.uber.hoodie.common.model.HoodieCommitMetadata; @@ -177,18 +177,6 @@ String getMetricsName(String action, String 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) () -> 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. */ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java index 924257493592e..e3fcc98c0800f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java @@ -16,17 +16,21 @@ package com.uber.hoodie.metrics; +import com.codahale.metrics.Gauge; import com.codahale.metrics.MetricRegistry; import com.google.common.io.Closeables; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import java.io.Closeable; 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. */ public class Metrics { + private static Logger logger = LogManager.getLogger(Metrics.class); private static volatile boolean initialized = false; private static Metrics metrics = null; @@ -72,6 +76,18 @@ public static synchronized void init(HoodieWriteConfig metricConfig) { initialized = true; } + public static void registerGauge(String metricName, final long value) { + try { + MetricRegistry registry = Metrics.getInstance().getRegistry(); + registry.register(metricName, (Gauge) () -> 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() { return registry; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index 06d66aeb408ab..d1713c5e40367 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -75,7 +75,9 @@ public class HoodieTestDataGenerator { + "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"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 List existingKeysList = new ArrayList<>(); @@ -100,7 +102,6 @@ public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths */ public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException { 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); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index e1a6d66dcb600..ac171d4dcad0b 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -160,7 +160,8 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, updatedRecords); + .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchemaWithMetadataFields, + updatedRecords); // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java b/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java index cb1a439698f87..c562b308a5c1c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java @@ -16,6 +16,7 @@ package com.uber.hoodie.metrics; +import static com.uber.hoodie.metrics.Metrics.registerGauge; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -39,7 +40,7 @@ public void start() throws ConfigurationException { @Test public void testRegisterGauge() { - metrics.registerGauge("metric1", 123L); + registerGauge("metric1", 123L); assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123")); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index f651b8ef1859a..93227c49cb977 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -671,8 +671,8 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, - updatedRecords); + .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), + HoodieTestDataGenerator.avroSchemaWithMetadataFields, updatedRecords); // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index 3f44732005dff..c46825cdcaec2 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -15,47 +15,58 @@ "fields":[ { "name":"fileId", - "type":["null","string"] + "type":["null","string"], + "default" : null }, { "name":"path", - "type":["null","string"] + "type":["null","string"], + "default" : null }, { "name":"prevCommit", - "type":["null","string"] + "type":["null","string"], + "default" : null }, { "name":"numWrites", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"numDeletes", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"numUpdateWrites", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"totalWriteBytes", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"totalWriteErrors", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"partitionPath", - "type":["null","string"] + "type":["null","string"], + "default" : null }, { "name":"totalLogRecords", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"totalLogFiles", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"totalUpdatedRecordsCompacted", @@ -69,15 +80,18 @@ }, { "name":"totalLogBlocks", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"totalCorruptLogBlock", - "type":["null","long"] + "type":["null","long"], + "default" : null }, { "name":"totalRollbackBlocks", - "type":["null","long"] + "type":["null","long"], + "default" : null } ] } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java index cef6e60a2632b..857dcaaa95a38 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -17,7 +17,9 @@ package com.uber.hoodie.common.model; import com.google.common.base.Objects; +import com.google.common.collect.ImmutableList; import java.io.Serializable; +import java.util.List; import java.util.Optional; /** @@ -31,6 +33,14 @@ public class HoodieRecord implements Serializable public static String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path"; public static String FILENAME_METADATA_FIELD = "_hoodie_file_name"; + public static final List HOODIE_META_COLUMNS = + new ImmutableList.Builder().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 */ diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java index 758ba0fcaabd8..f8dad81b2ce92 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DFSPropertiesConfiguration.java @@ -73,6 +73,20 @@ private void visitFile(Path file) { } visitedFiles.add(file.getName()); BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file))); + addProperties(reader); + } catch (IOException ioe) { + log.error("Error reading in properies from dfs", ioe); + throw new IllegalArgumentException("Cannot read properties from dfs", ioe); + } + } + + /** + * Add properties from input stream + * @param reader Buffered Reader + * @throws IOException + */ + public void addProperties(BufferedReader reader) throws IOException { + try { String line; while ((line = reader.readLine()) != null) { if (line.startsWith("#") || line.equals("") || !line.contains("=")) { @@ -85,10 +99,8 @@ private void visitFile(Path file) { props.setProperty(split[0], split[1]); } } + } finally { reader.close(); - } catch (IOException ioe) { - log.error("Error reading in properies from dfs", ioe); - throw new IllegalArgumentException("Cannot read properties from dfs", ioe); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index ae92b00f90262..b76b4aeb1a73a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -37,8 +37,8 @@ import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.BinaryEncoder; -import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.EncoderFactory; import org.codehaus.jackson.JsonNode; @@ -48,6 +48,10 @@ */ public class HoodieAvroUtils { + private static ThreadLocal reuseEncoder = ThreadLocal.withInitial(() -> null); + + private static ThreadLocal reuseDecoder = ThreadLocal.withInitial(() -> null); + // All metadata fields are optional strings. private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList( Schema.create(Schema.Type.NULL), @@ -62,7 +66,8 @@ public static byte[] avroToBytes(GenericRecord record) throws IOException { GenericDatumWriter writer = new GenericDatumWriter<>(record.getSchema()); ByteArrayOutputStream out = new ByteArrayOutputStream(); - BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null); + BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, reuseEncoder.get()); + reuseEncoder.set(encoder); writer.write(record, encoder); encoder.flush(); out.close(); @@ -73,7 +78,8 @@ public static byte[] avroToBytes(GenericRecord record) throws IOException { * Convert serialized bytes back into avro record */ public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException { - Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null); + BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, reuseDecoder.get()); + reuseDecoder.set(decoder); GenericDatumReader reader = new GenericDatumReader(schema); return reader.read(null, decoder); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java index 5674d2382484c..5acca156a72bd 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TypedProperties.java @@ -19,7 +19,10 @@ package com.uber.hoodie.common.util; import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import java.util.Properties; +import java.util.stream.Collectors; /** * Type-aware extension of {@link java.util.Properties} @@ -49,6 +52,13 @@ public String getString(String property, String defaultValue) { return containsKey(property) ? getProperty(property) : defaultValue; } + public List getStringList(String property, String delimiter, List defaultVal) { + if (!containsKey(property)) { + return defaultVal; + } + return Arrays.stream(getProperty(property).split(delimiter)).map(String::trim).collect(Collectors.toList()); + } + public int getInteger(String property) { checkKey(property); return Integer.valueOf(getProperty(property)); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java new file mode 100644 index 0000000000000..9a288af82724d --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -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 iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + + // make sure records have spilled to disk + assertTrue(records.sizeOfFileOnDiskInBytes() > 0); + Iterator> itr = records.iterator(); + List oRecords = new ArrayList<>(); + while (itr.hasNext()) { + HoodieRecord 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 hoodieRecords = SchemaTestUtil + .generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); + Set 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> itr = records.iterator(); + List oRecords = new ArrayList<>(); + while (itr.hasNext()) { + HoodieRecord 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 iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + + // perform some inserts + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); + + long fileSize = records.sizeOfFileOnDiskInBytes(); + // make sure records have spilled to disk + assertTrue(fileSize > 0); + + // generate updates from inserts + List 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> itr = records.iterator(); + while (itr.hasNext()) { + HoodieRecord 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 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 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 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); + } +} diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index fe1df4396a740..6ef41a32d24b0 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -60,6 +60,22 @@ org.apache.hadoop hadoop-hdfs + + ${hive.groupid} + hive-jdbc + ${hive.version} + + + commons-logging + commons-logging + + + + + ${hive.groupid} + hive-exec + ${hive.version} + commons-logging commons-logging @@ -105,58 +121,4 @@ - - - hive12 - - - !hive11 - - - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - - - commons-logging - commons-logging - - - - - ${hive12.groupid} - hive-exec - ${hive12.version} - - - - - hive11 - - - hive11 - - - - - ${hive11.groupid} - hive-jdbc - ${hive11.version} - - - commons-logging - commons-logging - - - - - ${hive11.groupid} - hive-exec - ${hive11.version} - - - - diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index f74e5ea8350b0..da4e9a0d7e299 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -99,6 +99,27 @@ junit junit + + ${hive.groupid} + hive-service + ${hive.version} + + + ${hive.groupid} + hive-jdbc + ${hive.version} + + + ${hive.groupid} + hive-metastore + ${hive.version} + + + ${hive.groupid} + hive-common + ${hive.version} + + org.apache.hadoop hadoop-hdfs @@ -175,67 +196,4 @@ - - - - hive12 - - - !hive11 - - - - - ${hive12.groupid} - hive-service - ${hive12.version} - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - - - ${hive12.groupid} - hive-metastore - ${hive12.version} - - - ${hive12.groupid} - hive-common - ${hive12.version} - - - - - hive11 - - - hive11 - - - - - org.apache.hive - hive-service - ${hive11.version} - - - org.apache.hive - hive-jdbc - ${hive11.version} - - - org.apache.hive - hive-metastore - ${hive11.version} - - - org.apache.hive - hive-common - ${hive11.version} - - - - diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index e3f9cb0feed76..d93939198fb8d 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -90,7 +90,7 @@ public class HoodieHiveClient { private Connection connection; private HoodieTimeline activeTimeline; - HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { + public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { this.syncConfig = cfg; this.fs = fs; this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true); @@ -231,7 +231,7 @@ List getPartitionEvents(List tablePartitions, /** * Scan table partitions */ - List scanTablePartitions() throws TException { + public List scanTablePartitions() throws TException { return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); } @@ -268,7 +268,7 @@ void createTable(MessageType storageSchema, String inputFormatClass, String outp /** * Get the table schema */ - Map getTableSchema() { + public Map getTableSchema() { if (!doesTableExist()) { throw new IllegalArgumentException( "Failed to get schema for table " + syncConfig.tableName + " does not exist"); @@ -435,7 +435,7 @@ private MessageType readSchemaFromDataFile(Path parquetFilePath) throws IOExcept /** * @return true if the configured table exists */ - boolean doesTableExist() { + public boolean doesTableExist() { try { return client.tableExists(syncConfig.databaseName, syncConfig.tableName); } catch (TException e) { @@ -449,7 +449,7 @@ boolean doesTableExist() { * * @param s SQL to execute */ - void updateHiveSQL(String s) { + public void updateHiveSQL(String s) { Statement stmt = null; try { stmt = connection.createStatement(); @@ -468,8 +468,10 @@ private void createHiveConnection() { BasicDataSource ds = new HiveDataSource(); ds.setDriverClassName(HiveDriver.class.getCanonicalName()); ds.setUrl(getHiveJdbcUrlWithDefaultDBName()); - ds.setUsername(syncConfig.hiveUser); - ds.setPassword(syncConfig.hivePass); + if (syncConfig.hiveUser != null) { + ds.setUsername(syncConfig.hiveUser); + ds.setPassword(syncConfig.hivePass); + } LOG.info("Getting Hive Connection from Datasource " + ds); try { this.connection = ds.getConnection(); @@ -520,7 +522,7 @@ public FileSystem getFs() { return fs; } - Optional getLastCommitTimeSynced() { + public Optional getLastCommitTimeSynced() { // Get the last commit time from the TBLproperties try { Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName); @@ -532,7 +534,7 @@ Optional getLastCommitTimeSynced() { } } - void close() { + public void close() { try { if (connection != null) { connection.close(); @@ -548,7 +550,7 @@ void close() { @SuppressWarnings("OptionalUsedAsFieldOrParameterType") List getPartitionsWrittenToSince(Optional lastCommitTimeSynced) { 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 { return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath, syncConfig.assumeDatePartitioning); @@ -573,6 +575,10 @@ List getPartitionsWrittenToSince(Optional lastCommitTimeSynced) } } + List getAllTables(String db) throws Exception { + return client.getAllTables(db); + } + void updateLastCommitTimeSynced() { // Set the last commit time from the TBLproperties String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java index 00475e1e9c8ca..b0bcc59f4eb28 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/MultiPartKeysValueExtractor.java @@ -16,8 +16,10 @@ package com.uber.hoodie.hive; +import com.google.common.base.Preconditions; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; /** * Partition Key extractor treating each value delimited by slash as separate key. @@ -27,6 +29,14 @@ public class MultiPartKeysValueExtractor implements PartitionValueExtractor { @Override public List extractPartitionValuesInPath(String partitionPath) { 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()); } } \ No newline at end of file diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java index 73776c4616c76..b32f7cf0caab6 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java @@ -18,6 +18,7 @@ package com.uber.hoodie.hive; +import java.io.Serializable; import java.util.List; /** @@ -28,7 +29,7 @@ * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path * /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] */ -public interface PartitionValueExtractor { +public interface PartitionValueExtractor extends Serializable { List extractPartitionValuesInPath(String partitionPath); } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java index 893b61e4cef1e..771f2771f70b3 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java @@ -33,12 +33,19 @@ */ public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor { - private final DateTimeFormatter dtfOut; + private transient DateTimeFormatter dtfOut; public SlashEncodedDayPartitionValueExtractor() { this.dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd"); } + private DateTimeFormatter getDtfOut() { + if (dtfOut == null) { + dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd"); + } + return dtfOut; + } + @Override public List extractPartitionValuesInPath(String partitionPath) { // partition path is expected to be in this format yyyy/mm/dd @@ -52,6 +59,6 @@ public List extractPartitionValuesInPath(String partitionPath) { int mm = Integer.parseInt(splits[1]); int dd = Integer.parseInt(splits[2]); DateTime dateTime = new DateTime(year, mm, dd, 0, 0); - return Lists.newArrayList(dtfOut.print(dateTime)); + return Lists.newArrayList(getDtfOut().print(dateTime)); } } diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index c4869f3b22799..d800c1a7218b3 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -221,6 +221,30 @@ commons-configuration2 + + ${hive.groupid} + hive-service + ${hive.version} + + + + ${hive.groupid} + hive-jdbc + ${hive.version} + + + + ${hive.groupid} + hive-metastore + ${hive.version} + + + + ${hive.groupid} + hive-common + ${hive.version} + + com.uber.hoodie hoodie-client @@ -264,67 +288,4 @@ test - - - hive12 - - - !hive11 - - - - - ${hive12.groupid} - hive-service - ${hive12.version} - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - - - ${hive12.groupid} - hive-metastore - ${hive12.version} - - - ${hive12.groupid} - hive-common - ${hive12.version} - - - - - hive11 - - - hive11 - - - - - ${hive11.groupid} - hive-service - ${hive11.version} - - - ${hive11.groupid} - hive-jdbc - ${hive11.version} - - - ${hive11.groupid} - hive-metastore - ${hive11.version} - - - ${hive11.groupid} - hive-common - ${hive11.version} - - - - - diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index b02c36675a2fd..429c43953a446 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -29,8 +29,13 @@ import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieException; 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 java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -86,6 +91,17 @@ public static KeyGenerator createKeyGenerator(String keyGeneratorClass, } } + /** + * 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. */ @@ -169,4 +185,28 @@ public static JavaRDD dropDuplicates(JavaSparkContext jssc, .withProps(parameters).build(); 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; + } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index 75d13e7cfbd74..df3f96438afdf 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -22,13 +22,18 @@ import java.sql.{Date, Timestamp} import java.util import com.databricks.spark.avro.SchemaConverters -import org.apache.avro.generic.GenericData.Record -import org.apache.avro.generic.GenericRecord +import com.databricks.spark.avro.SchemaConverters.IncompatibleSchemaException +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.spark.rdd.RDD 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.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} + +import scala.collection.JavaConverters._ 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, currentRecordNamespace: 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, structName: String, recordNamespace: String): Any => Any = { diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala index 5df7118bd4dcf..9973e4bceecac 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DataSourceOptions.scala @@ -43,7 +43,7 @@ object DataSourceReadOptions { val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental" val VIEW_TYPE_REALTIME_OPT_VAL = "realtime" val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL - + val DEFAULTPUSH_DOWN_FILTERS_OPT_VAL = "" /** * 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" + + /** + * 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" } /** diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index 1f7b47682abc3..0f13c9348a470 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -64,21 +64,33 @@ class IncrementalRelation(val sqlContext: SQLContext, throw new HoodieException(s"Specify the begin instant time to pull from using " + s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}") } + + val lastInstant = commitTimeline.lastInstant().get() + val commitsToReturn = commitTimeline.findInstantsInRange( optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY), - optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, - commitTimeline.lastInstant().get().getTimestamp)) + optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp)) .getInstants.iterator().toList // use schema from a file produced in the latest instant val latestSchema = { + // use last instant if instant range is empty + val instant = commitsToReturn.lastOption.getOrElse(lastInstant) 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() AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema( 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 buildScan(): RDD[Row] = { @@ -92,12 +104,17 @@ class IncrementalRelation(val sqlContext: SQLContext, // will filter out all the files incorrectly. sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class") val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) - sqlContext.read.options(sOpts) - .schema(latestSchema) // avoid AnalysisException for empty input - .parquet(fileIdToFullPath.values.toList: _*) - .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)) - .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)) - .toDF().rdd - + if (fileIdToFullPath.isEmpty) { + 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: _*) + .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)))((e, f) => e.filter(f)) + .toDF().rdd + } } } diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index 2f34beb4ed535..42def5bf11bc7 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -100,7 +100,6 @@ class DataSourceTest extends AssertionsForJUnit { .load(basePath + "/*/*/*/*"); assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated - // Read Incremental View // we have 2 commits, try pulling the first commit (which is not the latest) val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0); diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 8a9feac7f31ef..3cbf468d7c136 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -53,9 +53,10 @@ ${project.build.directory}/dependency-reduced-pom.xml - true + commons-dbcp:commons-dbcp + commons-pool:commons-pool com.uber.hoodie:hoodie-common com.uber.hoodie:hoodie-client com.uber.hoodie:hoodie-spark @@ -76,8 +77,50 @@ com.yammer.metrics:metrics-core com.101tec:zkclient org.apache.kafka:kafka-clients + org.apache.hive:hive-common + org.apache.hive:hive-service + org.apache.hive:hive-metastore + org.apache.hive:hive-jdbc + + + org.apache.commons.dbcp. + com.uber.hoodie.org.apache.commons.dbcp. + + + org.apache.commons.pool. + com.uber.hoodie.org.apache.commons.pool. + + + org.apache.hive.jdbc. + com.uber.hoodie.org.apache.hive.jdbc. + + + org.apache.hadoop.hive.metastore. + com.uber.hoodie.org.apache.hadoop_hive.metastore. + + + org.apache.hive.common. + com.uber.hoodie.org.apache.hive.common. + + + org.apache.hadoop.hive.common. + com.uber.hoodie.org.apache.hadoop_hive.common. + + + org.apache.hadoop.hive.conf. + com.uber.hoodie.org.apache.hadoop_hive.conf. + + + org.apache.hive.service. + com.uber.hoodie.org.apache.hive.service. + + + org.apache.hadoop.hive.service. + com.uber.hoodie.org.apache.hadoop_hive.service. + + @@ -123,6 +166,15 @@ test + + com.uber.hoodie + hoodie-hive + ${project.version} + tests + test-jar + test + + com.uber.hoodie hoodie-spark @@ -154,6 +206,30 @@ + + ${hive.groupid} + hive-exec + ${hive.version} + test + + + + ${hive.groupid} + hive-jdbc + ${hive.version} + standalone + + + org.slf4j + slf4j-api + + + javax.servlet + servlet-api + + + + com.uber.hoodie hoodie-hive @@ -185,6 +261,11 @@ commons-dbcp commons-dbcp + + commons-pool + commons-pool + + org.apache.httpcomponents httpcore @@ -303,59 +384,4 @@ - - - - hive12 - - - !hive11 - - - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - standalone - - - org.slf4j - slf4j-api - - - javax.servlet - servlet-api - - - - - - - hive11 - - - hive11 - - - - - org.apache.hive - hive-jdbc - ${hive11.version} - standalone - - - org.slf4j - slf4j-api - - - javax.servlet - servlet-api - - - - - - diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCleaner.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCleaner.java new file mode 100644 index 0000000000000..7ebca042c97ac --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCleaner.java @@ -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 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(); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index b5008cf31aabc..b24e3a2772d1e 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -30,9 +30,13 @@ import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.utilities.schema.SchemaProvider; 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.InputStream; +import java.io.StringReader; import java.nio.ByteBuffer; +import java.util.List; import java.util.Optional; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -43,6 +47,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; /** * Bunch of helper methods @@ -51,12 +56,12 @@ public class UtilHelpers { private static Logger logger = LogManager.getLogger(UtilHelpers.class); public static Source createSource(String sourceClass, TypedProperties cfg, - JavaSparkContext jssc, SchemaProvider schemaProvider) + JavaSparkContext jssc, SparkSession sparkSession, SchemaProvider schemaProvider) throws IOException { try { return (Source) ReflectionUtils.loadClass(sourceClass, - new Class[]{TypedProperties.class, JavaSparkContext.class, SchemaProvider.class}, - cfg, jssc, schemaProvider); + new Class[]{TypedProperties.class, JavaSparkContext.class, SparkSession.class, SchemaProvider.class}, + cfg, jssc, sparkSession, schemaProvider); } catch (Throwable e) { throw new IOException("Could not load source class " + sourceClass, e); } @@ -65,17 +70,31 @@ public static Source createSource(String sourceClass, TypedProperties cfg, public static SchemaProvider createSchemaProvider(String schemaProviderClass, TypedProperties cfg, JavaSparkContext jssc) throws IOException { try { - return (SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc); + return schemaProviderClass == null ? null : + (SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc); } catch (Throwable 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 overriddenProps) { 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) { throw new HoodieException("Unable to read props file at :" + cfgPath, e); } @@ -109,7 +128,7 @@ private static SparkConf buildSparkConf(String appName, String defaultMaster) { sparkConf.set("spark.eventLog.overwrite", "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.hadoop.mapred.output.compress", "true"); sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true"); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index c3573fdeb0c42..665fcff98f2f1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -18,10 +18,15 @@ 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.JCommander; import com.beust.jcommander.Parameter; import com.beust.jcommander.ParameterException; +import com.codahale.metrics.Timer; +import com.uber.hoodie.AvroConversionUtils; import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.KeyGenerator; @@ -36,32 +41,40 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; 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.HoodieIndexConfig; 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.utilities.HiveIncrementalPuller; import com.uber.hoodie.utilities.UtilHelpers; 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.sources.InputBatch; 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.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Optional; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; 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; /** @@ -81,7 +94,7 @@ public class HoodieDeltaStreamer implements Serializable { /** * 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 @@ -89,6 +102,11 @@ public class HoodieDeltaStreamer implements Serializable { */ private transient SchemaProvider schemaProvider; + /** + * Allows transforming source to target dataset before writing + */ + private transient Transformer transformer; + /** * Extract the key for the target dataset */ @@ -109,16 +127,30 @@ public class HoodieDeltaStreamer implements Serializable { */ 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. */ TypedProperties props; - 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.jssc = jssc; + this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate(); this.fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); if (fs.exists(new Path(cfg.targetBasePath))) { @@ -129,19 +161,28 @@ public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException 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()); this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc); + this.transformer = UtilHelpers.createTransformer(cfg.transformerClassName); 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 - List schemas = Arrays.asList(schemaProvider.getSourceSchema(), - schemaProvider.getTargetSchema()); - jssc.sc().getConf().registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList()); + this.formatAdapter = + new SourceFormatAdapter(UtilHelpers.createSource(cfg.sourceClassName, props, jssc, sparkSession, + schemaProvider)); + + this.hiveConf = hiveConf; + } + + private static HiveConf getDefaultHiveConf(Configuration cfg) { + HiveConf hiveConf = new HiveConf(); + hiveConf.addResource(cfg); + return hiveConf; } public void sync() throws Exception { + HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(null)); + Timer.Context overallTimerContext = metrics.getOverallTimerContext(); // Retrieve the previous round checkpoints, if any Optional resumeCheckpointStr = Optional.empty(); if (commitTimelineOpt.isPresent()) { @@ -163,16 +204,42 @@ public void sync() throws Exception { } log.info("Checkpoint to resume from : " + resumeCheckpointStr); - // Pull the data from the source & prepare the write - Pair>, String> dataAndCheckpoint = source.fetchNewData( - resumeCheckpointStr, cfg.sourceLimit); + final Optional> avroRDDOptional; + 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> dataAndCheckpoint = formatAdapter.fetchNewDataInRowFormat( + resumeCheckpointStr, cfg.sourceLimit); + + Optional> 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> dataAndCheckpoint = + formatAdapter.fetchNewDataInAvroFormat(resumeCheckpointStr, cfg.sourceLimit); + avroRDDOptional = dataAndCheckpoint.getBatch(); + checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch(); + schemaProvider = dataAndCheckpoint.getSchemaProvider(); + } - if (!dataAndCheckpoint.getKey().isPresent()) { + if ((!avroRDDOptional.isPresent()) || (avroRDDOptional.get().isEmpty())) { log.info("No new data, nothing to commit.. "); return; } - JavaRDD avroRDD = dataAndCheckpoint.getKey().get(); + registerAvroSchemas(schemaProvider); + + JavaRDD avroRDD = avroRDDOptional.get(); JavaRDD records = avroRDD.map(gr -> { HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr, (Comparable) gr.get(cfg.sourceOrderingField)); @@ -180,20 +247,20 @@ public void sync() throws Exception { }); // filter dupes if needed - HoodieWriteConfig hoodieCfg = getHoodieClientConfig(); + HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider); if (cfg.filterDupes) { // turn upserts to insert cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation; records = DataSourceUtils.dropDuplicates(jssc, records, hoodieCfg); - } - if (records.isEmpty()) { - log.info("No new data, nothing to commit.. "); - return; + if (records.isEmpty()) { + log.info("No new data, nothing to commit.. "); + return; + } } // Perform the write - HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg); + HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, true); String commitTime = client.startCommit(); log.info("Starting commit : " + commitTime); @@ -210,7 +277,7 @@ public void sync() throws Exception { // Simply commit for now. TODO(vc): Support better error handlers later on HashMap checkpointCommitMetadata = new HashMap<>(); - checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue()); + checkpointCommitMetadata.put(CHECKPOINT_KEY, checkpointStr); boolean success = client.commit(commitTime, writeStatusRDD, Optional.of(checkpointCommitMetadata)); @@ -220,17 +287,54 @@ public void sync() throws Exception { } else { 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(); + long overallTimeMs = overallTimerContext != null ? overallTimerContext.stop() : 0; + + // Send DeltaStreamer Metrics + metrics.updateDeltaStreamerMetrics(overallTimeMs, hiveSyncTimeMs); } - private HoodieWriteConfig getHoodieClientConfig() throws Exception { - return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) - .withAutoCommit(false) - .withSchema(schemaProvider.getTargetSchema().toString()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) - .forTable(cfg.targetTableName) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .withProps(props).build(); + public void syncHive() { + 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 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) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) + .forTable(cfg.targetTableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withProps(props); + if (null != schemaProvider) { + builder = builder.withSchema(schemaProvider.getTargetSchema().toString()); + } + return builder.build(); } public enum Operation { @@ -266,6 +370,10 @@ public static class Config implements Serializable { 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 configs = new ArrayList<>(); + @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, " + "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}") @@ -285,11 +393,22 @@ public static class Config implements Serializable { public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); @Parameter(names = {"--schemaprovider-class"}, description = "subclass of com.uber.hoodie.utilities.schema" - + ".SchemaProvider to attach schemas to input & target table data, built in options: FilebasedSchemaProvider") - public String schemaProviderClassName = FilebasedSchemaProvider.class.getName(); + + ".SchemaProvider to attach schemas to input & target table data, built in options: " + + "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, 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. " - + "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; @Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input " @@ -301,6 +420,9 @@ public static class Config implements Serializable { + "before insert/bulk-insert") 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.") public String sparkMaster = "local[2]"; @@ -319,4 +441,44 @@ public static void main(String[] args) throws Exception { JavaSparkContext jssc = UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName, cfg.sparkMaster); 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 getCommitTimelineOpt() { + return commitTimelineOpt; + } + + public JavaSparkContext getJssc() { + return jssc; + } + + public SparkSession getSparkSession() { + return sparkSession; + } + + public HiveConf getHiveConf() { + return hiveConf; + } + + public TypedProperties getProps() { + return props; + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java new file mode 100644 index 0000000000000..2fc2f81a36b8c --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java @@ -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; + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java new file mode 100644 index 0000000000000..8214f260bb21c --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java @@ -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> fetchNewDataInAvroFormat(Optional lastCkptStr, + long sourceLimit) { + switch (source.getSourceType()) { + case AVRO: + return ((AvroSource)source).fetchNext(lastCkptStr, sourceLimit); + case JSON: { + InputBatch> 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> 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> fetchNewDataInRowFormat(Optional lastCkptStr, long sourceLimit) { + switch (source.getSourceType()) { + case ROW: + return ((RowSource)source).fetchNext(lastCkptStr, sourceLimit); + case AVRO: { + InputBatch> 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> 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() + ")"); + } + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/RowBasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/RowBasedSchemaProvider.java new file mode 100644 index 0000000000000..0a9d7c616caf6 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/RowBasedSchemaProvider.java @@ -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); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java index 3312db5aae8f4..84d8f94fe1f4c 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/SchemaRegistryProvider.java @@ -42,12 +42,15 @@ public class SchemaRegistryProvider extends SchemaProvider { */ 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 targetSchema; - private String fetchSchemaFromRegistry(String registryUrl) throws IOException { + private static String fetchSchemaFromRegistry(String registryUrl) throws IOException { URL registry = new URL(registryUrl); ObjectMapper mapper = new ObjectMapper(); JsonNode node = mapper.readTree(registry.openStream()); @@ -56,17 +59,32 @@ private String fetchSchemaFromRegistry(String registryUrl) throws IOException { public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) { super(props, jssc); - DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SCHEMA_REGISTRY_URL_PROP)); - String registryUrl = props.getString(Config.SCHEMA_REGISTRY_URL_PROP); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_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 { - 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) { 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 public Schema getSourceSchema() { return schema; } + + @Override + public Schema getTargetSchema() { + return targetSchema; + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java index 335d06a9b0675..2f21f325305a4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java @@ -19,7 +19,10 @@ 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.SchemaProvider; +import com.uber.hoodie.utilities.sources.helpers.DFSPathSelector; +import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroKeyInputFormat; @@ -27,18 +30,33 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; /** * DFS Source that reads avro data */ -public class AvroDFSSource extends DFSSource { +public class AvroDFSSource extends AvroSource { - public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { - super(props, sparkContext, schemaProvider); + private final DFSPathSelector pathSelector; + + public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration()); } @Override - protected JavaRDD fromFiles(AvroConvertor convertor, String pathStr) { + protected InputBatch> fetchNewData(Optional lastCkptStr, + long sourceLimit) { + Pair, 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 fromFiles(String pathStr) { JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, sparkContext.hadoopConfiguration()); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java index 4e14714134bf7..f1e51e0ea637d 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java @@ -20,27 +20,55 @@ import com.uber.hoodie.common.util.TypedProperties; 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 java.util.Optional; 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.JavaSparkContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.OffsetRange; /** * 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) { - super(props, sparkContext, schemaProvider); + private static Logger log = LogManager.getLogger(AvroKafkaSource.class); + + private final KafkaOffsetGen offsetGen; + + public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + offsetGen = new KafkaOffsetGen(props); } @Override - protected JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { - return KafkaUtils - .createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, kafkaParams, - offsetRanges).values().map(obj -> (GenericRecord) obj); + protected InputBatch> fetchNewData(Optional 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() : ""); + } else { + log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); + } + JavaRDD newDataRDD = toRDD(offsetRanges); + return new InputBatch<>(Optional.of(newDataRDD), + KafkaOffsetGen.CheckpointUtils.offsetsToStr(offsetRanges)); + } + + private JavaRDD toRDD(OffsetRange[] offsetRanges) { + JavaRDD recordRDD = KafkaUtils + .createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class, + offsetGen.getKafkaParams(), offsetRanges).values().map(obj -> (GenericRecord) obj); + return recordRDD; } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroSource.java new file mode 100644 index 0000000000000..ba767ad62568e --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroSource.java @@ -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> { + + public AvroSource(TypedProperties props, + JavaSparkContext sparkContext, + SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider, SourceType.AVRO); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index 3d6af40fab588..0f5e78746e3f9 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -21,8 +21,6 @@ import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.util.FSUtils; 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.utilities.schema.SchemaProvider; import java.io.IOException; @@ -44,19 +42,20 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; 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 - * by commit and apply to the target table + * Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit by commit and apply + * to the target table *

    * The general idea here is to have commits sync across the data pipeline. *

    - * [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable - * {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...} + * [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable {c1,c2,c3,...} + * {c1,c2,c3,...} {c1,c2,c3,...} *

    * 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); @@ -73,9 +72,9 @@ static class Config { 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) { - super(props, sparkContext, schemaProvider); + super(props, sparkContext, sparkSession, schemaProvider); DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP)); this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP); this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration()); @@ -113,15 +112,15 @@ private Optional findCommitToPull(Optional latestTargetCommit) } @Override - public Pair>, String> fetchNewData( + protected InputBatch> fetchNewData( Optional lastCheckpointStr, long sourceLimit) { try { // find the source commit to pull Optional commitToPull = findCommitToPull(lastCheckpointStr); if (!commitToPull.isPresent()) { - return new ImmutablePair<>(Optional.empty(), - lastCheckpointStr.orElse("")); + return new InputBatch<>(Optional.empty(), + lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); } // read the files out. @@ -132,7 +131,7 @@ public Pair>, String> fetchNewData( JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, 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())); } catch (IOException ioe) { throw new HoodieIOException( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java new file mode 100644 index 0000000000000..430eb1e1e199d --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java @@ -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>, String> fetchNextBatch(Optional 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 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 beginInstant = + lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Optional.empty() : lastCkptStr : Optional.empty(); + + Pair 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 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 validated = source.map((MapFunction) (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 partitionVals = extractor.extractPartitionValuesInPath(hoodiePartitionPath).stream() + .map(o -> (Object) o).collect(Collectors.toList()); + Preconditions.checkArgument(partitionVals.size() == partitionFields.size(), + "#partition-fields != #partition-values-extracted"); + List 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 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()); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java new file mode 100644 index 0000000000000..9139057b924a7 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java @@ -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 { + + private final Optional batch; + private final String checkpointForNextBatch; + private final SchemaProvider schemaProvider; + + public InputBatch(Optional batch, String checkpointForNextBatch, + SchemaProvider schemaProvider) { + this.batch = batch; + this.checkpointForNextBatch = checkpointForNextBatch; + this.schemaProvider = schemaProvider; + } + + public InputBatch(Optional batch, String checkpointForNextBatch) { + this.batch = batch; + this.checkpointForNextBatch = checkpointForNextBatch; + this.schemaProvider = null; + } + + public Optional getBatch() { + return batch; + } + + public String getCheckpointForNextBatch() { + return checkpointForNextBatch; + } + + public SchemaProvider getSchemaProvider() { + return schemaProvider; + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java index 6b1018e152b98..bbf985ba0dbe1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java @@ -19,22 +19,38 @@ 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.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.JavaSparkContext; +import org.apache.spark.sql.SparkSession; /** * DFS Source that reads json data */ -public class JsonDFSSource extends DFSSource { +public class JsonDFSSource extends JsonSource { - public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { - super(props, sparkContext, schemaProvider); + private final DFSPathSelector pathSelector; + + public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration()); } @Override - protected JavaRDD fromFiles(AvroConvertor convertor, String pathStr) { - return sparkContext.textFile(pathStr).map(convertor::fromJson); + protected InputBatch> fetchNewData(Optional lastCkptStr, + long sourceLimit) { + Pair, 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 fromFiles(String pathStr) { + return sparkContext.textFile(pathStr); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java index b271e370494a9..339c37355f918 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java @@ -20,26 +20,49 @@ import com.uber.hoodie.common.util.TypedProperties; 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 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.JavaSparkContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.OffsetRange; /** * Read json kafka data */ -public class JsonKafkaSource extends KafkaSource { +public class JsonKafkaSource extends JsonSource { - public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { - super(properties, sparkContext, schemaProvider); + private static Logger log = LogManager.getLogger(JsonKafkaSource.class); + + private final KafkaOffsetGen offsetGen; + + public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(properties, sparkContext, sparkSession, schemaProvider); + offsetGen = new KafkaOffsetGen(properties); } @Override - protected JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor) { + protected InputBatch> fetchNewData(Optional 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 newDataRDD = toRDD(offsetRanges); + return new InputBatch<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); + } + + private JavaRDD toRDD(OffsetRange[] offsetRanges) { return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class, - kafkaParams, offsetRanges) - .values().map(avroConvertor::fromJson); + offsetGen.getKafkaParams(), offsetRanges).values(); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonSource.java new file mode 100644 index 0000000000000..27ec5f3eb2f23 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonSource.java @@ -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> { + + public JsonSource(TypedProperties props, + JavaSparkContext sparkContext, + SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider, SourceType.JSON); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java new file mode 100644 index 0000000000000..708e55d044d3b --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java @@ -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> { + + public RowSource(TypedProperties props, + JavaSparkContext sparkContext, + SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider, SourceType.ROW); + } + + protected abstract Pair>, String> fetchNextBatch(Optional lastCkptStr, + long sourceLimit); + + @Override + protected final InputBatch> fetchNewData(Optional lastCkptStr, long sourceLimit) { + Pair>, 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())); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java index 06b83e9b1440d..4744e937520a5 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java @@ -19,36 +19,67 @@ 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.SchemaProvider; import java.io.Serializable; import java.util.Optional; -import org.apache.avro.generic.GenericRecord; -import org.apache.spark.api.java.JavaRDD; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; 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. */ -public abstract class Source implements Serializable { +public abstract class Source implements Serializable { + protected static volatile Logger log = LogManager.getLogger(Source.class); - protected transient TypedProperties props; + public enum SourceType { + JSON, + AVRO, + ROW + } + protected transient TypedProperties props; 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.sparkContext = sparkContext; - this.schemaProvider = schemaProvider; + this.sparkSession = sparkSession; + this.overriddenSchemaProvider = schemaProvider; + this.sourceType = sourceType; } + protected abstract InputBatch fetchNewData(Optional lastCkptStr, long sourceLimit); + /** - * Fetches new data upto sourceLimit, from the provided checkpoint and returns an RDD of the - * data, as well as the checkpoint to be written as a result of that. + * Main API called by Hoodie Delta Streamer to fetch records + * @param lastCkptStr Last Checkpoint + * @param sourceLimit Source Limit + * @return */ - public abstract Pair>, String> fetchNewData( - Optional lastCheckpointStr, long sourceLimit); + public final InputBatch fetchNext(Optional lastCkptStr, long sourceLimit) { + InputBatch 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; + } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/AvroConvertor.java similarity index 93% rename from hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java rename to hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/AvroConvertor.java index feb06d5b362a3..ef022b7a8bcfe 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroConvertor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/AvroConvertor.java @@ -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.avro.GenericAvroCodecs; @@ -55,6 +55,10 @@ public AvroConvertor(String schemaStr) { this.schemaStr = schemaStr; } + public AvroConvertor(Schema schema) { + this.schemaStr = schema.toString(); + this.schema = schema; + } private void initSchema() { if (schema == null) { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java similarity index 76% rename from hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java rename to hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java index 6d962b276d57d..2c5f9f292b56b 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java @@ -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"); * 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.common.util.FSUtils; @@ -24,45 +24,38 @@ 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.utilities.schema.SchemaProvider; import java.io.IOException; import java.util.*; 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.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -/** - * Source to read data from a given DFS directory structure, incrementally - */ -public abstract class DFSSource extends Source { +public class DFSPathSelector { /** * Configs supported */ static class Config { + private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; } private static final List IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_"); private final transient FileSystem fs; + private final TypedProperties props; - public DFSSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { - super(props, sparkContext, schemaProvider); - DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP)); - this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), sparkContext.hadoopConfiguration()); + public DFSPathSelector(TypedProperties props, Configuration hadoopConf) { + DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); + this.props = props; + this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), hadoopConf); } - protected abstract JavaRDD fromFiles(final AvroConvertor convertor, String pathStr); - - @Override - public Pair>, String> fetchNewData( + public Pair, String> getNextFilePathsAndMaxModificationTime( Optional lastCheckpointStr, long sourceLimit) { try { @@ -111,11 +104,9 @@ public Pair>, String> fetchNewData( // read the files out. String pathStr = filteredFiles.stream().map(f -> f.getPath().toString()) .collect(Collectors.joining(",")); - String schemaStr = schemaProvider.getSourceSchema().toString(); - final AvroConvertor avroConvertor = new AvroConvertor(schemaStr); return new ImmutablePair<>( - Optional.of(fromFiles(avroConvertor, pathStr)), + Optional.ofNullable(pathStr), String.valueOf(maxModificationTime)); } catch (IOException ioe) { throw new HoodieIOException( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java new file mode 100644 index 0000000000000..93056012b593b --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java @@ -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 calculateBeginAndEndInstants( + JavaSparkContext jssc, String srcBasePath, int numInstantsPerFetch, Optional 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 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 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)"); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java similarity index 84% rename from hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java rename to hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java index 4699fcaf32cc5..947f3c48ae8f9 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java @@ -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.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.utilities.exception.HoodieDeltaStreamerException; -import com.uber.hoodie.utilities.schema.SchemaProvider; - -import java.util.*; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Optional; import java.util.stream.Collectors; import kafka.common.TopicAndPartition; -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.JavaSparkContext; import org.apache.spark.streaming.kafka.KafkaCluster; import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset; import org.apache.spark.streaming.kafka.OffsetRange; @@ -49,14 +47,13 @@ /** * 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 - - static class CheckpointUtils { + public static class CheckpointUtils { /** * Reconstruct checkpoint from string. @@ -90,7 +87,6 @@ public static String offsetsToStr(OffsetRange[] ranges) { return sb.toString(); } - /** * Compute the offset ranges to read from Kafka, while handling newly added partitions, skews, event limits. * @@ -174,19 +170,18 @@ enum KafkaResetOffsetStrategies { * Configs to be passed for this source. All standard Kafka consumer configs are also respected */ static class Config { + private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LARGEST; } - - protected HashMap kafkaParams; - + private final HashMap kafkaParams; + private final TypedProperties props; protected final String topicName; - public KafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { - super(props, sparkContext, schemaProvider); - - kafkaParams = new HashMap<>(); + public KafkaOffsetGen(TypedProperties props) { + this.props = props; + kafkaParams = new HashMap(); for (Object prop : props.keySet()) { kafkaParams.put(prop.toString(), props.getString(prop.toString())); } @@ -194,11 +189,7 @@ public KafkaSource(TypedProperties props, JavaSparkContext sparkContext, SchemaP topicName = props.getString(Config.KAFKA_TOPIC_NAME); } - protected abstract JavaRDD toAvroRDD(OffsetRange[] offsetRanges, AvroConvertor avroConvertor); - - @Override - public Pair>, String> fetchNewData( - Optional lastCheckpointStr, long sourceLimit) { + public OffsetRange[] getNextOffsetRanges(Optional lastCheckpointStr, long sourceLimit) { // Obtain current metadata for the topic KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); @@ -240,16 +231,15 @@ public Pair>, String> fetchNewData( // 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); OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents); - long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); - if (totalNewMsgs <= 0) { - return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.orElse("")); - } else { - log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName); - } - // Produce a RDD[GenericRecord] - final AvroConvertor avroConvertor = new AvroConvertor(schemaProvider.getSourceSchema().toString()); - JavaRDD newDataRDD = toAvroRDD(offsetRanges, avroConvertor); - return new ImmutablePair<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); + return offsetRanges; + } + + public String getTopicName() { + return topicName; + } + + public HashMap getKafkaParams() { + return kafkaParams; } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/IdentityTransformer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/IdentityTransformer.java new file mode 100644 index 0000000000000..b454cdf1db3fe --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/IdentityTransformer.java @@ -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 apply(JavaSparkContext jsc, SparkSession sparkSession, + Dataset rowDataset, TypedProperties properties) { + return rowDataset; + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/SqlQueryBasedTransformer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/SqlQueryBasedTransformer.java new file mode 100644 index 0000000000000..b967f45d1dcc0 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/SqlQueryBasedTransformer.java @@ -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 "\" + */ +public class SqlQueryBasedTransformer implements Transformer { + + private static volatile Logger log = LogManager.getLogger(SqlQueryBasedTransformer.class); + + private static final String SRC_PATTERN = ""; + 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 apply(JavaSparkContext jsc, SparkSession sparkSession, + Dataset 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); + } +} diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/Transformer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/Transformer.java new file mode 100644 index 0000000000000..32e80facdeb81 --- /dev/null +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/transform/Transformer.java @@ -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 apply(JavaSparkContext jsc, SparkSession sparkSession, + Dataset rowDataset, TypedProperties properties); +} diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java index 63d93b4144a59..acce004519a23 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java @@ -19,8 +19,10 @@ package com.uber.hoodie.utilities; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.uber.hoodie.DataSourceWriteOptions; import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -28,17 +30,31 @@ import com.uber.hoodie.common.util.DFSPropertiesConfiguration; import com.uber.hoodie.common.util.TypedProperties; 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.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.transform.SqlQueryBasedTransformer; +import com.uber.hoodie.utilities.transform.Transformer; import java.io.IOException; +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; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; 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.AfterClass; import org.junit.Before; @@ -55,17 +71,43 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { @BeforeClass public static void initClass() throws Exception { - UtilitiesTestBase.initClass(); + UtilitiesTestBase.initClass(true); // prepare the configs. 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/target.avsc", dfs, dfsBasePath + "/target.avsc"); + 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.partitionpath.field", "not_there"); 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"); + + // 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 @@ -86,17 +128,48 @@ public void teardown() throws Exception { } static class TestHelpers { - 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(); cfg.targetBasePath = basePath; cfg.targetTableName = "hoodie_trips"; cfg.storageType = "COPY_ON_WRITE"; cfg.sourceClassName = TestDataSource.class.getName(); + cfg.transformerClassName = transformerClassName; cfg.operation = op; + cfg.enableHiveSync = enableHiveSync; cfg.sourceOrderingField = "timestamp"; cfg.propsFilePath = dfsBasePath + "/test-source.properties"; 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 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; } @@ -110,15 +183,30 @@ static List countsPerCommit(String datasetPath, SQLContext sqlContext) { .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 { HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath); HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - HoodieInstant lastCommit = timeline.lastInstant().get(); + HoodieInstant lastInstant = timeline.lastInstant().get(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - timeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class); + timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class); assertEquals(totalCommits, timeline.countInstants()); assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY)); + return lastInstant.getTimestamp(); } } @@ -152,12 +240,14 @@ public void testBulkInsertsAndUpserts() throws Exception { HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT); new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); // No new data => no commits. cfg.sourceLimit = 0; new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1); // upsert() #1 @@ -165,11 +255,94 @@ public void testBulkInsertsAndUpserts() throws Exception { cfg.operation = Operation.UPSERT; new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2); List counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext); 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 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 public void testFilterDupes() throws Exception { String datasetBasePath = dfsBasePath + "/test_dupes_dataset"; @@ -192,4 +365,57 @@ public void testFilterDupes() throws Exception { assertEquals(1000, counts.get(0).getLong(1)); assertEquals(1000, counts.get(1).getLong(1)); } + + /** + * UDF to calculate Haversine distance + */ + public static class DistanceUDF implements UDF4 { + + /** + * + * 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 apply(JavaSparkContext jsc, SparkSession sparkSession, + Dataset 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"))); + } + } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java index eb2d00a65d61d..a5bba734314dc 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/UtilitiesTestBase.java @@ -18,10 +18,16 @@ package com.uber.hoodie.utilities; +import com.google.common.collect.ImmutableList; import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.minicluster.HdfsTestService; 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.hive.HiveSyncConfig; +import com.uber.hoodie.hive.HoodieHiveClient; +import com.uber.hoodie.hive.util.HiveTestService; import com.uber.hoodie.utilities.sources.TestDataSource; import java.io.BufferedReader; import java.io.IOException; @@ -32,8 +38,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; 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.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -51,15 +60,26 @@ public class UtilitiesTestBase { protected static MiniDFSCluster dfsCluster; protected static DistributedFileSystem dfs; protected transient JavaSparkContext jsc = null; + protected transient SparkSession sparkSession = null; protected transient SQLContext sqlContext; + protected static HiveServer2 hiveServer; @BeforeClass public static void initClass() throws Exception { + initClass(false); + } + + static void initClass(boolean startHiveService) throws Exception { hdfsTestService = new HdfsTestService(); dfsCluster = hdfsTestService.start(true); dfs = dfsCluster.getFileSystem(); dfsBasePath = dfs.getWorkingDirectory().toString(); dfs.mkdirs(new Path(dfsBasePath)); + if (startHiveService) { + HiveTestService hiveService = new HiveTestService(hdfsTestService.getHadoopConf()); + hiveServer = hiveService.start(); + clearHiveDb(); + } } @AfterClass @@ -67,6 +87,9 @@ public static void cleanupClass() throws Exception { if (hdfsTestService != null) { hdfsTestService.stop(); } + if (hiveServer != null) { + hiveServer.stop(); + } } @Before @@ -74,6 +97,7 @@ public void setup() throws Exception { TestDataSource.initDataGen(); jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]"); sqlContext = new SQLContext(jsc); + sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate(); } @After @@ -84,6 +108,42 @@ public void teardown() throws Exception { } } + /** + * 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().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 { // to get hold of resources bundled with jar diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java index d460e6ea8527a..828b78dd8b49f 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java @@ -20,16 +20,20 @@ import static org.junit.Assert.assertEquals; +import com.uber.hoodie.AvroConversionUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; 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.deltastreamer.SourceFormatAdapter; import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import java.io.IOException; import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; 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.AfterClass; import org.junit.Before; @@ -37,7 +41,7 @@ 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 { @@ -71,34 +75,47 @@ public void testJsonDFSSource() throws IOException { TypedProperties props = new TypedProperties(); 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 - 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( Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs, dfsBasePath + "/jsonFiles/1.json"); - assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), 10).getKey()); - Pair>, String> fetch1 = jsonSource.fetchNewData(Optional.empty(), 1000000); - assertEquals(100, fetch1.getKey().get().count()); + assertEquals(Optional.empty(), jsonSource.fetchNewDataInAvroFormat(Optional.empty(), 10).getBatch()); + InputBatch> fetch1 = + jsonSource.fetchNewDataInAvroFormat(Optional.empty(), 1000000); + assertEquals(100, fetch1.getBatch().get().count()); + // Test json -> Row format + InputBatch> fetch1AsRows = + jsonSource.fetchNewDataInRowFormat(Optional.empty(), 1000000); + assertEquals(100, fetch1AsRows.getBatch().get().count()); + // Test Avro -> Row format + Dataset fetch1Rows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), + schemaProvider.getSourceSchema().toString(), jsonDFSSource.getSparkSession()); + assertEquals(100, fetch1Rows.count()); // 2. Produce new data, extract new data UtilitiesTestBase.Helpers.saveStringsToDFS( Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)), dfs, dfsBasePath + "/jsonFiles/2.json"); - Pair>, String> fetch2 = jsonSource.fetchNewData( - Optional.of(fetch1.getValue()), Long.MAX_VALUE); - assertEquals(10000, fetch2.getKey().get().count()); + InputBatch> fetch2 = jsonSource.fetchNewDataInRowFormat( + Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(10000, fetch2.getBatch().get().count()); // 3. Extract with previous checkpoint => gives same data back (idempotent) - Pair>, String> fetch3 = jsonSource.fetchNewData( - Optional.of(fetch1.getValue()), Long.MAX_VALUE); - assertEquals(10000, fetch3.getKey().get().count()); - assertEquals(fetch2.getValue(), fetch3.getValue()); + InputBatch> fetch3 = jsonSource.fetchNewDataInRowFormat( + Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(10000, fetch3.getBatch().get().count()); + assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); + fetch3.getBatch().get().registerTempTable("test_dfs_table"); + Dataset rowDataset = new SQLContext(jsc.sc()).sql("select * from test_dfs_table"); + assertEquals(10000, rowDataset.count()); // 4. Extract with latest checkpoint => no new data returned - Pair>, String> fetch4 = jsonSource.fetchNewData( - Optional.of(fetch2.getValue()), Long.MAX_VALUE); - assertEquals(Optional.empty(), fetch4.getKey()); + InputBatch> fetch4 = jsonSource.fetchNewDataInAvroFormat( + Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Optional.empty(), fetch4.getBatch()); } -} +} \ No newline at end of file diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java index 57369de33cbf3..ad074149d9333 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java @@ -21,8 +21,6 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; 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 java.io.IOException; import java.util.ArrayList; @@ -35,11 +33,12 @@ import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; /** * 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); @@ -54,8 +53,9 @@ public static void resetDataGen() { dataGenerator = null; } - public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) { - super(props, sparkContext, schemaProvider); + public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); } private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) { @@ -68,14 +68,14 @@ private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) { } @Override - public Pair>, String> fetchNewData(Optional lastCheckpointStr, + protected InputBatch> fetchNewData(Optional lastCheckpointStr, long sourceLimit) { int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0); String commitTime = String.format("%05d", nextCommitNum); // No new data. if (sourceLimit <= 0) { - return new ImmutablePair<>(Optional.empty(), commitTime); + return new InputBatch<>(Optional.empty(), commitTime); } // generate `sourceLimit` number of upserts each time. @@ -94,6 +94,6 @@ public Pair>, String> fetchNewData(Optional avroRDD = sparkContext.parallelize(records, 4); - return new ImmutablePair<>(Optional.of(avroRDD), commitTime); + return new InputBatch<>(Optional.of(avroRDD), commitTime); } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java index 785e805694dd9..1adcbc3713ab7 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java @@ -18,20 +18,23 @@ package com.uber.hoodie.utilities.sources; -import static com.uber.hoodie.utilities.sources.KafkaSource.CheckpointUtils; import static org.junit.Assert.assertEquals; +import com.uber.hoodie.AvroConversionUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; 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.deltastreamer.SourceFormatAdapter; import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; +import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; import java.io.IOException; import java.util.HashMap; import java.util.Optional; import kafka.common.TopicAndPartition; 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.streaming.kafka.KafkaCluster.LeaderOffset; import org.apache.spark.streaming.kafka.KafkaTestUtils; import org.apache.spark.streaming.kafka.OffsetRange; @@ -42,7 +45,7 @@ import org.junit.Test; /** - * Tests against {@link KafkaSource} + * Tests against {@link AvroKafkaSource} */ public class TestKafkaSource extends UtilitiesTestBase { @@ -89,30 +92,44 @@ public void testJsonKafkaSource() throws IOException { props.setProperty("key.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 - 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))); - Pair>, String> fetch1 = kafkaSource.fetchNewData(Optional.empty(), 900); - assertEquals(900, fetch1.getKey().get().count()); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Optional.empty(), 900); + assertEquals(900, fetch1.getBatch().get().count()); + // Test Avro To DataFrame path + Dataset fetch1AsRows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), + schemaProvider.getSourceSchema().toString(), jsonSource.getSparkSession()); + assertEquals(900, fetch1AsRows.count()); // 2. Produce new data, extract new data testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); - Pair>, String> fetch2 = kafkaSource.fetchNewData( - Optional.of(fetch1.getValue()), Long.MAX_VALUE); - assertEquals(1100, fetch2.getKey().get().count()); + InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat( + Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(1100, fetch2.getBatch().get().count()); // 3. Extract with previous checkpoint => gives same data back (idempotent) - Pair>, String> fetch3 = kafkaSource.fetchNewData( - Optional.of(fetch1.getValue()), Long.MAX_VALUE); - assertEquals(fetch2.getKey().get().count(), fetch3.getKey().get().count()); - assertEquals(fetch2.getValue(), fetch3.getValue()); + InputBatch> fetch3 = kafkaSource.fetchNewDataInAvroFormat( + Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count()); + assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); + // Same using Row API + InputBatch> 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 - Pair>, String> fetch4 = kafkaSource.fetchNewData( - Optional.of(fetch2.getValue()), Long.MAX_VALUE); - assertEquals(Optional.empty(), fetch4.getKey()); + InputBatch> fetch4 = kafkaSource.fetchNewDataInAvroFormat( + Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Optional.empty(), fetch4.getBatch()); + // Same using Row API + InputBatch> fetch4AsRows = + kafkaSource.fetchNewDataInRowFormat(Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Optional.empty(), fetch4AsRows.getBatch()); } diff --git a/hoodie-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties b/hoodie-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties new file mode 100644 index 0000000000000..87038c36bea12 --- /dev/null +++ b/hoodie-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties @@ -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 a \ No newline at end of file diff --git a/hoodie-utilities/src/test/resources/delta-streamer-config/target.avsc b/hoodie-utilities/src/test/resources/delta-streamer-config/target.avsc new file mode 100644 index 0000000000000..d2d41036300c2 --- /dev/null +++ b/hoodie-utilities/src/test/resources/delta-streamer-config/target.avsc @@ -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" + }] +} + diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 79a4a757a4ac4..4e2e72d477ca0 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -66,6 +66,48 @@ hadoop-auth + + ${hive.groupid} + hive-jdbc + ${hive.version} + + + commons-logging + commons-logging + + + + + ${hive.groupid} + hive-exec + ${hive.version} + + + ${hive.groupid} + hive-service + ${hive.version} + + + ${hive.groupid} + hive-shims + ${hive.version} + + + ${hive.groupid} + hive-serde + ${hive.version} + + + ${hive.groupid} + hive-metastore + ${hive.version} + + + ${hive.groupid} + hive-common + ${hive.version} + + org.apache.hadoop hadoop-hdfs @@ -182,116 +224,4 @@ true - - - - hive12 - - - !hive11 - - - - - - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - - - commons-logging - commons-logging - - - - - ${hive12.groupid} - hive-exec - ${hive12.version} - - - ${hive12.groupid} - hive-service - ${hive12.version} - - - ${hive12.groupid} - hive-shims - ${hive12.version} - - - ${hive12.groupid} - hive-serde - ${hive12.version} - - - ${hive12.groupid} - hive-metastore - ${hive12.version} - - - ${hive12.groupid} - hive-common - ${hive12.version} - - - - - hive11 - - - hive11 - - - - .hive11 - - - - ${hive11.groupid} - hive-service - ${hive11.version} - - - ${hive11.groupid} - hive-shims - ${hive11.version} - - - ${hive11.groupid} - hive-jdbc - ${hive11.version} - - - commons-logging - commons-logging - - - - - ${hive11.groupid} - hive-serde - ${hive11.version} - - - ${hive11.groupid} - hive-metastore - ${hive11.version} - - - ${hive11.groupid} - hive-common - ${hive11.version} - - - ${hive11.groupid} - hive-exec - ${hive11.version} - - - - - diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index eafad25922921..ee43709faa29b 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -44,6 +44,26 @@ org.apache.hadoop hadoop-auth + + ${hive.groupid} + hive-service + ${hive.version} + + + ${hive.groupid} + hive-jdbc + ${hive.version} + + + ${hive.groupid} + hive-metastore + ${hive.version} + + + ${hive.groupid} + hive-common + ${hive.version} + com.google.guava guava @@ -195,73 +215,4 @@ true - - - - hive12 - - - !hive11 - - - - - - - - ${hive12.groupid} - hive-service - ${hive12.version} - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - - - ${hive12.groupid} - hive-metastore - ${hive12.version} - - - ${hive12.groupid} - hive-common - ${hive12.version} - - - - - hive11 - - - hive11 - - - - .hive11 - - - - org.apache.hive - hive-service - ${hive11.version} - - - org.apache.hive - hive-jdbc - ${hive11.version} - - - org.apache.hive - hive-metastore - ${hive11.version} - - - org.apache.hive - hive-common - ${hive11.version} - - - - diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 95b4aedd42e54..5c932e59c174a 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -239,6 +239,26 @@ org.apache.avro avro + + ${hive.groupid} + hive-service + ${hive.version} + + + ${hive.groupid} + hive-jdbc + ${hive.version} + + + ${hive.groupid} + hive-metastore + ${hive.version} + + + ${hive.groupid} + hive-common + ${hive.version} + org.apache.commons commons-configuration2 @@ -269,74 +289,5 @@ ${project.version} - - - - hive12 - - - !hive11 - - - - - - - - ${hive12.groupid} - hive-service - ${hive12.version} - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - - - ${hive12.groupid} - hive-metastore - ${hive12.version} - - - ${hive12.groupid} - hive-common - ${hive12.version} - - - - - hive11 - - - hive11 - - - - .hive11 - - - - ${hive11.groupid} - hive-service - ${hive11.version} - - - ${hive11.groupid} - hive-jdbc - ${hive11.version} - - - ${hive11.groupid} - hive-metastore - ${hive11.version} - - - ${hive11.groupid} - hive-common - ${hive11.version} - - - - diff --git a/pom.xml b/pom.xml index 88196084c9a47..a0ec312c51f2a 100644 --- a/pom.xml +++ b/pom.xml @@ -129,10 +129,8 @@ 1.2.17 2.9.9 2.7.3 - org.apache.hive - 1.2.1 - org.apache.hive - 1.1.1 + org.apache.hive + 1.2.1 3.1.1 2.1.0 1.7.7 @@ -589,6 +587,11 @@ commons-dbcp 1.4 + + commons-pool + commons-pool + 1.4 + org.apache.httpcomponents httpcore @@ -656,7 +659,48 @@ jackson-mapper-asl 1.9.13 - + + ${hive.groupid} + hive-service + ${hive.version} + provided + + + ${hive.groupid} + hive-shims + ${hive.version} + provided + + + ${hive.groupid} + hive-jdbc + ${hive.version} + provided + + + ${hive.groupid} + hive-serde + ${hive.version} + provided + + + ${hive.groupid} + hive-metastore + ${hive.version} + provided + + + ${hive.groupid} + hive-common + ${hive.version} + provided + + + ${hive.groupid} + hive-exec + ${hive.version} + provided + org.apache.hadoop hadoop-hdfs @@ -708,109 +752,6 @@ - - hive12 - - - !hive11 - - - - - ${hive12.groupid} - hive-service - ${hive12.version} - provided - - - ${hive12.groupid} - hive-shims - ${hive12.version} - provided - - - ${hive12.groupid} - hive-jdbc - ${hive12.version} - provided - - - ${hive12.groupid} - hive-serde - ${hive12.version} - provided - - - ${hive12.groupid} - hive-metastore - ${hive12.version} - provided - - - ${hive12.groupid} - hive-common - ${hive12.version} - provided - - - ${hive12.groupid} - hive-exec - ${hive12.version} - provided - - - - - hive11 - - - hive11 - - - - - org.apache.hive - hive-service - ${hive11.version} - - - org.apache.hive - hive-shims - ${hive11.version} - provided - - - org.apache.hive - hive-jdbc - ${hive11.version} - provided - - - org.apache.hive - hive-serde - ${hive11.version} - provided - - - org.apache.hive - hive-metastore - ${hive11.version} - provided - - - org.apache.hive - hive-common - ${hive11.version} - provided - - - org.apache.hive - hive-exec - ${hive11.version} - provided - - - release From 3e187691103e420ef608cb50de9aa45176fff9db Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Mon, 11 Feb 2019 11:39:29 -0800 Subject: [PATCH 146/374] Fix Hoodie Record Reader to work with non-partitioned dataset --- .../AbstractRealtimeRecordReader.java | 7 +++++-- .../hoodie/hadoop/InputFormatTestUtil.java | 19 ++++++++++++++++--- .../HoodieRealtimeRecordReaderTest.java | 18 +++++++++++++++--- 3 files changed, 36 insertions(+), 8 deletions(-) diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java index b48f891467c6d..0dfef07bcb56e 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -333,8 +333,11 @@ private void init() throws IOException { } // Add partitioning fields to writer schema for resulting row to contain null values for these fields - List partitioningFields = Arrays.stream( - jobConf.get("partition_columns", "").split(",")).collect(Collectors.toList()); + + String partitionFields = jobConf.get("partition_columns", ""); + List partitioningFields = + partitionFields.length() > 0 ? Arrays.stream(partitionFields.split(",")).collect(Collectors.toList()) + : new ArrayList<>(); writerSchema = addPartitionFields(writerSchema, partitioningFields); List projectionFields = orderFields( diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java index 4088afdd05285..351a306fef2d3 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java @@ -99,11 +99,26 @@ public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema basePath.create(); HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); + createData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber); + return partitionPath; + } + + public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema, + int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { + baseDir.create(); + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), baseDir.getRoot().toString()); + File basePath = baseDir.getRoot(); + createData(schema, basePath, numberOfFiles, numberOfRecords, commitNumber); + return basePath; + } + + private static void createData(Schema schema, + File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber) + throws IOException { AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { String fileId = FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i); File dataFile = new File(partitionPath, fileId); - // dataFile.createNewFile(); parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema); try { for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, @@ -114,8 +129,6 @@ public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema parquetWriter.close(); } } - return partitionPath; - } private static Iterable generateAvroRecords(Schema schema, diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index b3b095592ab1a..fbd635e319f28 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -109,13 +109,23 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St @Test public void testReader() throws Exception { + testReader(true); + } + + @Test + public void testNonPartitionedReader() throws Exception { + testReader(false); + } + + public void testReader(boolean partitioned) throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; - File partitionDir = InputFormatTestUtil - .prepareParquetDataset(basePath, schema, 1, 100, commitTime); + File partitionDir = + partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, commitTime) + : InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, commitTime); InputFormatTestUtil.commit(basePath, commitTime); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); @@ -146,7 +156,9 @@ public void testReader() throws Exception { .collect(Collectors.joining(",")); jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); - jobConf.set("partition_columns", "datestr"); + if (partitioned) { + jobConf.set("partition_columns", "datestr"); + } //validate record reader compaction HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); From b8887b4aa14f39e52d66eb33e703c48b1430c14b Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 5 Feb 2019 16:54:31 -0800 Subject: [PATCH 147/374] Table rollback for inflight compactions MUST not delete instant files at any time to avoid race conditions --- .../com/uber/hoodie/HoodieWriteClient.java | 7 ++- .../hoodie/table/HoodieCopyOnWriteTable.java | 26 ++++++-- .../hoodie/table/HoodieMergeOnReadTable.java | 14 +++-- .../com/uber/hoodie/table/HoodieTable.java | 4 +- .../com/uber/hoodie/TestAsyncCompaction.java | 60 +++++++++++++++++++ .../java/com/uber/hoodie/TestCleaner.java | 4 +- 6 files changed, 97 insertions(+), 18 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index c879c06ccb560..8deaeb220800b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -839,7 +839,7 @@ private void rollback(List commits) { // Remove interleaving pending compactions before rolling back commits pendingCompactionToRollback.forEach(this::deletePendingCompaction); - List stats = table.rollback(jsc, commitsToRollback); + List stats = table.rollback(jsc, commitsToRollback, true); // cleanup index entries commitsToRollback.forEach(s -> { @@ -1206,8 +1206,9 @@ protected void commitCompaction(JavaRDD compactedStatuses, HoodieTa * @param inflightInstant Inflight Compaction Instant * @param table Hoodie Table */ - private void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) throws IOException { - table.rollback(jsc, ImmutableList.copyOf(new String[] { inflightInstant.getTimestamp() })); + @VisibleForTesting + void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) throws IOException { + table.rollback(jsc, ImmutableList.copyOf(new String[] { inflightInstant.getTimestamp() }), false); // Revert instant state file table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 346d083a7b0e0..af272448b4bb1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -347,7 +347,7 @@ protected Map deleteCleanedFiles(Map r } @Override - public List rollback(JavaSparkContext jsc, List commits) + public List rollback(JavaSparkContext jsc, List commits, boolean deleteInstants) throws IOException { String actionType = metaClient.getCommitActionType(); HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); @@ -375,13 +375,29 @@ public List rollback(JavaSparkContext jsc, List comm // clean temporary data files cleanTemporaryDataFiles(jsc); - // Remove the rolled back inflight commits - commits.stream().map(s -> new HoodieInstant(true, actionType, s)) - .forEach(activeTimeline::deleteInflight); - logger.info("Deleted inflight commits " + commits); + // Delete Inflight instants if enabled + deleteInflightInstants(deleteInstants, activeTimeline, + commits.stream().map(s -> new HoodieInstant(true, actionType, s)).collect(Collectors.toList())); return stats; } + /** + * Delete Inflight instants if enabled + * @param deleteInstants Enable Deletion of Inflight instants + * @param activeTimeline Hoodie active timeline + * @param instantsToBeDeleted Instants to be deleted + */ + protected static void deleteInflightInstants(boolean deleteInstants, HoodieActiveTimeline activeTimeline, + List instantsToBeDeleted) { + // Remove the rolled back inflight commits + if (deleteInstants) { + instantsToBeDeleted.forEach(activeTimeline::deleteInflight); + logger.info("Deleted inflight commits " + instantsToBeDeleted); + } else { + logger.warn("Rollback finished without deleting inflight instant files. Instants=" + instantsToBeDeleted); + } + } + /** * Finalize the written data files * diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index de1e563e9331e..29280a49efd83 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -165,7 +165,7 @@ public JavaRDD compact(JavaSparkContext jsc, String compactionInsta } @Override - public List rollback(JavaSparkContext jsc, List commits) + public List rollback(JavaSparkContext jsc, List commits, boolean deleteInstants) throws IOException { //At the moment, MOR table type does not support nested rollbacks @@ -274,11 +274,13 @@ public List rollback(JavaSparkContext jsc, List comm return hoodieRollbackStats; }).collect(Collectors.toList())).flatMap(List::iterator).filter(Objects::nonNull).collect(); - commitsAndCompactions.entrySet().stream().map( - entry -> new HoodieInstant(true, entry.getValue().getAction(), - entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight); - logger - .debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); + // Delete Inflight instants if enabled + deleteInflightInstants(deleteInstants, this.getActiveTimeline(), + commitsAndCompactions.entrySet().stream().map( + entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp())) + .collect(Collectors.toList())); + + logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); return allRollbackStats; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 2f51cbc8c77e1..749fc6cb24fe1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -251,9 +251,9 @@ public abstract JavaRDD compact(JavaSparkContext jsc, String compac /** * Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1) * Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files - * / log blocks (4) Finally, delete ..commit or ..inflight file + * / log blocks (4) Finally, delete ..commit or ..inflight file if deleteInstants = true */ - public abstract List rollback(JavaSparkContext jsc, List commits) + public abstract List rollback(JavaSparkContext jsc, List commits, boolean deleteInstants) throws IOException; /** diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java index dca6d9d69660a..0df9da859e8e3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -33,6 +33,7 @@ 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.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.CompactionUtils; @@ -51,6 +52,7 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.junit.Test; @@ -78,6 +80,64 @@ public void tearDown() throws IOException { super.tearDown(); } + @Test + public void testRollbackForInflightCompaction() throws Exception { + // Rollback inflight compaction + HoodieWriteConfig cfg = getConfig(false); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true); + + String firstInstantTime = "001"; + String secondInstantTime = "004"; + String compactionInstantTime = "005"; + + int numRecs = 2000; + + List records = dataGen.generateInserts(firstInstantTime, numRecs); + runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime), + records, cfg, true, new ArrayList<>()); + + // Schedule compaction but do not run them + scheduleCompaction(compactionInstantTime, client, cfg); + + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + + HoodieInstant pendingCompactionInstant = + metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); + assertTrue("Pending Compaction instant has expected instant time", + pendingCompactionInstant.getTimestamp().equals(compactionInstantTime)); + assertTrue("Pending Compaction instant has expected state", + pendingCompactionInstant.getState().equals(State.REQUESTED)); + + moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg); + + // Reload and rollback inflight compaction + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + hoodieTable.rollback(jsc, Arrays.asList(compactionInstantTime), false); + + client.rollbackInflightCompaction( + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline() + .getInstants().findFirst().get(); + assertEquals("compaction", pendingCompactionInstant.getAction()); + assertEquals(State.REQUESTED, pendingCompactionInstant.getState()); + assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp()); + + // We indirectly test for the race condition where a inflight instant was first deleted then created new. Every + // time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie + // reads compaction plan from aux path which is untouched). TO test for regression, we simply get file status + // and look at the file size + FileStatus fstatus = + metaClient.getFs().getFileStatus(new Path(metaClient.getMetaPath(), pendingCompactionInstant.getFileName())); + assertTrue(fstatus.getLen() > 0); + } + + private Path getInstantPath(HoodieTableMetaClient metaClient, String timestamp, String action, State state) { + HoodieInstant instant = new HoodieInstant(state, action, timestamp); + return new Path(metaClient.getMetaPath(), instant.getFileName()); + } + @Test public void testRollbackInflightIngestionWithPendingCompaction() throws Exception { // Rollback inflight ingestion when there is pending compaction diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index f3a9616dd5d91..2ef3f79601300 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -634,14 +634,14 @@ public void testCleanTemporaryDataFilesOnRollback() throws IOException { HoodieTable table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - table.rollback(jsc, Collections.emptyList()); + table.rollback(jsc, Collections.emptyList(), true); assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) .withUseTempFolderCopyOnWriteForMerge(false).build(); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - table.rollback(jsc, Collections.emptyList()); + table.rollback(jsc, Collections.emptyList(), true); assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); } From 1263bb49fabaabf5e823ed6954217dc1b1eb5fb7 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 13 Feb 2019 16:46:11 +0800 Subject: [PATCH 148/374] typo: bundle jar with unrecongnized variables --- packaging/hoodie-hadoop-mr-bundle/pom.xml | 2 +- packaging/hoodie-hive-bundle/pom.xml | 2 +- packaging/hoodie-spark-bundle/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index 4e2e72d477ca0..f25b5d1a5c9cf 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -213,7 +213,7 @@ commons-logging:commons-logging - ${project.artifactId}-${project.version}${hiveJarSuffix} + ${project.artifactId}-${project.version} diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index ee43709faa29b..e9843741e29df 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -204,7 +204,7 @@ org.apache.derby:derby - ${project.artifactId}-${project.version}${hiveJarSuffix} + ${project.artifactId}-${project.version} diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 5c932e59c174a..b918b0b7fa6ab 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -165,7 +165,7 @@ org.apache.spark:* - ${project.artifactId}-${project.version}${hiveJarSuffix} + ${project.artifactId}-${project.version} From 720e0d349f62f7c13765e7017c974d236dcd91e0 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Fri, 15 Feb 2019 21:28:39 -0800 Subject: [PATCH 149/374] Update site url in README --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index be2a80a6164a0..52b66e016f65d 100644 --- a/README.md +++ b/README.md @@ -4,4 +4,4 @@ Hudi (pronounced Hoodie) stands for `Hadoop Upserts anD Incrementals`. Hudi mana * **Read Optimized Table** - Provides excellent query performance via purely columnar storage (e.g. [Parquet](https://parquet.apache.org/)) * **Near-Real time Table (WIP)** - Provides queries on real-time data, using a combination of columnar & row based storage (e.g Parquet + [Avro](http://avro.apache.org/docs/current/mr.html)) -For more, head over [here](https://uber.github.io/hudi) +For more, head over [here](https://hudi.apache.org) From 6891e05ba0f60612e72b9326446b3c5e24c7cf05 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Fri, 15 Feb 2019 21:53:08 -0800 Subject: [PATCH 150/374] Create CNAME --- docs/CNAME | 1 + 1 file changed, 1 insertion(+) create mode 100644 docs/CNAME diff --git a/docs/CNAME b/docs/CNAME new file mode 100644 index 0000000000000..c8dd5d09f7065 --- /dev/null +++ b/docs/CNAME @@ -0,0 +1 @@ +hudi.apache.org \ No newline at end of file From 713fb47e97fbc243d97095d098de68e9e7443de8 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 14 Feb 2019 16:50:46 +0800 Subject: [PATCH 151/374] handle no such element exception in HoodieSparkSqlWriter --- .../scala/com/uber/hoodie/HoodieSparkSqlWriter.scala | 7 +++---- .../com/uber/hoodie/HoodieSparkSqlWriterSuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala index 4092fe3c13675..e7ae8e068742b 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala @@ -52,11 +52,10 @@ private[hoodie] object HoodieSparkSqlWriter { if (path.isEmpty || tblName.isEmpty) { throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.") } - val serializer = sparkContext.getConf.get("spark.serializer") - if (!serializer.equals("org.apache.spark.serializer.KryoSerializer")) { - throw new HoodieException(s"${serializer} serialization is not supported by hoodie. Please use kryo.") + sparkContext.getConf.getOption("spark.serializer") match { + case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") => + case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer") } - val storageType = parameters(STORAGE_TYPE_OPT_KEY) val operation = // It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true diff --git a/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala b/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala index cd3c21343aa95..eb528dce891e6 100644 --- a/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala +++ b/hoodie-spark/src/test/scala/com/uber/hoodie/HoodieSparkSqlWriterSuite.scala @@ -19,6 +19,9 @@ package com.uber.hoodie import org.scalatest.{FunSuite, Matchers} import DataSourceWriteOptions._ +import com.uber.hoodie.config.HoodieWriteConfig +import com.uber.hoodie.exception.HoodieException +import org.apache.spark.sql.{SaveMode, SparkSession} class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { @@ -38,4 +41,12 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { } } + test("throw hoodie exception when invalid serializer") { + val session = SparkSession.builder().appName("hoodie_test").master("local").getOrCreate() + val sqlContext = session.sqlContext + val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TABLE_NAME -> "hoodie_test_tbl") + val e = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.ErrorIfExists, options, session.emptyDataFrame)) + assert(e.getMessage.contains("spark.serializer")) + } + } From 0ff597097ad13494ba22775b2cb5d46127564cd9 Mon Sep 17 00:00:00 2001 From: Bhavani Sudha Saktheeswaran Date: Wed, 13 Feb 2019 12:54:01 -0800 Subject: [PATCH 152/374] Close FSDataInputStream for meta file open in HoodiePartitionMetadata --- .../hoodie/common/model/HoodiePartitionMetadata.java | 9 +++++++-- .../java/com/uber/hoodie/hadoop/HoodieInputFormat.java | 3 ++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java index 960042a84bd9a..cfdcebe0aca5d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java @@ -120,14 +120,19 @@ public void trySave(int taskPartitionId) { /** * Read out the metadata for this partition */ - public void readFromFS() { + public void readFromFS() throws IOException { + FSDataInputStream is = null; try { Path metaFile = new Path(partitionPath, HOODIE_PARTITION_METAFILE); - FSDataInputStream is = fs.open(metaFile); + is = fs.open(metaFile); props.load(is); } catch (IOException ioe) { throw new HoodieException("Error reading Hoodie partition metadata for " + partitionPath, ioe); + } finally { + if (is != null) { + is.close(); + } } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index c7931b5ce7e12..b8abff224fded 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -206,7 +206,8 @@ public RecordReader getRecordReader(final InputSplit split, * Read the table metadata from a data path. This assumes certain hierarchy of files which should * be changed once a better way is figured out to pass in the hoodie meta directory */ - protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path dataPath) { + protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path dataPath) + throws IOException { int levels = HoodieHiveUtil.DEFAULT_LEVELS_TO_BASEPATH; if (HoodiePartitionMetadata.hasPartitionMetadata(fs, dataPath)) { HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, dataPath); From 374c2877ecc05773bf2c03a3e1ee60ad73679862 Mon Sep 17 00:00:00 2001 From: Bhavani Sudha Saktheeswaran Date: Tue, 12 Feb 2019 17:13:49 -0800 Subject: [PATCH 153/374] Create hoodie-presto bundle jar Exclude common dependencies that are available in Presto --- packaging/hoodie-presto-bundle/pom.xml | 189 ++++++++++++++++++ .../com/uber/hoodie/presto/bundle/Main.java | 36 ++++ pom.xml | 1 + 3 files changed, 226 insertions(+) create mode 100644 packaging/hoodie-presto-bundle/pom.xml create mode 100644 packaging/hoodie-presto-bundle/src/main/java/com/uber/hoodie/presto/bundle/Main.java diff --git a/packaging/hoodie-presto-bundle/pom.xml b/packaging/hoodie-presto-bundle/pom.xml new file mode 100644 index 0000000000000..0fc3260aa55db --- /dev/null +++ b/packaging/hoodie-presto-bundle/pom.xml @@ -0,0 +1,189 @@ + + + + + + hoodie + com.uber.hoodie + 0.4.5-SNAPSHOT + ../../pom.xml + + 4.0.0 + + hoodie-presto-bundle + jar + + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-auth + + + com.google.guava + guava + + + org.apache.thrift + libthrift + 0.9.2 + + + + joda-time + joda-time + + + + + commons-dbcp + commons-dbcp + + + + commons-io + commons-io + + + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + com.beust + jcommander + + + + org.apache.httpcomponents + httpcore + + + + org.apache.httpcomponents + httpclient + + + + com.twitter + parquet-avro + + + + com.uber.hoodie + hoodie-hadoop-mr-bundle + ${project.version} + + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + 2.4 + + + package + + shade + + + + + com.beust. + com.uber.hoodie.com.beust. + + + org.joda. + com.uber.hoodie.org.joda. + + + com.google. + com.uber.hoodie.com.google. + + + org.slf4j. + com.uber.hoodie.org.slf4j. + + + org.apache.commons. + com.uber.hoodie.org.apache.commons. + + + parquet.column + com.uber.hoodie.parquet.column + + + parquet.format. + com.uber.hoodie.parquet.format. + + + parquet.hadoop. + com.uber.hoodie.parquet.hadoop. + + + parquet.schema. + com.uber.hoodie.parquet.schema. + + + false + + + log4j:log4j + org.apache.hadoop:* + org.apache.hive:* + org.apache.derby:derby + + org.apache.thrift:* + + org.apache.httpcomponents:* + + + ${project.artifactId}-${project.version} + + + + + + + + + true + + diff --git a/packaging/hoodie-presto-bundle/src/main/java/com/uber/hoodie/presto/bundle/Main.java b/packaging/hoodie-presto-bundle/src/main/java/com/uber/hoodie/presto/bundle/Main.java new file mode 100644 index 0000000000000..aacbd45854128 --- /dev/null +++ b/packaging/hoodie-presto-bundle/src/main/java/com/uber/hoodie/presto/bundle/Main.java @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * + */ + +package com.uber.hoodie.presto.bundle; + +import com.uber.hoodie.common.util.ReflectionUtils; + +/** + * A simple main class to dump all classes loaded in current classpath + * + * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating + * javadoc and sources plugins do not generate corresponding jars if there are no source files. + * + * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy. + */ +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} diff --git a/pom.xml b/pom.xml index a0ec312c51f2a..dfe9c0dd75396 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,7 @@ packaging/hoodie-hadoop-mr-bundle packaging/hoodie-hive-bundle packaging/hoodie-spark-bundle + packaging/hoodie-presto-bundle docker/hoodie/hadoop hoodie-integ-test From 91d3976ee1cce35822ae91b56e668d1295a2c608 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Wed, 27 Feb 2019 06:46:22 -0800 Subject: [PATCH 154/374] Update RELEASE_NOTES for 0.4.5 --- RELEASE_NOTES.md | 60 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index c95724796044e..90b6d42b9b874 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -1,3 +1,63 @@ +Release 0.4.5 +------------------------------------ + +### Highlights + * Dockerized demo with support for different Hive versions + * Smoother handling of append log on cloud stores + * Introducing a global bloom index, that enforces unique constraint across partitions + * CLI commands to analyze workloads, manage compactions + * Migration guide for folks wanting to move datasets to Hudi + * Added Spark Structured Streaming support, with a Hudi sink + * In-built support for filtering duplicates in DeltaStreamer + * Support for plugging in custom transformation in DeltaStreamer + * Better support for non-partitioned Hive tables + * Support hard deletes for Merge on Read storage + * New slack url & site urls + * Added presto bundle for easier integration + * Tons of bug fixes, reliability improvements + + +### Full PR List + + * **@bhasudha** - Create hoodie-presto bundle jar. fixes #567 #571 + * **@bhasudha** - Close FSDataInputStream for meta file open in HoodiePartitionMetadata . Fixes issue #573 #574 + * **@yaoqinn** - handle no such element exception in HoodieSparkSqlWriter #576 + * **@vinothchandar** - Update site url in README + * **@yaooqinn** - typo: bundle jar with unrecognized variables #570 + * **@bvaradar** - Table rollback for inflight compactions MUST not delete instant files at any time to avoid race conditions #565 + * **@bvaradar** - Fix Hoodie Record Reader to work with non-partitioned dataset ( ISSUE-561) #569 + * **@bvaradar** - Hoodie Delta Streamer Features : Transformation and Hoodie Incremental Source with Hive integration #485 + * **@vinothchandar** - Updating new slack signup link #566 + * **@yaooqinn** - Using immutable map instead of mutables to generate parameters #559 + * **@n3nash** - Fixing behavior of buffering in Create/Merge handles for invalid/wrong schema records #558 + * **@n3nash** - cleaner should now use commit timeline and not include deltacommits #539 + * **@n3nash** - Adding compaction to HoodieClient example #551 + * **@n3nash** - Filtering partition paths before performing a list status on all partitions #541 + * **@n3nash** - Passing a path filter to avoid including folders under .hoodie directory as partition paths #548 + * **@n3nash** - Enabling hard deletes for MergeOnRead table type #538 + * **@msridhar** - Add .m2 directory to Travis cache #534 + * **@artem0** - General enhancements #520 + * **@bvaradar** - Ensure Hoodie works for non-partitioned Hive table #515 + * **@xubo245** - fix some spell errorin Hudi #530 + * **@leletan** - feat(SparkDataSource): add structured streaming sink #486 + * **@n3nash** - Serializing the complete payload object instead of serializing just the GenericRecord in HoodieRecordConverter #495 + * **@n3nash** - Returning empty Statues for an empty spark partition caused due to incorrect bin packing #510 + * **@bvaradar** - Avoid WriteStatus collect() call when committing batch to prevent Driver side OOM errors #512 + * **@vinothchandar** - Explicitly handle lack of append() support during LogWriting #511 + * **@n3nash** - Fixing number of insert buckets to be generated by rounding off to the closest greater integer #500 + * **@vinothchandar** - Enabling auto tuning of insert splits by default #496 + * **@bvaradar** - Useful Hudi CLI commands to debug/analyze production workloads #477 + * **@bvaradar** - Compaction validate, unschedule and repair #481 + * **@shangxinli** - Fix addMetadataFields() to carry over 'props' #484 + * **@n3nash** - Adding documentation for migration guide and COW vs MOR tradeoffs #470 + * **@leletan** - Add additional feature to drop later arriving dups #468 + * **@bvaradar** - Fix regression bug which broke HoodieInputFormat handling of non-hoodie datasets #482 + * **@vinothchandar** - Add --filter-dupes to DeltaStreamer #478 + * **@bvaradar** - A quickstart demo to showcase Hudi functionalities using docker along with support for integration-tests #455 + * **@bvaradar** - Ensure Hoodie metadata folder and files are filtered out when constructing Parquet Data Source #473 + * **@leletan** - Adds HoodieGlobalBloomIndex #438 + + Release 0.4.4 ------------------------------------ From a117a33d7abf35df6a148522ac6730ea12e8c98b Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Wed, 27 Feb 2019 07:16:15 -0800 Subject: [PATCH 155/374] [maven-release-plugin] prepare release hoodie-0.4.5 --- docker/hoodie/hadoop/base/pom.xml | 2 +- docker/hoodie/hadoop/datanode/pom.xml | 2 +- docker/hoodie/hadoop/historyserver/pom.xml | 2 +- docker/hoodie/hadoop/hive_base/pom.xml | 14 +++++--------- docker/hoodie/hadoop/namenode/pom.xml | 2 +- docker/hoodie/hadoop/pom.xml | 2 +- docker/hoodie/hadoop/spark_base/pom.xml | 2 +- docker/hoodie/hadoop/sparkadhoc/pom.xml | 2 +- docker/hoodie/hadoop/sparkmaster/pom.xml | 2 +- docker/hoodie/hadoop/sparkworker/pom.xml | 2 +- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-integ-test/pom.xml | 5 ++--- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 2 +- packaging/hoodie-hive-bundle/pom.xml | 2 +- packaging/hoodie-presto-bundle/pom.xml | 2 +- packaging/hoodie-spark-bundle/pom.xml | 2 +- pom.xml | 4 ++-- 23 files changed, 29 insertions(+), 34 deletions(-) diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml index c975319e371d5..6e0f6d414ce1b 100644 --- a/docker/hoodie/hadoop/base/pom.xml +++ b/docker/hoodie/hadoop/base/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml index 94f5a4892ef24..c4e6715c0be9d 100644 --- a/docker/hoodie/hadoop/datanode/pom.xml +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml index 2e7e002545d14..62f0d177ec965 100644 --- a/docker/hoodie/hadoop/historyserver/pom.xml +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml index ec6b4fa6a56a2..c07b0bb068906 100644 --- a/docker/hoodie/hadoop/hive_base/pom.xml +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom @@ -52,14 +52,10 @@ package - - - - + + + + diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml index 2604a99a6a716..66b5f0cd63880 100644 --- a/docker/hoodie/hadoop/namenode/pom.xml +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index 7ef639c7f33de..7d0de454bdf37 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 ../../../pom.xml 4.0.0 diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml index fbd34e9909da9..c77987a2fd1af 100644 --- a/docker/hoodie/hadoop/spark_base/pom.xml +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml index b77a4c0a771aa..0a5a99daa449e 100644 --- a/docker/hoodie/hadoop/sparkadhoc/pom.xml +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml index f988ed9ef2516..53bd343873275 100644 --- a/docker/hoodie/hadoop/sparkmaster/pom.xml +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml index 93064bb11c3f4..81b57205bc913 100644 --- a/docker/hoodie/hadoop/sparkworker/pom.xml +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 pom diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 2920f81ccc899..18c41ab66b3b2 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index c813ef24a499b..2ac058d8594fa 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index e56d35279da68..35f1defa1527c 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 6ef41a32d24b0..8e83160c3b78e 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index da4e9a0d7e299..12bbd8701b75a 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/hoodie-integ-test/pom.xml b/hoodie-integ-test/pom.xml index ecacc9420fee2..3b677a96b5b44 100644 --- a/hoodie-integ-test/pom.xml +++ b/hoodie-integ-test/pom.xml @@ -15,12 +15,11 @@ ~ limitations under the License. --> - + hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 ../pom.xml hoodie-integ-test diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index d800c1a7218b3..5a4b8d83bf17a 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 3cbf468d7c136..01c184ab64f4a 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index f25b5d1a5c9cf..a7b46dffeeba4 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index e9843741e29df..6cd9ec21e35a7 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-presto-bundle/pom.xml b/packaging/hoodie-presto-bundle/pom.xml index 0fc3260aa55db..056c5591968c1 100644 --- a/packaging/hoodie-presto-bundle/pom.xml +++ b/packaging/hoodie-presto-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index b918b0b7fa6ab..1784aa5c1375b 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.5-SNAPSHOT + 0.4.5 ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index dfe9c0dd75396..efdc2ea4b5517 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.5-SNAPSHOT + 0.4.5 Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -144,7 +144,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - HEAD + hoodie-0.4.5 From 8ba34aaf72a1f7d731fe0d9ee75671a87b67b676 Mon Sep 17 00:00:00 2001 From: vinothchandar Date: Wed, 27 Feb 2019 07:16:27 -0800 Subject: [PATCH 156/374] [maven-release-plugin] prepare for next development iteration --- docker/hoodie/hadoop/base/pom.xml | 2 +- docker/hoodie/hadoop/datanode/pom.xml | 2 +- docker/hoodie/hadoop/historyserver/pom.xml | 2 +- docker/hoodie/hadoop/hive_base/pom.xml | 2 +- docker/hoodie/hadoop/namenode/pom.xml | 2 +- docker/hoodie/hadoop/pom.xml | 2 +- docker/hoodie/hadoop/spark_base/pom.xml | 2 +- docker/hoodie/hadoop/sparkadhoc/pom.xml | 2 +- docker/hoodie/hadoop/sparkmaster/pom.xml | 2 +- docker/hoodie/hadoop/sparkworker/pom.xml | 2 +- hoodie-cli/pom.xml | 2 +- hoodie-client/pom.xml | 2 +- hoodie-common/pom.xml | 2 +- hoodie-hadoop-mr/pom.xml | 2 +- hoodie-hive/pom.xml | 2 +- hoodie-integ-test/pom.xml | 2 +- hoodie-spark/pom.xml | 2 +- hoodie-utilities/pom.xml | 2 +- packaging/hoodie-hadoop-mr-bundle/pom.xml | 2 +- packaging/hoodie-hive-bundle/pom.xml | 2 +- packaging/hoodie-presto-bundle/pom.xml | 2 +- packaging/hoodie-spark-bundle/pom.xml | 2 +- pom.xml | 4 ++-- 23 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml index 6e0f6d414ce1b..4bd40c0a9e3cb 100644 --- a/docker/hoodie/hadoop/base/pom.xml +++ b/docker/hoodie/hadoop/base/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml index c4e6715c0be9d..70691ce797726 100644 --- a/docker/hoodie/hadoop/datanode/pom.xml +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml index 62f0d177ec965..4f3b7b9a64f50 100644 --- a/docker/hoodie/hadoop/historyserver/pom.xml +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml index c07b0bb068906..c8eebbb06f19a 100644 --- a/docker/hoodie/hadoop/hive_base/pom.xml +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml index 66b5f0cd63880..841ac62ccc193 100644 --- a/docker/hoodie/hadoop/namenode/pom.xml +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index 7d0de454bdf37..d7e1a98b977d5 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT ../../../pom.xml 4.0.0 diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml index c77987a2fd1af..841d58a977f09 100644 --- a/docker/hoodie/hadoop/spark_base/pom.xml +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml index 0a5a99daa449e..4e5b7662492f0 100644 --- a/docker/hoodie/hadoop/sparkadhoc/pom.xml +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml index 53bd343873275..bce4511eb8478 100644 --- a/docker/hoodie/hadoop/sparkmaster/pom.xml +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml index 81b57205bc913..8c0c77af9113b 100644 --- a/docker/hoodie/hadoop/sparkworker/pom.xml +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -19,7 +19,7 @@ hoodie-hadoop-docker com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 pom diff --git a/hoodie-cli/pom.xml b/hoodie-cli/pom.xml index 18c41ab66b3b2..32a5fb6233386 100644 --- a/hoodie-cli/pom.xml +++ b/hoodie-cli/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml index 2ac058d8594fa..368d8cde01d51 100644 --- a/hoodie-client/pom.xml +++ b/hoodie-client/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index 35f1defa1527c..5a15ad187121f 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/hoodie-hadoop-mr/pom.xml b/hoodie-hadoop-mr/pom.xml index 8e83160c3b78e..3842c771aea20 100644 --- a/hoodie-hadoop-mr/pom.xml +++ b/hoodie-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index 12bbd8701b75a..0207bad0505f7 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/hoodie-integ-test/pom.xml b/hoodie-integ-test/pom.xml index 3b677a96b5b44..0e7d273a5486d 100644 --- a/hoodie-integ-test/pom.xml +++ b/hoodie-integ-test/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT ../pom.xml hoodie-integ-test diff --git a/hoodie-spark/pom.xml b/hoodie-spark/pom.xml index 5a4b8d83bf17a..ac12f1e87b1d3 100644 --- a/hoodie-spark/pom.xml +++ b/hoodie-spark/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml index 01c184ab64f4a..c95950a4db8b1 100644 --- a/hoodie-utilities/pom.xml +++ b/hoodie-utilities/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT 4.0.0 diff --git a/packaging/hoodie-hadoop-mr-bundle/pom.xml b/packaging/hoodie-hadoop-mr-bundle/pom.xml index a7b46dffeeba4..b80b5ede49404 100644 --- a/packaging/hoodie-hadoop-mr-bundle/pom.xml +++ b/packaging/hoodie-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index 6cd9ec21e35a7..52021c30901f8 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-presto-bundle/pom.xml b/packaging/hoodie-presto-bundle/pom.xml index 056c5591968c1..c4a2bdb994d9f 100644 --- a/packaging/hoodie-presto-bundle/pom.xml +++ b/packaging/hoodie-presto-bundle/pom.xml @@ -19,7 +19,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hoodie-spark-bundle/pom.xml b/packaging/hoodie-spark-bundle/pom.xml index 1784aa5c1375b..b62557d31f119 100644 --- a/packaging/hoodie-spark-bundle/pom.xml +++ b/packaging/hoodie-spark-bundle/pom.xml @@ -21,7 +21,7 @@ hoodie com.uber.hoodie - 0.4.5 + 0.4.6-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index efdc2ea4b5517..e267ac2b383e5 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ com.uber.hoodie hoodie pom - 0.4.5 + 0.4.6-SNAPSHOT Hoodie is a Apache Spark library that provides the ability to efficiently do incremental processing on datasets in HDFS @@ -144,7 +144,7 @@ scm:git:git@github.com:uber/hoodie.git scm:git:git@github.com:uber/hoodie.git git@github.com:uber/hoodie.git - hoodie-0.4.5 + HEAD From bddc5b17cca04afdd953035f3540118802e9a0c6 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 13 Feb 2019 19:53:28 -0800 Subject: [PATCH 157/374] Upgrade various jar, gem versions for maintenance --- docs/Gemfile.lock | 44 ++++++++++++++++++++++---------------------- hoodie-hive/pom.xml | 2 +- pom.xml | 11 +++-------- 3 files changed, 26 insertions(+), 31 deletions(-) diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index 65eae02f0a366..8359d57575c75 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -13,19 +13,19 @@ GEM execjs coffee-script-source (1.12.2) colorator (1.1.0) - concurrent-ruby (1.0.5) - ethon (0.11.0) + concurrent-ruby (1.1.4) + ethon (0.12.0) ffi (>= 1.3.0) execjs (2.7.0) - faraday (0.13.1) + faraday (0.15.4) multipart-post (>= 1.2, < 3) - ffi (1.9.24) + ffi (1.10.0) forwardable-extended (2.6.0) gemoji (2.1.0) github-pages (106) activesupport (= 4.2.7) github-pages-health-check (= 1.2.0) - jekyll (= 3.6.3) + jekyll (= 3.3.1) jekyll-avatar (= 0.4.2) jekyll-coffeescript (= 1.0.1) jekyll-feed (= 0.8.0) @@ -53,12 +53,12 @@ GEM octokit (~> 4.0) public_suffix (~> 1.4) typhoeus (~> 0.7) - html-pipeline (2.7.1) + html-pipeline (2.10.0) activesupport (>= 2) nokogiri (>= 1.4) - i18n (0.9.1) + i18n (0.9.5) concurrent-ruby (~> 1.0) - jekyll (3.6.3) + jekyll (3.3.1) addressable (~> 2.4) colorator (~> 1.0) jekyll-sass-converter (~> 1.0) @@ -110,24 +110,24 @@ GEM rb-fsevent (>= 0.9.3) rb-inotify (>= 0.9.7) mercenary (0.3.6) - mini_portile2 (2.3.0) + mini_portile2 (2.4.0) minima (2.0.0) - minitest (5.11.1) + minitest (5.11.3) multipart-post (2.0.0) - net-dns (0.8.0) - nokogiri (1.8.2) - mini_portile2 (~> 2.3.0) - octokit (4.8.0) + net-dns (0.9.0) + nokogiri (1.10.1) + mini_portile2 (~> 2.4.0) + octokit (4.13.0) sawyer (~> 0.8.0, >= 0.5.3) - pathutil (0.16.1) + pathutil (0.16.2) forwardable-extended (~> 2.6) public_suffix (1.5.3) - rb-fsevent (0.10.2) - rb-inotify (0.9.10) - ffi (>= 0.5.0, < 2) + rb-fsevent (0.10.3) + rb-inotify (0.10.0) + ffi (~> 1.0) rouge (1.11.1) safe_yaml (1.0.4) - sass (3.5.5) + sass (3.7.3) sass-listen (~> 4.0.0) sass-listen (4.0.0) rb-fsevent (~> 0.9, >= 0.9.4) @@ -140,16 +140,16 @@ GEM thread_safe (0.3.6) typhoeus (0.8.0) ethon (>= 0.8.0) - tzinfo (1.2.4) + tzinfo (1.2.5) thread_safe (~> 0.1) - unicode-display_width (1.3.0) + unicode-display_width (1.4.1) PLATFORMS ruby DEPENDENCIES github-pages (~> 106) - jekyll (= 3.6.3) + jekyll (= 3.3.1) jekyll-feed (~> 0.6) BUNDLED WITH diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index 0207bad0505f7..ff7487aa8dfbf 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -50,7 +50,7 @@ org.apache.thrift libthrift - 0.9.2 + 0.12.0 diff --git a/pom.xml b/pom.xml index e267ac2b383e5..b9e9bb87693fb 100644 --- a/pom.xml +++ b/pom.xml @@ -123,7 +123,7 @@ 2.10 2.6 2.19.1 - 2.6.5 + 2.8.11 1.8.1 4.11 1.9.5 @@ -601,7 +601,7 @@ org.apache.httpcomponents httpclient - 4.3.2 + 4.3.6 org.slf4j @@ -635,7 +635,7 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.version} + ${fasterxml.version}.1 @@ -650,11 +650,6 @@ 1.9.13 - - com.fasterxml.jackson.core - jackson-databind - 2.6.0 - org.codehaus.jackson jackson-mapper-asl From d04dfbc0f4179aac188408898a0cff382b7a25fc Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 12 Feb 2019 21:29:14 -0800 Subject: [PATCH 158/374] FileSystem View must treat same fileIds present in different partitions as different file-groups and handle pending compaction correctly --- .../cli/commands/FileSystemViewCommand.java | 2 +- .../uber/hoodie/CompactionAdminClient.java | 27 +-- .../uber/hoodie/io/HoodieAppendHandle.java | 2 +- .../com/uber/hoodie/io/HoodieCleanHelper.java | 9 +- .../hoodie/io/compact/HoodieCompactor.java | 4 +- .../compact/HoodieRealtimeTableCompactor.java | 12 +- .../hoodie/table/HoodieMergeOnReadTable.java | 2 +- .../com/uber/hoodie/TestAsyncCompaction.java | 11 +- .../java/com/uber/hoodie/TestCleaner.java | 15 +- .../hoodie/TestCompactionAdminClient.java | 2 +- .../common/model/CompactionOperation.java | 19 +- .../uber/hoodie/common/model/FileSlice.java | 23 ++- .../hoodie/common/model/HoodieFileGroup.java | 32 ++-- .../common/model/HoodieFileGroupId.java | 69 ++++++++ .../table/view/HoodieTableFileSystemView.java | 55 +++--- .../hoodie/common/util/CompactionUtils.java | 19 +- .../view/HoodieTableFileSystemViewTest.java | 166 +++++++++++++++++- .../common/util/CompactionTestUtils.java | 14 +- .../common/util/TestCompactionUtils.java | 19 +- .../utilities/HoodieCompactionAdminTool.java | 6 +- 20 files changed, 388 insertions(+), 120 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroupId.java diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java index dd42c47e203fe..5d3d098a00327 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java @@ -78,7 +78,7 @@ public String showAllFileSlices( // For ReadOptimized Views, do not display any delta-file related columns Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 8]; row[idx++] = fg.getPartitionPath(); - row[idx++] = fg.getId(); + row[idx++] = fg.getFileGroupId().getFileId(); row[idx++] = fs.getBaseInstantTime(); row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : ""; row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java index 6a43c9c88da53..715bb3d4fcd79 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -145,26 +146,28 @@ public List unscheduleCompactionPlan( * * This operation MUST be executed with compactions and writer turned OFF. * - * @param fileId FileId to be unscheduled + * @param fgId FileGroupId to be unscheduled * @param skipValidation Skip validation * @param dryRun Dry Run Mode */ - public List unscheduleCompactionFileId(String fileId, + public List unscheduleCompactionFileId(HoodieFileGroupId fgId, boolean skipValidation, boolean dryRun) throws Exception { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); List> renameActions = - getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fileId, Optional.absent(), skipValidation); + getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId, + Optional.absent(), skipValidation); List res = runRenamingOps(metaClient, renameActions, 1, dryRun); if (!dryRun && !res.isEmpty() && res.get(0).isExecuted() && res.get(0).isSuccess()) { // Ready to remove this file-Id from compaction request Pair compactionOperationWithInstant = - CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fileId); + CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fgId); HoodieCompactionPlan plan = CompactionUtils .getCompactionPlan(metaClient, compactionOperationWithInstant.getKey()); List newOps = plan.getOperations().stream() - .filter(op -> !op.getFileId().equals(fileId)).collect(Collectors.toList()); + .filter(op -> (!op.getFileId().equals(fgId.getFileId())) + && (!op.getPartitionPath().equals(fgId.getPartitionPath()))).collect(Collectors.toList()); HoodieCompactionPlan newPlan = HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build(); HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, @@ -465,23 +468,23 @@ public List> getRenamingActionsForUnschedulin * writer (ingestion/compaction) is running. * * @param metaClient Hoodie Table MetaClient - * @param fileId FileId to remove compaction + * @param fgId FileGroupId to remove compaction * @param fsViewOpt Cached File System View * @param skipValidation Skip Validation * @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule * compaction. */ public List> getRenamingActionsForUnschedulingCompactionForFileId( - HoodieTableMetaClient metaClient, String fileId, Optional fsViewOpt, - boolean skipValidation) throws IOException { - Map> allPendingCompactions = + HoodieTableMetaClient metaClient, HoodieFileGroupId fgId, + Optional fsViewOpt, boolean skipValidation) throws IOException { + Map> allPendingCompactions = CompactionUtils.getAllPendingCompactionOperations(metaClient); - if (allPendingCompactions.containsKey(fileId)) { - Pair opWithInstant = allPendingCompactions.get(fileId); + if (allPendingCompactions.containsKey(fgId)) { + Pair opWithInstant = allPendingCompactions.get(fgId); return getRenamingActionsForUnschedulingCompactionOperation(metaClient, opWithInstant.getKey(), CompactionOperation.convertFromAvroRecordInstance(opWithInstant.getValue()), fsViewOpt, skipValidation); } - throw new HoodieException("FileId " + fileId + " not in pending compaction"); + throw new HoodieException("FileGroupId " + fgId + " not in pending compaction"); } /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 516ab5b784da5..69e36ee6b05f6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -118,7 +118,7 @@ private void init(HoodieRecord record) { baseInstantTime = fileSlice.get().getBaseInstantTime(); } else { // This means there is no base data file, start appending to a new log file - fileSlice = Optional.of(new FileSlice(baseInstantTime, this.fileId)); + fileSlice = Optional.of(new FileSlice(partitionPath, baseInstantTime, this.fileId)); logger.info("New InsertHandle for partition :" + partitionPath); } writeStatus.getStat().setPrevCommit(baseInstantTime); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index 537bbedaf89be..aeb99554792d5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -21,6 +21,7 @@ import com.uber.hoodie.common.model.HoodieCleaningPolicy; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieFileGroup; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTimeline; @@ -52,7 +53,7 @@ public class HoodieCleanHelper> { private final TableFileSystemView fileSystemView; private final HoodieTimeline commitTimeline; - private final Map fileIdToPendingCompactionOperations; + private final Map fgIdToPendingCompactionOperations; private HoodieTable hoodieTable; private HoodieWriteConfig config; @@ -61,8 +62,8 @@ public HoodieCleanHelper(HoodieTable hoodieTable, HoodieWriteConfig config) { this.fileSystemView = hoodieTable.getCompletedFileSystemView(); this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); this.config = config; - this.fileIdToPendingCompactionOperations = - ((HoodieTableFileSystemView)hoodieTable.getRTFileSystemView()).getFileIdToPendingCompaction().entrySet() + this.fgIdToPendingCompactionOperations = + ((HoodieTableFileSystemView)hoodieTable.getRTFileSystemView()).getFgIdToPendingCompaction().entrySet() .stream().map(entry -> Pair.of(entry.getKey(), entry.getValue().getValue())) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } @@ -249,7 +250,7 @@ public Optional getEarliestCommitToRetain() { * @return true if file slice needs to be preserved, false otherwise. */ private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) { - CompactionOperation op = fileIdToPendingCompactionOperations.get(fileSlice.getFileId()); + CompactionOperation op = fgIdToPendingCompactionOperations.get(fileSlice.getFileGroupId()); if (null != op) { // If file slice's instant time is newer or same as that of operation, do not clean return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), op.getBaseInstantTime(), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index 27f04c426c48e..84cadbfedda85 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -18,6 +18,7 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; @@ -38,12 +39,13 @@ public interface HoodieCompactor extends Serializable { * @param hoodieTable Hoodie Table * @param config Hoodie Write Configuration * @param compactionCommitTime scheduled compaction commit time + * @param fgIdsInPendingCompactions partition-fileId pairs for which compaction is pending * @return Compaction Plan * @throws IOException when encountering errors */ HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime, - Set fileIdsWithPendingCompactions) + Set fgIdsInPendingCompactions) throws IOException; /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 549034e3bb6cc..9848631aa8060 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -26,6 +26,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; @@ -160,7 +161,7 @@ private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, @Override public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime, - Set fileIdsWithPendingCompactions) throws IOException { + Set fgIdsInPendingCompactions) throws IOException { totalLogFiles = new LongAccumulator(); totalFileSlices = new LongAccumulator(); @@ -190,7 +191,8 @@ public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, jsc.parallelize(partitionPaths, partitionPaths.size()) .flatMap((FlatMapFunction) partitionPath -> fileSystemView .getLatestFileSlices(partitionPath) - .filter(slice -> !fileIdsWithPendingCompactions.contains(slice.getFileId())) + .filter(slice -> + !fgIdsInPendingCompactions.contains(slice.getFileGroupId())) .map( s -> { List logFiles = s.getLogFiles().sorted(HoodieLogFile @@ -215,11 +217,11 @@ public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, // compactions only HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations, CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); - Preconditions.checkArgument(compactionPlan.getOperations().stream() - .filter(op -> fileIdsWithPendingCompactions.contains(op.getFileId())).count() == 0, + Preconditions.checkArgument(compactionPlan.getOperations().stream().noneMatch( + op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. " + "Please fix your strategy implementation." - + "FileIdsWithPendingCompactions :" + fileIdsWithPendingCompactions + + "FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions + ", Selected workload :" + compactionPlan); if (compactionPlan.getOperations().isEmpty()) { log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 29280a49efd83..83c9fb64cc47e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -147,7 +147,7 @@ public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String inst try { return compactor.generateCompactionPlan(jsc, this, config, instantTime, new HashSet<>(((HoodieTableFileSystemView)getRTFileSystemView()) - .getFileIdToPendingCompaction().keySet())); + .getFgIdToPendingCompaction().keySet())); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java index 0df9da859e8e3..4684eef1e23fa 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; @@ -398,13 +399,13 @@ public void testInterleavedCompaction() throws Exception { **/ private void validateDeltaCommit(String latestDeltaCommit, - final Map> fileIdToCompactionOperation, + final Map> fgIdToCompactionOperation, HoodieWriteConfig cfg) throws IOException { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); List fileSliceList = getCurrentLatestFileSlices(table, cfg); fileSliceList.forEach(fileSlice -> { - Pair opPair = fileIdToCompactionOperation.get(fileSlice.getFileId()); + Pair opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId()); if (opPair != null) { System.out.println("FileSlice :" + fileSlice); assertTrue("Expect baseInstant to match compaction Instant", @@ -430,7 +431,7 @@ private List runNextDeltaCommits(HoodieWriteClient client, List pc.getKey().getTimestamp()).sorted().collect(Collectors.toList()); assertEquals(expPendingCompactionInstants, gotPendingCompactionInstants); - Map> fileIdToCompactionOperation = + Map> fgIdToCompactionOperation = CompactionUtils.getAllPendingCompactionOperations(metaClient); if (insertFirst) { @@ -451,7 +452,7 @@ private List runNextDeltaCommits(HoodieWriteClient client, List dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.stream().findAny().isPresent()); - validateDeltaCommit(firstInstant, fileIdToCompactionOperation, cfg); + validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg); } int numRecords = records.size(); @@ -459,7 +460,7 @@ private List runNextDeltaCommits(HoodieWriteClient client, List selectedFileIdForCompaction = new HashMap<>(); - Map compactionFileIdToLatestFileSlice = new HashMap<>(); + Map compactionFileIdToLatestFileSlice = new HashMap<>(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); for (String partitionPath : dataGen.getPartitionPaths()) { TableFileSystemView fsView = table.getFileSystemView(); Optional added = fsView.getAllFileGroups(partitionPath).findFirst() .map(fg -> { - selectedFileIdForCompaction.put(fg.getId(), partitionPath); - fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getId(), fs)); + fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getFileGroupId(), fs)); return true; }); if (added.isPresent()) { @@ -224,7 +223,7 @@ private void testInsertAndCleanByVersions( // Create workload with selected file-slices List> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream() - .map(e -> Pair.of(selectedFileIdForCompaction.get(e.getKey()), e.getValue())).collect(Collectors.toList()); + .map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList()); HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Optional.empty(), Optional.empty()); List instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1); @@ -270,18 +269,18 @@ private void testInsertAndCleanByVersions( List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { - if (selectedFileIdForCompaction.containsKey(fileGroup.getId())) { + if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) { // Ensure latest file-slice selected for compaction is retained Optional dataFileForCompactionPresent = fileGroup.getAllDataFiles().filter(df -> { - return compactionFileIdToLatestFileSlice.get(fileGroup.getId()) + return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId()) .getBaseInstantTime().equals(df.getCommitTime()); }).findAny(); Assert.assertTrue("Data File selected for compaction is retained", dataFileForCompactionPresent.isPresent()); } else { // file has no more than max versions - String fileId = fileGroup.getId(); + String fileId = fileGroup.getFileGroupId().getFileId(); List dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList()); assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions", diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java index a385f09475fb4..0ed1f7c8872fe 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java @@ -334,7 +334,7 @@ private void validateUnScheduleFileId(CompactionAdminClient client, String inges .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); // Call the main unschedule API - client.unscheduleCompactionFileId(op.getFileId(), false, false); + client.unscheduleCompactionFileId(op.getFileGroupId(), false, false); metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); final HoodieTableFileSystemView newFsView = diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java index b094c49c55b11..93a7f2e87b9d7 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java @@ -38,8 +38,7 @@ public class CompactionOperation implements Serializable { private Optional dataFileCommitTime; private List deltaFilePaths; private Optional dataFilePath; - private String fileId; - private String partitionPath; + private HoodieFileGroupId id; private Map metrics; //Only for serialization/de-serialization @@ -52,17 +51,16 @@ public CompactionOperation(java.util.Optional dataFile, String p if (dataFile.isPresent()) { this.baseInstantTime = dataFile.get().getCommitTime(); this.dataFilePath = Optional.of(dataFile.get().getPath()); - this.fileId = dataFile.get().getFileId(); + this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId()); this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime()); } else { assert logFiles.size() > 0; this.dataFilePath = Optional.absent(); this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath()); - this.fileId = FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()); + this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath())); this.dataFileCommitTime = Optional.absent(); } - this.partitionPath = partitionPath; this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) .collect(Collectors.toList()); this.metrics = metrics; @@ -85,17 +83,21 @@ public Optional getDataFilePath() { } public String getFileId() { - return fileId; + return id.getFileId(); } public String getPartitionPath() { - return partitionPath; + return id.getPartitionPath(); } public Map getMetrics() { return metrics; } + public HoodieFileGroupId getFileGroupId() { + return id; + } + /** * Convert Avro generated Compaction operation to POJO for Spark RDD operation * @param operation Hoodie Compaction Operation @@ -106,9 +108,8 @@ public static CompactionOperation convertFromAvroRecordInstance(HoodieCompaction op.baseInstantTime = operation.getBaseInstantTime(); op.dataFilePath = Optional.fromNullable(operation.getDataFilePath()); op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths()); - op.fileId = operation.getFileId(); + op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId()); op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics()); - op.partitionPath = operation.getPartitionPath(); return op; } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java index 0720fc99b134b..30ef2dcfdded4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java @@ -30,9 +30,9 @@ public class FileSlice implements Serializable { /** - * id of the slice + * File Group Id of the Slice */ - private String fileId; + private HoodieFileGroupId fileGroupId; /** * Point in the timeline, at which the slice was created @@ -50,8 +50,12 @@ public class FileSlice implements Serializable { */ private final TreeSet logFiles; - public FileSlice(String baseInstantTime, String fileId) { - this.fileId = fileId; + public FileSlice(String partitionPath, String baseInstantTime, String fileId) { + this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime); + } + + public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime) { + this.fileGroupId = fileGroupId; this.baseInstantTime = baseInstantTime; this.dataFile = null; this.logFiles = new TreeSet<>(HoodieLogFile.getBaseInstantAndLogVersionComparator()); @@ -73,8 +77,16 @@ public String getBaseInstantTime() { return baseInstantTime; } + public String getPartitionPath() { + return fileGroupId.getPartitionPath(); + } + public String getFileId() { - return fileId; + return fileGroupId.getFileId(); + } + + public HoodieFileGroupId getFileGroupId() { + return fileGroupId; } public Optional getDataFile() { @@ -84,6 +96,7 @@ public Optional getDataFile() { @Override public String toString() { final StringBuilder sb = new StringBuilder("FileSlice {"); + sb.append("fileGroupId=").append(fileGroupId); sb.append("baseCommitTime=").append(baseInstantTime); sb.append(", dataFile='").append(dataFile).append('\''); sb.append(", logFiles='").append(logFiles).append('\''); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java index 01aac41272d54..798a6714e5ada 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java @@ -34,14 +34,9 @@ public static Comparator getReverseCommitTimeComparator() { } /** - * Partition containing the file group. + * file group id */ - private final String partitionPath; - - /** - * uniquely identifies the file group - */ - private final String id; + private final HoodieFileGroupId fileGroupId; /** * Slices of files in this group, sorted with greater commit first. @@ -59,8 +54,11 @@ public static Comparator getReverseCommitTimeComparator() { private final Optional lastInstant; public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) { - this.partitionPath = partitionPath; - this.id = id; + this(new HoodieFileGroupId(partitionPath, id), timeline); + } + + public HoodieFileGroup(HoodieFileGroupId fileGroupId, HoodieTimeline timeline) { + this.fileGroupId = fileGroupId; this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator()); this.timeline = timeline; this.lastInstant = timeline.lastInstant(); @@ -72,7 +70,7 @@ public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) */ public void addNewFileSliceAtInstant(String baseInstantTime) { if (!fileSlices.containsKey(baseInstantTime)) { - fileSlices.put(baseInstantTime, new FileSlice(baseInstantTime, id)); + fileSlices.put(baseInstantTime, new FileSlice(fileGroupId, baseInstantTime)); } } @@ -81,7 +79,7 @@ public void addNewFileSliceAtInstant(String baseInstantTime) { */ public void addDataFile(HoodieDataFile dataFile) { if (!fileSlices.containsKey(dataFile.getCommitTime())) { - fileSlices.put(dataFile.getCommitTime(), new FileSlice(dataFile.getCommitTime(), id)); + fileSlices.put(dataFile.getCommitTime(), new FileSlice(fileGroupId, dataFile.getCommitTime())); } fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile); } @@ -91,17 +89,17 @@ public void addDataFile(HoodieDataFile dataFile) { */ public void addLogFile(HoodieLogFile logFile) { if (!fileSlices.containsKey(logFile.getBaseCommitTime())) { - fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(logFile.getBaseCommitTime(), id)); + fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(fileGroupId, logFile.getBaseCommitTime())); } fileSlices.get(logFile.getBaseCommitTime()).addLogFile(logFile); } - public String getId() { - return id; + public String getPartitionPath() { + return fileGroupId.getPartitionPath(); } - public String getPartitionPath() { - return partitionPath; + public HoodieFileGroupId getFileGroupId() { + return fileGroupId; } /** @@ -197,7 +195,7 @@ public Stream getAllDataFiles() { @Override public String toString() { final StringBuilder sb = new StringBuilder("HoodieFileGroup {"); - sb.append("id=").append(id); + sb.append("id=").append(fileGroupId); sb.append(", fileSlices='").append(fileSlices).append('\''); sb.append('}'); return sb.toString(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroupId.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroupId.java new file mode 100644 index 0000000000000..98f51b439c243 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroupId.java @@ -0,0 +1,69 @@ +/* + * Copyright (c) 2019 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.model; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Unique ID to identify a file-group in a data-set + */ +public class HoodieFileGroupId implements Serializable { + + private final String partitionPath; + + private final String fileId; + + public HoodieFileGroupId(String partitionPath, String fileId) { + this.partitionPath = partitionPath; + this.fileId = fileId; + } + + public String getPartitionPath() { + return partitionPath; + } + + public String getFileId() { + return fileId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HoodieFileGroupId that = (HoodieFileGroupId) o; + return Objects.equals(partitionPath, that.partitionPath) + && Objects.equals(fileId, that.fileId); + } + + @Override + public int hashCode() { + return Objects.hash(partitionPath, fileId); + } + + @Override + public String toString() { + return "HoodieFileGroupId{" + + "partitionPath='" + partitionPath + '\'' + + ", fileId='" + fileId + '\'' + + '}'; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index cf5cb9eecd7be..93f61b7c34062 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -21,6 +21,7 @@ import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieFileGroup; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -70,12 +71,12 @@ public class HoodieTableFileSystemView implements TableFileSystemView, // mapping from partition paths to file groups contained within them protected HashMap> partitionToFileGroupsMap; // mapping from file id to the file group. - protected HashMap fileGroupMap; + protected HashMap fileGroupMap; /** - * File Id to pending compaction instant time + * PartitionPath + File-Id to pending compaction instant time */ - private final Map> fileIdToPendingCompaction; + private final Map> fgIdToPendingCompaction; /** * Create a file system view, as of the given timeline @@ -90,7 +91,7 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, // Build fileId to Pending Compaction Instants List pendingCompactionInstants = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); - this.fileIdToPendingCompaction = ImmutableMap.copyOf( + this.fgIdToPendingCompaction = ImmutableMap.copyOf( CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream() .map(entry -> Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(), CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue())))) @@ -123,6 +124,10 @@ private void writeObject(java.io.ObjectOutputStream out) out.defaultWriteObject(); } + private String getPartitionPathFromFileStatus(FileStatus fileStatus) { + return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), fileStatus.getPath().getParent()); + } + /** * Adds the provided statuses into the file system view, and also caches it inside this object. */ @@ -130,9 +135,7 @@ private List addFilesToView(FileStatus[] statuses) { Map, List> dataFiles = convertFileStatusesToDataFiles( statuses) .collect(Collectors.groupingBy((dataFile) -> { - String partitionPathStr = FSUtils.getRelativePartitionPath( - new Path(metaClient.getBasePath()), - dataFile.getFileStatus().getPath().getParent()); + String partitionPathStr = getPartitionPathFromFileStatus(dataFile.getFileStatus()); return Pair.of(partitionPathStr, dataFile.getFileId()); })); Map, List> logFiles = convertFileStatusesToLogFiles( @@ -157,17 +160,18 @@ private List addFilesToView(FileStatus[] statuses) { if (logFiles.containsKey(pair)) { logFiles.get(pair).forEach(group::addLogFile); } - if (fileIdToPendingCompaction.containsKey(fileId)) { + HoodieFileGroupId fgId = group.getFileGroupId(); + if (fgIdToPendingCompaction.containsKey(fgId)) { // If there is no delta-commit after compaction request, this step would ensure a new file-slice appears // so that any new ingestion uses the correct base-instant - group.addNewFileSliceAtInstant(fileIdToPendingCompaction.get(fileId).getKey()); + group.addNewFileSliceAtInstant(fgIdToPendingCompaction.get(fgId).getKey()); } fileGroups.add(group); }); // add to the cache. fileGroups.forEach(group -> { - fileGroupMap.put(group.getId(), group); + fileGroupMap.put(group.getFileGroupId(), group); if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) { partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>()); } @@ -198,7 +202,9 @@ private Stream convertFileStatusesToLogFiles(FileStatus[] statuse * @param dataFile Data File */ private boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) { - Pair compactionWithInstantTime = fileIdToPendingCompaction.get(dataFile.getFileId()); + final String partitionPath = getPartitionPathFromFileStatus(dataFile.getFileStatus()); + HoodieFileGroupId fgId = new HoodieFileGroupId(partitionPath, dataFile.getFileId()); + Pair compactionWithInstantTime = fgIdToPendingCompaction.get(fgId); if ((null != compactionWithInstantTime) && (null != compactionWithInstantTime.getLeft()) && dataFile.getCommitTime().equals(compactionWithInstantTime.getKey())) { return true; @@ -210,7 +216,8 @@ private boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) { public Stream getLatestDataFiles(final String partitionPath) { return getAllFileGroups(partitionPath) .map(fileGroup -> { - return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst(); + return fileGroup.getAllDataFiles() + .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst(); }) .filter(Optional::isPresent) .map(Optional::get); @@ -278,7 +285,7 @@ public Stream getLatestFileSlices(String partitionPath) { .map(HoodieFileGroup::getLatestFileSlice) .filter(Optional::isPresent) .map(Optional::get) - .map(this::filterDataFileAfterPendingCompaction); + .map(fs -> filterDataFileAfterPendingCompaction(fs)); } @Override @@ -288,7 +295,7 @@ public Stream getLatestUnCompactedFileSlices(String partitionPath) { FileSlice fileSlice = fileGroup.getLatestFileSlice().get(); // if the file-group is under compaction, pick the latest before compaction instant time. if (isFileSliceAfterPendingCompaction(fileSlice)) { - String compactionInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()).getLeft(); + String compactionInstantTime = fgIdToPendingCompaction.get(fileSlice.getFileGroupId()).getLeft(); return fileGroup.getLatestFileSliceBefore(compactionInstantTime); } return Optional.of(fileSlice); @@ -303,7 +310,8 @@ public Stream getLatestUnCompactedFileSlices(String partitionPath) { * @return */ private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) { - Pair compactionWithInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()); + Pair compactionWithInstantTime = + fgIdToPendingCompaction.get(fileSlice.getFileGroupId()); return (null != compactionWithInstantTime) && fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey()); } @@ -318,7 +326,8 @@ private FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) { if (isFileSliceAfterPendingCompaction(fileSlice)) { // Data file is filtered out of the file-slice as the corresponding compaction // instant not completed yet. - FileSlice transformed = new FileSlice(fileSlice.getBaseInstantTime(), fileSlice.getFileId()); + FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(), + fileSlice.getBaseInstantTime(), fileSlice.getFileId()); fileSlice.getLogFiles().forEach(transformed::addLogFile); return transformed; } @@ -332,7 +341,7 @@ public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime)) .filter(Optional::isPresent) .map(Optional::get) - .map(this::filterDataFileAfterPendingCompaction); + .map(fs -> filterDataFileAfterPendingCompaction(fs)); } /** @@ -342,7 +351,8 @@ public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, * @param penultimateSlice Penultimate file slice for a file-group in commit timeline order */ private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) { - FileSlice merged = new FileSlice(penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId()); + FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(), + penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId()); if (penultimateSlice.getDataFile().isPresent()) { merged.setDataFile(penultimateSlice.getDataFile().get()); } @@ -361,8 +371,9 @@ private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, F */ private FileSlice getMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) { // if the file-group is under construction, pick the latest before compaction instant time. - if (fileIdToPendingCompaction.containsKey(fileSlice.getFileId())) { - String compactionInstantTime = fileIdToPendingCompaction.get(fileSlice.getFileId()).getKey(); + HoodieFileGroupId fgId = fileSlice.getFileGroupId(); + if (fgIdToPendingCompaction.containsKey(fgId)) { + String compactionInstantTime = fgIdToPendingCompaction.get(fgId).getKey(); if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) { Optional prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime); if (prevFileSlice.isPresent()) { @@ -426,8 +437,8 @@ public Stream getAllFileGroups(String partitionPathStr) { } } - public Map> getFileIdToPendingCompaction() { - return fileIdToPendingCompaction; + public Map> getFgIdToPendingCompaction() { + return fgIdToPendingCompaction; } public Stream getAllFileGroups() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java index dec7244db5c9e..f3552b4522e90 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -20,6 +20,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; @@ -134,36 +135,38 @@ public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaC } /** - * Get all file-ids with pending Compaction operations and their target compaction instant time + * Get all PartitionPath + file-ids with pending Compaction operations and their target compaction instant time * * @param metaClient Hoodie Table Meta Client */ - public static Map> getAllPendingCompactionOperations( + public static Map> getAllPendingCompactionOperations( HoodieTableMetaClient metaClient) { List> pendingCompactionPlanWithInstants = getAllPendingCompactionPlans(metaClient); - Map> fileIdToPendingCompactionWithInstantMap = new HashMap<>(); + Map> fgIdToPendingCompactionWithInstantMap = + new HashMap<>(); pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> { HoodieInstant instant = instantPlanPair.getKey(); HoodieCompactionPlan compactionPlan = instantPlanPair.getValue(); List ops = compactionPlan.getOperations(); if (null != ops) { return ops.stream().map(op -> { - return Pair.of(op.getFileId(), Pair.of(instant.getTimestamp(), op)); + return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()), + Pair.of(instant.getTimestamp(), op)); }); } else { return Stream.empty(); } }).forEach(pair -> { // Defensive check to ensure a single-fileId does not have more than one pending compaction - if (fileIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) { + if (fgIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) { String msg = "Hoodie File Id (" + pair.getKey() + ") has more thant 1 pending compactions. Instants: " - + pair.getValue() + ", " + fileIdToPendingCompactionWithInstantMap.get(pair.getKey()); + + pair.getValue() + ", " + fgIdToPendingCompactionWithInstantMap.get(pair.getKey()); throw new IllegalStateException(msg); } - fileIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue()); + fgIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue()); }); - return fileIdToPendingCompactionWithInstantMap; + return fgIdToPendingCompactionWithInstantMap; } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index b650695f8c66f..bfd9310920ceb 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -26,6 +26,7 @@ import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieFileGroup; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -51,6 +52,7 @@ import java.util.stream.Stream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -676,7 +678,7 @@ public void testStreamEveryVersionInPartition() throws IOException { assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { - String fileId = fileGroup.getId(); + String fileId = fileGroup.getFileGroupId().getFileId(); Set filenames = Sets.newHashSet(); fileGroup.getAllDataFiles().forEach(dataFile -> { assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId()); @@ -850,16 +852,17 @@ public void streamLatestVersions() throws IOException { assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { List slices = fileGroup.getAllFileSlices().collect(Collectors.toList()); - if (fileGroup.getId().equals(fileId1)) { + String fileId = fileGroup.getFileGroupId().getFileId(); + if (fileId.equals(fileId1)) { assertEquals(2, slices.size()); assertEquals(commitTime4, slices.get(0).getBaseInstantTime()); assertEquals(commitTime1, slices.get(1).getBaseInstantTime()); - } else if (fileGroup.getId().equals(fileId2)) { + } else if (fileId.equals(fileId2)) { assertEquals(3, slices.size()); assertEquals(commitTime3, slices.get(0).getBaseInstantTime()); assertEquals(commitTime2, slices.get(1).getBaseInstantTime()); assertEquals(commitTime1, slices.get(2).getBaseInstantTime()); - } else if (fileGroup.getId().equals(fileId3)) { + } else if (fileId.equals(fileId3)) { assertEquals(2, slices.size()); assertEquals(commitTime4, slices.get(0).getBaseInstantTime()); assertEquals(commitTime3, slices.get(1).getBaseInstantTime()); @@ -876,4 +879,159 @@ public void streamLatestVersions() throws IOException { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3))); } + + @Test + public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws Exception { + // Put some files in the partition + String partitionPath1 = "2016/05/01"; + String partitionPath2 = "2016/05/02"; + String partitionPath3 = "2016/05/03"; + + String fullPartitionPath1 = basePath + "/" + partitionPath1 + "/"; + new File(fullPartitionPath1).mkdirs(); + String fullPartitionPath2 = basePath + "/" + partitionPath2 + "/"; + new File(fullPartitionPath2).mkdirs(); + String fullPartitionPath3 = basePath + "/" + partitionPath3 + "/"; + new File(fullPartitionPath3).mkdirs(); + String instantTime1 = "1"; + String deltaInstantTime1 = "2"; + String deltaInstantTime2 = "3"; + String fileId = UUID.randomUUID().toString(); + + String dataFileName = FSUtils.makeDataFileName(instantTime1, 1, fileId); + new File(fullPartitionPath1 + dataFileName).createNewFile(); + + String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0); + new File(fullPartitionPath1 + fileName1) + .createNewFile(); + new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, 1, fileId)).createNewFile(); + new File(fullPartitionPath2 + fileName1) + .createNewFile(); + new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, 1, fileId)).createNewFile(); + new File(fullPartitionPath3 + fileName1) + .createNewFile(); + + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + + commitTimeline.saveAsComplete(instant1, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant2, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant3, Optional.empty()); + + // Now we list all partitions + FileStatus[] statuses = metaClient.getFs().listStatus(new Path[] { + new Path(fullPartitionPath1), new Path(fullPartitionPath2), new Path(fullPartitionPath3) + }); + assertEquals(6, statuses.length); + refreshFsView(statuses); + + List groups = fsView.getAllFileGroups().collect(Collectors.toList()); + Assert.assertEquals("Expected number of file-groups", 3, groups.size()); + Assert.assertEquals("Partitions must be different for file-groups", 3, + groups.stream().map(HoodieFileGroup::getPartitionPath).collect(Collectors.toSet()).size()); + Set fileIds = groups.stream().map(HoodieFileGroup::getFileGroupId) + .map(HoodieFileGroupId::getFileId).collect(Collectors.toSet()); + Assert.assertEquals("File Id must be same", 1, fileIds.size()); + Assert.assertTrue("Expected FileId", fileIds.contains(fileId)); + + // Setup Pending compaction for all of these fileIds. + List> partitionFileSlicesPairs = new ArrayList<>(); + List fileSlices = rtView.getLatestFileSlices(partitionPath1).collect(Collectors.toList()); + partitionFileSlicesPairs.add(Pair.of(partitionPath1, fileSlices.get(0))); + fileSlices = rtView.getLatestFileSlices(partitionPath2).collect(Collectors.toList()); + partitionFileSlicesPairs.add(Pair.of(partitionPath2, fileSlices.get(0))); + fileSlices = rtView.getLatestFileSlices(partitionPath3).collect(Collectors.toList()); + partitionFileSlicesPairs.add(Pair.of(partitionPath3, fileSlices.get(0))); + + String compactionRequestedTime = "2"; + String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId); + HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, + Optional.empty(), Optional.empty()); + + // Create a Data-file for some of the partitions but this should be skipped by view + new File(basePath + "/" + partitionPath1 + "/" + compactDataFileName).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + compactDataFileName).createNewFile(); + + HoodieInstant compactionInstant = + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); + HoodieInstant requested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + metaClient.getActiveTimeline().saveToCompactionRequested(requested, + AvroUtils.serializeCompactionPlan(compactionPlan)); + metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(requested); + + // Fake delta-ingestion after compaction-requested + String deltaInstantTime4 = "3"; + String deltaInstantTime5 = "6"; + List allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2, + compactionRequestedTime, deltaInstantTime4, deltaInstantTime5); + String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 0); + String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 1); + new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile(); + new File(basePath + "/" + partitionPath3 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath3 + "/" + fileName4).createNewFile(); + + HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4); + HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5); + commitTimeline.saveAsComplete(deltaInstant4, Optional.empty()); + commitTimeline.saveAsComplete(deltaInstant5, Optional.empty()); + refreshFsView(null); + + // Test Data Files + List dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList()); + assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size()); + assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime()); + dataFiles = roView.getAllDataFiles(partitionPath2).collect(Collectors.toList()); + assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size()); + assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime()); + + /** Merge API Tests **/ + Arrays.asList(partitionPath1, partitionPath2, partitionPath3).stream().forEach(partitionPath -> { + List fileSliceList = rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5) + .collect(Collectors.toList()); + assertEquals("Expect file-slice to be merged", 1, fileSliceList.size()); + FileSlice fileSlice = fileSliceList.get(0); + assertEquals(fileId, fileSlice.getFileId()); + assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName()); + assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1, + fileSlice.getBaseInstantTime()); + List logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Log files must include those after compaction request", 3, logFiles.size()); + assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); + assertEquals("Log File Order check", fileName1, logFiles.get(2).getFileName()); + + fileSliceList = rtView.getLatestFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5) + .collect(Collectors.toList()); + assertEquals("Expect only one file-id", 1, fileSliceList.size()); + fileSlice = fileSliceList.get(0); + assertEquals(fileId, fileSlice.getFileId()); + assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent()); + assertEquals("Compaction requested instant must be base instant", compactionRequestedTime, + fileSlice.getBaseInstantTime()); + logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); + assertEquals("Log files must include only those after compaction request", 2, logFiles.size()); + assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName()); + assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); + }); + + Assert.assertEquals(3, fsView.getFgIdToPendingCompaction().size()); + Set partitionsInCompaction = + fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getPartitionPath) + .collect(Collectors.toSet()); + Assert.assertEquals(3, partitionsInCompaction.size()); + Assert.assertTrue(partitionsInCompaction.contains(partitionPath1)); + Assert.assertTrue(partitionsInCompaction.contains(partitionPath2)); + Assert.assertTrue(partitionsInCompaction.contains(partitionPath3)); + + Set fileIdsInCompaction = + fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getFileId) + .collect(Collectors.toSet()); + Assert.assertEquals(1, fileIdsInCompaction.size()); + Assert.assertTrue(fileIdsInCompaction.contains(fileId)); + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java index 5fae7a3b3a7eb..e2718c47e3e12 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java @@ -26,6 +26,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.FileSlice; import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; @@ -48,7 +49,7 @@ public class CompactionTestUtils { - public static Map> setupAndValidateCompactionOperations( + public static Map> setupAndValidateCompactionOperations( HoodieTableMetaClient metaClient, boolean inflight, int numEntriesInPlan1, int numEntriesInPlan2, int numEntriesInPlan3, int numEntriesInPlan4) throws IOException { @@ -91,10 +92,10 @@ public static Map> setupAndValid }); metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath(), true); - Map> pendingCompactionMap = + Map> pendingCompactionMap = CompactionUtils.getAllPendingCompactionOperations(metaClient); - Map> expPendingCompactionMap = + Map> expPendingCompactionMap = generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4), baseInstantsToCompaction); // Ensure Compaction operations are fine. @@ -102,12 +103,13 @@ public static Map> setupAndValid return expPendingCompactionMap; } - public static Map> generateExpectedCompactionOperations( + public static Map> generateExpectedCompactionOperations( List plans, Map baseInstantsToCompaction) { return plans.stream() .flatMap(plan -> { if (plan.getOperations() != null) { - return plan.getOperations().stream().map(op -> Pair.of(op.getFileId(), + return plan.getOperations().stream().map(op -> Pair.of( + new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()), Pair.of(baseInstantsToCompaction.get(op.getBaseInstantTime()), op))); } return Stream.empty(); @@ -146,7 +148,7 @@ public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient me instantId, fileId, Optional.of(1)); HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId, Optional.of(2)); - FileSlice slice = new FileSlice(instantId, fileId); + FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId); if (createDataFile) { slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + "/" + FSUtils.makeDataFileName(instantId, 1, fileId))); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java index c65216260ec07..90c1832491295 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java @@ -26,6 +26,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; @@ -69,20 +70,20 @@ public void init() throws IOException { @Test public void testBuildFromFileSlice() { // Empty File-Slice with no data and log files - FileSlice emptyFileSlice = new FileSlice("000", "empty1"); + FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1"); HoodieCompactionOperation op = CompactionUtils.buildFromFileSlice( DEFAULT_PARTITION_PATHS[0], emptyFileSlice, Optional.of(metricsCaptureFn)); testFileSliceCompactionOpEquality(emptyFileSlice, op, DEFAULT_PARTITION_PATHS[0]); // File Slice with data-file but no log files - FileSlice noLogFileSlice = new FileSlice("000", "noLog1"); + FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1"); noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); op = CompactionUtils.buildFromFileSlice( DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Optional.of(metricsCaptureFn)); testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]); //File Slice with no data-file but log files present - FileSlice noDataFileSlice = new FileSlice("000", "noData1"); + FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1"); noDataFileSlice.addLogFile(new HoodieLogFile(new Path( FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); noDataFileSlice.addLogFile(new HoodieLogFile(new Path( @@ -92,7 +93,7 @@ public void testBuildFromFileSlice() { testFileSliceCompactionOpEquality(noDataFileSlice, op, DEFAULT_PARTITION_PATHS[0]); //File Slice with data-file and log files present - FileSlice fileSlice = new FileSlice("000", "noData1"); + FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1"); fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); fileSlice.addLogFile(new HoodieLogFile(new Path( FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); @@ -107,16 +108,16 @@ public void testBuildFromFileSlice() { * Generate input for compaction plan tests */ private Pair>, HoodieCompactionPlan> buildCompactionPlan() { - FileSlice emptyFileSlice = new FileSlice("000", "empty1"); - FileSlice fileSlice = new FileSlice("000", "noData1"); + FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1"); + FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1"); fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); fileSlice.addLogFile(new HoodieLogFile(new Path( FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); fileSlice.addLogFile(new HoodieLogFile(new Path( FSUtils.makeLogFileName("noData1", ".log", "000", 2)))); - FileSlice noLogFileSlice = new FileSlice("000", "noLog1"); + FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1"); noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet")); - FileSlice noDataFileSlice = new FileSlice("000", "noData1"); + FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1"); noDataFileSlice.addLogFile(new HoodieLogFile(new Path( FSUtils.makeLogFileName("noData1", ".log", "000", 1)))); noDataFileSlice.addLogFile(new HoodieLogFile(new Path( @@ -161,7 +162,7 @@ public void testGetAllPendingCompactionOperationsWithDupFileId() throws IOExcept // schedule same plan again so that there will be duplicates scheduleCompaction(metaClient, "005", plan1); metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); - Map> res = + Map> res = CompactionUtils.getAllPendingCompactionOperations(metaClient); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java index 0a225e91ae68b..da464c682d3c4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactionAdminTool.java @@ -5,6 +5,7 @@ import com.uber.hoodie.CompactionAdminClient; import com.uber.hoodie.CompactionAdminClient.RenameOpResult; import com.uber.hoodie.CompactionAdminClient.ValidationOpResult; +import com.uber.hoodie.common.model.HoodieFileGroupId; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; import java.io.ObjectOutputStream; @@ -60,7 +61,8 @@ public void run(JavaSparkContext jsc) throws Exception { break; case UNSCHEDULE_FILE: List r = - admin.unscheduleCompactionFileId(cfg.fileId, cfg.skipValidation, cfg.dryRun); + admin.unscheduleCompactionFileId(new HoodieFileGroupId(cfg.partitionPath, cfg.fileId), + cfg.skipValidation, cfg.dryRun); if (cfg.printOutput) { System.out.println(r); } @@ -132,6 +134,8 @@ public static class Config implements Serializable { public String basePath = null; @Parameter(names = {"--instant-time", "-in"}, description = "Compaction Instant time", required = false) public String compactionInstantTime = null; + @Parameter(names = {"--partition-path", "-pp"}, description = "Partition Path", required = false) + public String partitionPath = null; @Parameter(names = {"--file-id", "-id"}, description = "File Id", required = false) public String fileId = null; @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false) From 8854d11c6c7bc18c26df063977b83ed4a37863c4 Mon Sep 17 00:00:00 2001 From: kaka11chen Date: Wed, 6 Mar 2019 12:29:03 +0800 Subject: [PATCH 159/374] Fix avro doesn't have short and byte byte. --- .../main/scala/com/uber/hoodie/AvroConversionUtils.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index df3f96438afdf..337ff6a5b0aed 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -265,8 +265,12 @@ object AvroConversionUtils { case null => null case bytes: Array[Byte] => ByteBuffer.wrap(bytes) } - case ByteType | ShortType | IntegerType | LongType | + case IntegerType | LongType | FloatType | DoubleType | StringType | BooleanType => identity + case ByteType => (item: Any) => + if (item == null) null else item.asInstanceOf[Byte].intValue + case ShortType => (item: Any) => + if (item == null) null else item.asInstanceOf[Short].intValue case _: DecimalType => (item: Any) => if (item == null) null else item.toString case TimestampType => (item: Any) => if (item == null) null else item.asInstanceOf[Timestamp].getTime From ea1802a8536465e2a4a67d38172959539dc8f9b4 Mon Sep 17 00:00:00 2001 From: Omkar Joshi Date: Fri, 1 Mar 2019 15:34:46 -0800 Subject: [PATCH 160/374] Handling duplicate record update for single partition (duplicates in single or different parquet files) --- .../uber/hoodie/index/bloom/HoodieBloomIndex.java | 13 ++++++------- .../java/com/uber/hoodie/io/HoodieMergeHandle.java | 4 +++- .../com/uber/hoodie/common/model/HoodieRecord.java | 6 ++++++ 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index fcbd90f0b3d22..5c187870f866d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -391,15 +391,14 @@ private JavaRDD> tagLocationBacktoRecords( return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(v1 -> { HoodieRecord record = v1._1(); if (v1._2().isPresent()) { + // When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD + // will have 2 entries with the same exact in memory copy of the HoodieRecord and the 2 + // separate filenames that the record is found in. This will result in setting + // currentLocation 2 times and it will fail the second time. So creating a new in memory + // copy of the hoodie record. + record = new HoodieRecord<>(v1._1()); String filename = v1._2().get(); if (filename != null && !filename.isEmpty()) { - // When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD will have 2 - // entries with the same exact in memory copy of the HoodieRecord and the 2 separate filenames that the - // record is found in. This will result in setting currentLocation 2 times and it will fail the second time. - // This check will create a new in memory copy of the hoodie record. - if (record.getCurrentLocation() != null) { - record = new HoodieRecord(record.getKey(), record.getData()); - } record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename))); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 0700b4cbb71c8..1b12931ee22fb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -202,9 +202,11 @@ private boolean writeRecord(HoodieRecord hoodieRecord, Optional hoodieRecord = keyToNewRecords.get(key); boolean copyOldRecord = true; if (keyToNewRecords.containsKey(key)) { + // If we have duplicate records that we are updating, then the hoodie record will be deflated after + // writing the first record. So make a copy of the record to be merged + HoodieRecord hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key)); try { Optional combinedAvroRecord = hoodieRecord.getData() .combineAndGetUpdateValue(oldRecord, schema); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java index 857dcaaa95a38..52fa27988434f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -68,6 +68,12 @@ public HoodieRecord(HoodieKey key, T data) { this.newLocation = null; } + public HoodieRecord(HoodieRecord record) { + this(record.key, record.data); + this.currentLocation = record.currentLocation; + this.newLocation = record.newLocation; + } + public HoodieKey getKey() { return key; } From 693f30514207afddbaed9985d1670786efffa195 Mon Sep 17 00:00:00 2001 From: Bhavani Sudha Saktheeswaran Date: Fri, 8 Mar 2019 10:16:22 -0800 Subject: [PATCH 161/374] Fix quickstart documentation for querying via Presto --- docs/quickstart.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index bb7be6edda54b..70848d0992489 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -261,7 +261,7 @@ You can also use the sample queries in __hoodie-utilities/src/test/java/HoodieSp Checkout the 'master' branch on OSS Presto, build it, and place your installation somewhere. -* Copy the hoodie-hadoop-mr-* jar into $PRESTO_INSTALL/plugin/hive-hadoop2/ +* Copy the hudi/packaging/hoodie-presto-bundle/target/hoodie-presto-bundle-*.jar into $PRESTO_INSTALL/plugin/hive-hadoop2/ * Startup your server and you should be able to query the same Hive table via Presto ``` From 69be0874ff029ea3d4bb747f2d98caa786af755c Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Fri, 1 Mar 2019 11:17:53 -0800 Subject: [PATCH 162/374] Fix hive sync (libfb version mismatch) and deltastreamer issue (missing cmdline argument) in demo --- docs/quickstart.md | 9 +++++---- hoodie-hive/pom.xml | 2 +- packaging/hoodie-hive-bundle/pom.xml | 7 ++++++- packaging/hoodie-presto-bundle/pom.xml | 2 +- pom.xml | 1 + 5 files changed, 14 insertions(+), 7 deletions(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index 70848d0992489..882e660de89f3 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -450,13 +450,14 @@ automatically initializes the datasets in the file-system if they do not exist y docker exec -it adhoc-2 /bin/bash # Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_cow dataset in HDFS -spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type COPY_ON_WRITE --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_cow --target-table stock_ticks_cow --props /var/demo/config/kafka-source.properties +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type COPY_ON_WRITE --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_cow --target-table stock_ticks_cow --schemaprovider-class com.uber.hoodie.utilities.schema.FilebasedSchemaProvider --props /var/demo/config/kafka-source.properties .... .... 2018-09-24 22:20:00 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint:54 - OutputCommitCoordinator stopped! 2018-09-24 22:20:00 INFO SparkContext:54 - Successfully stopped SparkContext + # Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_mor dataset in HDFS -spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type MERGE_ON_READ --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_mor --target-table stock_ticks_mor --props /var/demo/config/kafka-source.properties +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type MERGE_ON_READ --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_mor --target-table stock_ticks_mor --schemaprovider-class com.uber.hoodie.utilities.schema.FilebasedSchemaProvider --props /var/demo/config/kafka-source.properties .... 2018-09-24 22:22:01 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint:54 - OutputCommitCoordinator stopped! 2018-09-24 22:22:01 INFO SparkContext:54 - Successfully stopped SparkContext @@ -724,10 +725,10 @@ cat docker/demo/data/batch_2.json | kafkacat -b kafkabroker -t stock_ticks -P docker exec -it adhoc-2 /bin/bash # Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_cow dataset in HDFS -spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type COPY_ON_WRITE --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_cow --target-table stock_ticks_cow --props /var/demo/config/kafka-source.properties +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type COPY_ON_WRITE --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_cow --target-table stock_ticks_cow --schemaprovider-class com.uber.hoodie.utilities.schema.FilebasedSchemaProvider --props /var/demo/config/kafka-source.properties # Run the following spark-submit command to execute the delta-streamer and ingest to stock_ticks_mor dataset in HDFS -spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type MERGE_ON_READ --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_mor --target-table stock_ticks_mor --props /var/demo/config/kafka-source.properties +spark-submit --class com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer $HUDI_UTILITIES_BUNDLE --storage-type MERGE_ON_READ --source-class com.uber.hoodie.utilities.sources.JsonKafkaSource --source-ordering-field ts --target-base-path /user/hive/warehouse/stock_ticks_mor --target-table stock_ticks_mor --schemaprovider-class com.uber.hoodie.utilities.schema.FilebasedSchemaProvider --props /var/demo/config/kafka-source.properties exit ``` diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml index ff7487aa8dfbf..a856a5e655466 100644 --- a/hoodie-hive/pom.xml +++ b/hoodie-hive/pom.xml @@ -50,7 +50,7 @@ org.apache.thrift libthrift - 0.12.0 + ${thrift.version} diff --git a/packaging/hoodie-hive-bundle/pom.xml b/packaging/hoodie-hive-bundle/pom.xml index 52021c30901f8..61462360f7cf9 100644 --- a/packaging/hoodie-hive-bundle/pom.xml +++ b/packaging/hoodie-hive-bundle/pom.xml @@ -71,7 +71,12 @@ org.apache.thrift libthrift - 0.9.2 + ${thrift.version} + + + org.apache.thrift + libfb303 + 0.9.3 diff --git a/packaging/hoodie-presto-bundle/pom.xml b/packaging/hoodie-presto-bundle/pom.xml index c4a2bdb994d9f..750fba9a8d54d 100644 --- a/packaging/hoodie-presto-bundle/pom.xml +++ b/packaging/hoodie-presto-bundle/pom.xml @@ -51,7 +51,7 @@ org.apache.thrift libthrift - 0.9.2 + ${thrift.version} diff --git a/pom.xml b/pom.xml index b9e9bb87693fb..d3799207d864d 100644 --- a/pom.xml +++ b/pom.xml @@ -138,6 +138,7 @@ 2.11.8 2.11 file://${project.basedir}/src/test/resources/log4j-surefire.properties + 0.12.0 From f5a92fb10e12e8ce4a5293504531344fd357c206 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Thu, 14 Mar 2019 17:23:36 -0700 Subject: [PATCH 163/374] Removing docs folder from master branch - Only asf-site branch contains the docs - Helps streamline doc contributions --- docs/.gitignore | 4 - docs/404.md | 6 - docs/CNAME | 1 - docs/Dockerfile | 26 - docs/Gemfile | 10 - docs/Gemfile.lock | 156 - docs/_config.yml | 87 - docs/_data/alerts.yml | 15 - docs/_data/sidebars/mydoc_sidebar.yml | 66 - docs/_data/strings.yml | 5 - docs/_data/tags.yml | 10 - docs/_data/topnav.yml | 30 - docs/_includes/archive.html | 15 - docs/_includes/callout.html | 1 - .../custom/getting_started_series.html | 19 - .../custom/getting_started_series_next.html | 10 - docs/_includes/custom/series_acme.html | 19 - docs/_includes/custom/series_acme_next.html | 30 - docs/_includes/custom/sidebarconfigs.html | 19 - docs/_includes/custom/usermap.html | 14 - docs/_includes/custom/usermapcomplex.html | 91 - docs/_includes/disqus.html | 16 - docs/_includes/feedback.html | 13 - docs/_includes/footer.html | 9 - docs/_includes/google_analytics.html | 6 - docs/_includes/head.html | 47 - docs/_includes/head_print.html | 33 - docs/_includes/image.html | 1 - docs/_includes/important.html | 1 - docs/_includes/initialize_shuffle.html | 130 - docs/_includes/inline_image.html | 1 - docs/_includes/links.html | 44 - docs/_includes/note.html | 1 - docs/_includes/sidebar.html | 56 - docs/_includes/taglogic.html | 32 - docs/_includes/tip.html | 1 - docs/_includes/toc.html | 21 - docs/_includes/topnav.html | 75 - docs/_includes/warning.html | 1 - docs/_layouts/default.html | 80 - docs/_layouts/default_print.html | 25 - docs/_layouts/none.html | 3 - docs/_layouts/page.html | 70 - docs/_layouts/page_print.html | 15 - docs/_layouts/post.html | 41 - docs/_posts/2016-12-30-strata-talk-2017.md | 11 - docs/admin_guide.md | 444 -- docs/api_docs.md | 10 - docs/code_and_design.md | 38 - docs/community.md | 22 - docs/comparison.md | 61 - docs/concepts.md | 159 - docs/configurations.md | 203 - docs/css/bootstrap.min.css | 5 - docs/css/customstyles.css | 1181 ---- docs/css/font-awesome.min.css | 4 - docs/css/fonts/FontAwesome.otf | Bin 93888 -> 0 bytes docs/css/fonts/fontawesome-webfont.eot | Bin 60767 -> 0 bytes docs/css/fonts/fontawesome-webfont.svg | 565 -- docs/css/fonts/fontawesome-webfont.ttf | Bin 122092 -> 0 bytes docs/css/fonts/fontawesome-webfont.woff | Bin 71508 -> 0 bytes docs/css/fonts/fontawesome-webfont.woff2 | Bin 56780 -> 0 bytes docs/css/lavish-bootstrap.css | 5898 ----------------- docs/css/modern-business.css | 93 - docs/css/printstyles.css | 160 - docs/css/syntax.css | 60 - docs/css/theme-blue.css | 103 - docs/css/theme-green.css | 99 - docs/dev_setup.md | 13 - docs/feed.xml | 32 - docs/fonts/FontAwesome.otf | Bin 85908 -> 0 bytes docs/fonts/fontawesome-webfont.eot | Bin 56006 -> 0 bytes docs/fonts/fontawesome-webfont.svg | 520 -- docs/fonts/fontawesome-webfont.ttf | Bin 112160 -> 0 bytes docs/fonts/fontawesome-webfont.woff | Bin 65452 -> 0 bytes docs/fonts/glyphicons-halflings-regular.eot | Bin 20127 -> 0 bytes docs/fonts/glyphicons-halflings-regular.svg | 288 - docs/fonts/glyphicons-halflings-regular.ttf | Bin 45404 -> 0 bytes docs/fonts/glyphicons-halflings-regular.woff | Bin 23424 -> 0 bytes docs/fonts/glyphicons-halflings-regular.woff2 | Bin 18028 -> 0 bytes docs/gcs_filesystem.md | 62 - docs/images/androidsdkmanagericon.png | Bin 795 -> 0 bytes docs/images/async_compac_1.png | Bin 60344 -> 0 bytes docs/images/async_compac_2.png | Bin 54164 -> 0 bytes docs/images/async_compac_3.png | Bin 70516 -> 0 bytes docs/images/async_compac_4.png | Bin 66932 -> 0 bytes docs/images/authorizegithubscreen2.png | Bin 76388 -> 0 bytes docs/images/authorizeongithub.png | Bin 22571 -> 0 bytes docs/images/company_logo.png | Bin 3105 -> 0 bytes docs/images/company_logo_big.png | Bin 9588 -> 0 bytes docs/images/favicon.ico | Bin 1150 -> 0 bytes docs/images/helpapi-01.png | Bin 91640 -> 0 bytes docs/images/helpapi.svg | 1661 ----- docs/images/hoodie_commit_duration.png | Bin 869144 -> 0 bytes docs/images/hoodie_cow.png | Bin 31136 -> 0 bytes docs/images/hoodie_intro_1.png | Bin 23478 -> 0 bytes docs/images/hoodie_log_format_v2.png | Bin 223676 -> 0 bytes docs/images/hoodie_mor.png | Bin 56002 -> 0 bytes docs/images/hoodie_query_perf_hive.png | Bin 158481 -> 0 bytes docs/images/hoodie_query_perf_presto.png | Bin 33727 -> 0 bytes docs/images/hoodie_query_perf_spark.png | Bin 29384 -> 0 bytes docs/images/hoodie_timeline.png | Bin 23093 -> 0 bytes docs/images/hoodie_upsert_dag.png | Bin 503771 -> 0 bytes docs/images/hoodie_upsert_perf1.png | Bin 15984 -> 0 bytes docs/images/hoodie_upsert_perf2.png | Bin 18954 -> 0 bytes docs/images/illustratoroptions.png | Bin 118175 -> 0 bytes docs/images/itermexample.png | Bin 68886 -> 0 bytes docs/images/jekyll.png | Bin 5375 -> 0 bytes docs/images/killalljekyll.png | Bin 66686 -> 0 bytes docs/images/liningup.png | Bin 74728 -> 0 bytes docs/images/workflowarrow.png | Bin 3595 -> 0 bytes docs/implementation.md | 275 - docs/incremental_processing.md | 237 - docs/index.md | 26 - docs/js/customscripts.js | 55 - docs/js/jekyll-search.js | 1 - docs/js/jquery.ba-throttle-debounce.min.js | 9 - docs/js/jquery.localScroll.min.js | 7 - docs/js/jquery.navgoco.min.js | 8 - docs/js/jquery.scrollTo.min.js | 7 - docs/js/jquery.shuffle.min.js | 1588 ----- docs/js/mydoc_scroll.html | 240 - docs/js/toc.js | 82 - docs/licenses/LICENSE | 21 - docs/licenses/LICENSE-BSD-NAVGOCO.txt | 27 - docs/migration_guide.md | 71 - docs/pages/news/news.html | 32 - docs/pages/news/news_archive.html | 41 - docs/powered_by.md | 40 - docs/quickstart.md | 1248 ---- docs/roadmap.md | 14 - docs/s3_filesystem.md | 79 - docs/search.json | 33 - docs/sitemap.xml | 24 - docs/sql_queries.md | 68 - docs/use_cases.md | 77 - 136 files changed, 17358 deletions(-) delete mode 100644 docs/.gitignore delete mode 100644 docs/404.md delete mode 100644 docs/CNAME delete mode 100644 docs/Dockerfile delete mode 100644 docs/Gemfile delete mode 100644 docs/Gemfile.lock delete mode 100644 docs/_config.yml delete mode 100644 docs/_data/alerts.yml delete mode 100644 docs/_data/sidebars/mydoc_sidebar.yml delete mode 100644 docs/_data/strings.yml delete mode 100644 docs/_data/tags.yml delete mode 100644 docs/_data/topnav.yml delete mode 100644 docs/_includes/archive.html delete mode 100644 docs/_includes/callout.html delete mode 100644 docs/_includes/custom/getting_started_series.html delete mode 100644 docs/_includes/custom/getting_started_series_next.html delete mode 100644 docs/_includes/custom/series_acme.html delete mode 100644 docs/_includes/custom/series_acme_next.html delete mode 100644 docs/_includes/custom/sidebarconfigs.html delete mode 100644 docs/_includes/custom/usermap.html delete mode 100644 docs/_includes/custom/usermapcomplex.html delete mode 100644 docs/_includes/disqus.html delete mode 100644 docs/_includes/feedback.html delete mode 100755 docs/_includes/footer.html delete mode 100644 docs/_includes/google_analytics.html delete mode 100644 docs/_includes/head.html delete mode 100644 docs/_includes/head_print.html delete mode 100644 docs/_includes/image.html delete mode 100644 docs/_includes/important.html delete mode 100644 docs/_includes/initialize_shuffle.html delete mode 100644 docs/_includes/inline_image.html delete mode 100644 docs/_includes/links.html delete mode 100644 docs/_includes/note.html delete mode 100644 docs/_includes/sidebar.html delete mode 100644 docs/_includes/taglogic.html delete mode 100644 docs/_includes/tip.html delete mode 100644 docs/_includes/toc.html delete mode 100644 docs/_includes/topnav.html delete mode 100644 docs/_includes/warning.html delete mode 100644 docs/_layouts/default.html delete mode 100644 docs/_layouts/default_print.html delete mode 100644 docs/_layouts/none.html delete mode 100644 docs/_layouts/page.html delete mode 100644 docs/_layouts/page_print.html delete mode 100644 docs/_layouts/post.html delete mode 100644 docs/_posts/2016-12-30-strata-talk-2017.md delete mode 100644 docs/admin_guide.md delete mode 100644 docs/api_docs.md delete mode 100644 docs/code_and_design.md delete mode 100644 docs/community.md delete mode 100644 docs/comparison.md delete mode 100644 docs/concepts.md delete mode 100644 docs/configurations.md delete mode 100755 docs/css/bootstrap.min.css delete mode 100644 docs/css/customstyles.css delete mode 100644 docs/css/font-awesome.min.css delete mode 100644 docs/css/fonts/FontAwesome.otf delete mode 100644 docs/css/fonts/fontawesome-webfont.eot delete mode 100644 docs/css/fonts/fontawesome-webfont.svg delete mode 100644 docs/css/fonts/fontawesome-webfont.ttf delete mode 100644 docs/css/fonts/fontawesome-webfont.woff delete mode 100644 docs/css/fonts/fontawesome-webfont.woff2 delete mode 100644 docs/css/lavish-bootstrap.css delete mode 100755 docs/css/modern-business.css delete mode 100644 docs/css/printstyles.css delete mode 100644 docs/css/syntax.css delete mode 100644 docs/css/theme-blue.css delete mode 100644 docs/css/theme-green.css delete mode 100644 docs/dev_setup.md delete mode 100644 docs/feed.xml delete mode 100644 docs/fonts/FontAwesome.otf delete mode 100644 docs/fonts/fontawesome-webfont.eot delete mode 100644 docs/fonts/fontawesome-webfont.svg delete mode 100644 docs/fonts/fontawesome-webfont.ttf delete mode 100644 docs/fonts/fontawesome-webfont.woff delete mode 100644 docs/fonts/glyphicons-halflings-regular.eot delete mode 100644 docs/fonts/glyphicons-halflings-regular.svg delete mode 100644 docs/fonts/glyphicons-halflings-regular.ttf delete mode 100644 docs/fonts/glyphicons-halflings-regular.woff delete mode 100644 docs/fonts/glyphicons-halflings-regular.woff2 delete mode 100644 docs/gcs_filesystem.md delete mode 100644 docs/images/androidsdkmanagericon.png delete mode 100644 docs/images/async_compac_1.png delete mode 100644 docs/images/async_compac_2.png delete mode 100644 docs/images/async_compac_3.png delete mode 100644 docs/images/async_compac_4.png delete mode 100644 docs/images/authorizegithubscreen2.png delete mode 100644 docs/images/authorizeongithub.png delete mode 100644 docs/images/company_logo.png delete mode 100644 docs/images/company_logo_big.png delete mode 100644 docs/images/favicon.ico delete mode 100644 docs/images/helpapi-01.png delete mode 100644 docs/images/helpapi.svg delete mode 100644 docs/images/hoodie_commit_duration.png delete mode 100644 docs/images/hoodie_cow.png delete mode 100644 docs/images/hoodie_intro_1.png delete mode 100644 docs/images/hoodie_log_format_v2.png delete mode 100644 docs/images/hoodie_mor.png delete mode 100644 docs/images/hoodie_query_perf_hive.png delete mode 100644 docs/images/hoodie_query_perf_presto.png delete mode 100644 docs/images/hoodie_query_perf_spark.png delete mode 100644 docs/images/hoodie_timeline.png delete mode 100644 docs/images/hoodie_upsert_dag.png delete mode 100644 docs/images/hoodie_upsert_perf1.png delete mode 100644 docs/images/hoodie_upsert_perf2.png delete mode 100644 docs/images/illustratoroptions.png delete mode 100644 docs/images/itermexample.png delete mode 100644 docs/images/jekyll.png delete mode 100644 docs/images/killalljekyll.png delete mode 100644 docs/images/liningup.png delete mode 100644 docs/images/workflowarrow.png delete mode 100644 docs/implementation.md delete mode 100644 docs/incremental_processing.md delete mode 100644 docs/index.md delete mode 100644 docs/js/customscripts.js delete mode 100644 docs/js/jekyll-search.js delete mode 100644 docs/js/jquery.ba-throttle-debounce.min.js delete mode 100644 docs/js/jquery.localScroll.min.js delete mode 100755 docs/js/jquery.navgoco.min.js delete mode 100644 docs/js/jquery.scrollTo.min.js delete mode 100644 docs/js/jquery.shuffle.min.js delete mode 100644 docs/js/mydoc_scroll.html delete mode 100644 docs/js/toc.js delete mode 100644 docs/licenses/LICENSE delete mode 100644 docs/licenses/LICENSE-BSD-NAVGOCO.txt delete mode 100644 docs/migration_guide.md delete mode 100644 docs/pages/news/news.html delete mode 100644 docs/pages/news/news_archive.html delete mode 100644 docs/powered_by.md delete mode 100644 docs/quickstart.md delete mode 100644 docs/roadmap.md delete mode 100644 docs/s3_filesystem.md delete mode 100644 docs/search.json delete mode 100644 docs/sitemap.xml delete mode 100644 docs/sql_queries.md delete mode 100644 docs/use_cases.md diff --git a/docs/.gitignore b/docs/.gitignore deleted file mode 100644 index f380cae1bd07f..0000000000000 --- a/docs/.gitignore +++ /dev/null @@ -1,4 +0,0 @@ -_site -.sass-cache -.jekyll-metadata -.ruby-version diff --git a/docs/404.md b/docs/404.md deleted file mode 100644 index a7b58c002144f..0000000000000 --- a/docs/404.md +++ /dev/null @@ -1,6 +0,0 @@ ---- -title: "Page Not Found" -search: exclude ---- - -Sorry, but the page you were trying to view does not exist. Try searching for it or looking at the URL to see if it looks correct. diff --git a/docs/CNAME b/docs/CNAME deleted file mode 100644 index c8dd5d09f7065..0000000000000 --- a/docs/CNAME +++ /dev/null @@ -1 +0,0 @@ -hudi.apache.org \ No newline at end of file diff --git a/docs/Dockerfile b/docs/Dockerfile deleted file mode 100644 index b1fa52c47361c..0000000000000 --- a/docs/Dockerfile +++ /dev/null @@ -1,26 +0,0 @@ -FROM ruby:2.1 -MAINTAINER mrafayaleem@gmail.com - -RUN apt-get clean \ - && mv /var/lib/apt/lists /var/lib/apt/lists.broke \ - && mkdir -p /var/lib/apt/lists/partial - -RUN apt-get update - -RUN apt-get install -y \ - node \ - python-pygments \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/ - -WORKDIR /tmp -ADD Gemfile /tmp/ -ADD Gemfile.lock /tmp/ -RUN bundle install - -VOLUME /src -EXPOSE 4000 - -WORKDIR /src -ENTRYPOINT ["jekyll"] - diff --git a/docs/Gemfile b/docs/Gemfile deleted file mode 100644 index b301edabe3404..0000000000000 --- a/docs/Gemfile +++ /dev/null @@ -1,10 +0,0 @@ -source "https://rubygems.org" - - -gem "jekyll", "3.3.1" - - -group :jekyll_plugins do - gem "jekyll-feed", "~> 0.6" - gem 'github-pages', '~> 106' -end diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock deleted file mode 100644 index 8359d57575c75..0000000000000 --- a/docs/Gemfile.lock +++ /dev/null @@ -1,156 +0,0 @@ -GEM - remote: https://rubygems.org/ - specs: - activesupport (4.2.7) - i18n (~> 0.7) - json (~> 1.7, >= 1.7.7) - minitest (~> 5.1) - thread_safe (~> 0.3, >= 0.3.4) - tzinfo (~> 1.1) - addressable (2.4.0) - coffee-script (2.4.1) - coffee-script-source - execjs - coffee-script-source (1.12.2) - colorator (1.1.0) - concurrent-ruby (1.1.4) - ethon (0.12.0) - ffi (>= 1.3.0) - execjs (2.7.0) - faraday (0.15.4) - multipart-post (>= 1.2, < 3) - ffi (1.10.0) - forwardable-extended (2.6.0) - gemoji (2.1.0) - github-pages (106) - activesupport (= 4.2.7) - github-pages-health-check (= 1.2.0) - jekyll (= 3.3.1) - jekyll-avatar (= 0.4.2) - jekyll-coffeescript (= 1.0.1) - jekyll-feed (= 0.8.0) - jekyll-gist (= 1.4.0) - jekyll-github-metadata (= 2.2.0) - jekyll-mentions (= 1.2.0) - jekyll-paginate (= 1.1.0) - jekyll-redirect-from (= 0.11.0) - jekyll-relative-links (= 0.2.1) - jekyll-sass-converter (= 1.3.0) - jekyll-seo-tag (= 2.1.0) - jekyll-sitemap (= 0.12.0) - jekyll-swiss (= 0.4.0) - jemoji (= 0.7.0) - kramdown (= 1.11.1) - liquid (= 3.0.6) - listen (= 3.0.6) - mercenary (~> 0.3) - minima (= 2.0.0) - rouge (= 1.11.1) - terminal-table (~> 1.4) - github-pages-health-check (1.2.0) - addressable (~> 2.3) - net-dns (~> 0.8) - octokit (~> 4.0) - public_suffix (~> 1.4) - typhoeus (~> 0.7) - html-pipeline (2.10.0) - activesupport (>= 2) - nokogiri (>= 1.4) - i18n (0.9.5) - concurrent-ruby (~> 1.0) - jekyll (3.3.1) - addressable (~> 2.4) - colorator (~> 1.0) - jekyll-sass-converter (~> 1.0) - jekyll-watch (~> 1.1) - kramdown (~> 1.3) - liquid (~> 3.0) - mercenary (~> 0.3.3) - pathutil (~> 0.9) - rouge (~> 1.7) - safe_yaml (~> 1.0) - jekyll-avatar (0.4.2) - jekyll (~> 3.0) - jekyll-coffeescript (1.0.1) - coffee-script (~> 2.2) - jekyll-feed (0.8.0) - jekyll (~> 3.3) - jekyll-gist (1.4.0) - octokit (~> 4.2) - jekyll-github-metadata (2.2.0) - jekyll (~> 3.1) - octokit (~> 4.0, != 4.4.0) - jekyll-mentions (1.2.0) - activesupport (~> 4.0) - html-pipeline (~> 2.3) - jekyll (~> 3.0) - jekyll-paginate (1.1.0) - jekyll-redirect-from (0.11.0) - jekyll (>= 2.0) - jekyll-relative-links (0.2.1) - jekyll (~> 3.3) - jekyll-sass-converter (1.3.0) - sass (~> 3.2) - jekyll-seo-tag (2.1.0) - jekyll (~> 3.3) - jekyll-sitemap (0.12.0) - jekyll (~> 3.3) - jekyll-swiss (0.4.0) - jekyll-watch (1.5.1) - listen (~> 3.0) - jemoji (0.7.0) - activesupport (~> 4.0) - gemoji (~> 2.0) - html-pipeline (~> 2.2) - jekyll (>= 3.0) - json (1.8.6) - kramdown (1.11.1) - liquid (3.0.6) - listen (3.0.6) - rb-fsevent (>= 0.9.3) - rb-inotify (>= 0.9.7) - mercenary (0.3.6) - mini_portile2 (2.4.0) - minima (2.0.0) - minitest (5.11.3) - multipart-post (2.0.0) - net-dns (0.9.0) - nokogiri (1.10.1) - mini_portile2 (~> 2.4.0) - octokit (4.13.0) - sawyer (~> 0.8.0, >= 0.5.3) - pathutil (0.16.2) - forwardable-extended (~> 2.6) - public_suffix (1.5.3) - rb-fsevent (0.10.3) - rb-inotify (0.10.0) - ffi (~> 1.0) - rouge (1.11.1) - safe_yaml (1.0.4) - sass (3.7.3) - sass-listen (~> 4.0.0) - sass-listen (4.0.0) - rb-fsevent (~> 0.9, >= 0.9.4) - rb-inotify (~> 0.9, >= 0.9.7) - sawyer (0.8.1) - addressable (>= 2.3.5, < 2.6) - faraday (~> 0.8, < 1.0) - terminal-table (1.8.0) - unicode-display_width (~> 1.1, >= 1.1.1) - thread_safe (0.3.6) - typhoeus (0.8.0) - ethon (>= 0.8.0) - tzinfo (1.2.5) - thread_safe (~> 0.1) - unicode-display_width (1.4.1) - -PLATFORMS - ruby - -DEPENDENCIES - github-pages (~> 106) - jekyll (= 3.3.1) - jekyll-feed (~> 0.6) - -BUNDLED WITH - 1.14.3 diff --git a/docs/_config.yml b/docs/_config.yml deleted file mode 100644 index baf6748dc0e50..0000000000000 --- a/docs/_config.yml +++ /dev/null @@ -1,87 +0,0 @@ - -output: web -# this property is useful for conditional filtering of content that is separate from the PDF. - -topnav_title: Hoodie -# this appears on the top navigation bar next to the home button - -site_title: Hoodie -# this appears in the html browser tab for the site title (seen mostly by search engines, not users) - -company_name: Uber Technologies Inc. -# this appears in the footer - -github_editme_path: -# if you're using Github, provide the basepath to the branch you've created for reviews, following the sample here. if not, leave this value blank. - -disqus_shortname: -# if you're using disqus for comments, add the shortname here. if not, leave this value blank. - -host: 127.0.0.1 -# the preview server used. Leave as is. - -port: 4000 -# the port where the preview is rendered. You can leave this as is unless you have other Jekyll builds using this same port that might cause conflicts. in that case, use another port such as 4006. - -exclude: - - .idea/ - - .gitignore -# these are the files and directories that jekyll will exclude from the build - -feedback_subject_line: Hoodie Documentation - -feedback_email: hoodie-dev-group@uber.com -# used as a contact email for the Feedback link in the top navigation bar - -# feedback_disable: true -# if you uncomment the previous line, the Feedback link gets removed - -# feedback_text: "Need help?" -# if you uncomment the previous line, it changes the Feedback text - -# feedback_link: "http://helpy.io/" -# if you uncomment the previous line, it changes where the feedback link points to - -highlighter: rouge -# library used for syntax highlighting - -markdown: kramdown -kramdown: - input: GFM - auto_ids: true - hard_wrap: false - syntax_highlighter: rouge - -# filter used to process markdown. note that kramdown differs from github-flavored markdown in some subtle ways - -defaults: - - - scope: - path: "" - type: "pages" - values: - layout: "page" - comments: true - search: true - sidebar: mydoc_sidebar - - - - scope: - path: "" - type: "posts" - values: - layout: "post" - comments: true - search: true - sidebar: mydoc_sidebar - -# these are defaults used for the frontmatter for these file types - -sidebars: -- mydoc_sidebar - -description: "Hoodie is a Spark Library, that provides upserts and incremental processing capaibilities on Hadoop datasets" -# the description is used in the feed.xml file - -# needed for sitemap.xml file only -url: http://github.com/uber/hoodie diff --git a/docs/_data/alerts.yml b/docs/_data/alerts.yml deleted file mode 100644 index 157e1622b008b..0000000000000 --- a/docs/_data/alerts.yml +++ /dev/null @@ -1,15 +0,0 @@ -tip: '