Importing Hoodie Client from internal repo
Abberved History: * 25c6991 Removed non-opensource modules * a62abf3 Removing email from pom.xml * 0931b68 Misspelt in the copyright * c1cac7d Preperation for OSS: Added License and rat plugin check. Also added meta information about the project in pom.xml * 16b07b3 Preparation of OSS - Remove hoodie specific URL from hoodie cli * fd3e0dd Small code cleanups * 8aa7e34 Adding a de-duplication command to CLI * b464842 Adding a de-duplication command to CLI * 59265b1 RegisterDataset should pass the right zkNodeName after the support for multiple databases added * b295f70 [maven-release-plugin] prepare for next development iteration * 1006e4b [maven-release-plugin] prepare release hoodie-0.2.4 * 4c99437 Move to using hdrone release 0.7.4 * 1891939 Auto tuning the buckets needed for plain inserts also - Off by default for now - Enhanced an existing unit test * b4563bd Change HoodieReadClient to use commit metadata for incremental pull * ee20183 Add full file path onto HoodieWriteStat - This will become an issue later on for incremental processing use cases - Tested with cli, that is able to read older HoodieCommitMetadata * 7dcd5d5 Address skew in cleaner work distribution * 8d7c15d Fixing bug around partial failures of rollback * d4ada1d Empty RDD should not throw java.lang.IllegalArgumentException: Positive number of slices required * 076bea9 Dont clean if there are no partitions to clean * c014f80 Minor changes to SQLStreamer * a96d4df Minor changes to SQLStreamer * bc289cc [maven-release-plugin] prepare for next development iteration * 4160107 [maven-release-plugin] prepare release hoodie-0.2.3 * 409b07a [maven-release-plugin] prepare for next development iteration * 3d71514 [maven-release-plugin] prepare release hoodie-0.2.2 * 4969d52 Fix test failures * ac62609 Implement Review Comments for: Parallelize cleaning and including cleaning time and commit archival time in commit time graphite reporting * cebe65a Parallelize cleaning and including cleaning time and commit archival time in commit time graphite reporting * 2e5b372 Migrating to CDH 5.7.2 * 899ae12 Remove filtering of /tmp/hive/hive paths from HoodieInputFormat. This fixes Join with temporary tables with HoodieCombineHiveInputFormat * 69a68f6 Implement equals and hashCode for HoodieTableMetadata, its used in hash based structures * 12d29c6 Update hive staging url * 1c5c88a Copy filterExists to WriteClient * 76aee67 [maven-release-plugin] prepare for next development iteration * 1f0a715 [maven-release-plugin] prepare release hoodie-0.2.1 * dbfd1d4 HoodieReadClient and HoodieWriteClient separation * c39a98b Revamped HoodieRecordPayload API that supports merging of old & new values during update * 79e5bbd Add a helper to configure SparkConf for SparkSQL on Hoodie tables * f56f423 [maven-release-plugin] prepare for next development iteration * 780fc44 [maven-release-plugin] prepare release hoodie-0.2 * 1ea2238 Modifying the git utl * b0af8dc Depending on hdrone release version * 7753693 Removing a System.out.println which got in by mistake * 1f5b019 Adding HBase Config to HoodieClientConfig * 2fce97f Implement Review comments and merge into master * f389820 Bunch of API changes * 909a856 HoodieClientConfig split up and revamp * c2ad946 Fix TestHoodieClient to not double persist in testFilterExists * 3ab0da6 Fix breaking test * 2860542 CR feedback for small inserts turned to updates * 0dfce57 Small inserts are now turned into upserts * bb1a8b3 Add filterExist API for Hoodie Records * d983c24 Implement review comments * c0bd5d1 Implement HoodieClient.checkExists() * db078f6 Pick up HoodieTable based on hoodie.properties * ad023e9 Refactor upsert() using HoodieTable interface * ee9b9b3 Refactor upsert() using HoodieTable interface * 2d6fdc9 Adding a utility to generate the percentage of updates in commit * ea3ad58 Adding additional optimizations to remove similar queries from the perf test (using levenshtein distance) * 1e443a0 Add test case for the added support for SchemaEvolution during updates * 1cadcbb Add more logging * 6163dfe Parquet read of old file should have the right read schema specified * 29c746a Few fixes in ReduceByKey parallelism, HoodieInputFormat.filterFiles for non-hoodie paths and more logging in upsert schema issues * 5a33af6 Fixing an issue in HoodieReader, target temp directory not created * 09a5e8e Adding more logging in HoodieReader * 1474250 Adding more logging in HoodieReader * a3b0567 Make targetDb not required in HoodieReader * e9c08b9 Setting the inputformat as the CombineHiveInputFormat in the HoodieReader * 61c75d2 Hoodie Query Performance: Add Support for CombineHiveInputFormat and implement CombineFileInputFormat * 38c6e44 Improvements to Hoodie Reader * ac7398a Add totalWriteErrors to HoodieCommitMetadata * fc0536e Change archive location to be under .hoodie * e313294 Implement Hive Perf comparison for Hoodie and non-Hoodie datasets * 17cfe2a Fix bug in HoodieInputFormat, where it filters out files from archived commits * 30de990 Add note about showpartitions command to README * 8634ffb Add commits showpartitions command to show break down per partition * 324b24e Adding a CLI command to print file size stats * 56532ff T484792. Deterministically report metrics during shutdown * 3571768 Fixes to Hoodie Cleaner. Upgrade HDrone version. Changes to HoodieReader. * a02c97f Bumping hdrone-api to 0.7.2 * b29ce67 Bug in RegisterDataset dataset creation * 5a15a9a Fixing bug in cleaning up partial files * dbf6669 Comment out predicate pushdown test * 44ed4d1 Merge branch 'lazyitr-fixes-1' |\ | * e913d3b Fixing bug in LazyInsertIterable | * 8a1fecd Wrapping upsert() inside HoodieUpsertException | * 39cfe39 Fixing bug in LazyInsertIterable - Return a List<WriteStatus> to handle last record in itr, belonging to a separate file - Remove insert() related code form UpsertMapFunction | * 00252e5 Making TestHoodieBloomIndex less flaky * | 6f2d417 Making TestHoodieBloomIndex less flaky * | 63ebbdc fs.mkdirs does not honor permission umask passed. Need to use the static method FileSystem.mkdirs for that. * | f49ef67 Adding more logging to Hoodie Reader * | 9f5a699 Fixing permission on the base intermediate folder created in HoodieReader |/ * 70e501f Fixing the drop table before create table in HoodieReader * 120cda8 Hoodie tools jar should not require jars in the CDH classpath to be available. Needed for HoodieReader to run in Docker. * 60b59de Adding client configurations. Needed to run the HoodieReader in Docker (where CDH is not installed) * fece98d Merge conflicts w/ master * 64e58b0 Auto tuning parallelism in BloomIndex & Upsert() * 930199e Fixing skew in Index join when new partition paths dont exist yet * 9a3e511 Adding subpartitioning to scale join in HoodieBloomIndex * 57512a7 Changing sort key for IndexLookup to (filename, record) to split more evenly * 3ede14c Major changes to BloomIndex & Upsert DAG * 1c4071a Implement Dataset creation if a Hoodie dataset was not already registered * 944f007 Implement Review comments * 6a5b675 Implement Review Comments * bfde3a9 Implement review comments * d195ab3 Implementing Commit Archiving * 8af656b Exception refactor - part 2 * 697a699 HoodieTableMetadata refactor and Exception refactor * 7804ca3 Adding HoodieAppendLog (fork of SequenceFile) & Initial Impl of HoodieCommitArchiveLog * 2db4931 Adjust partitionFileRDD parallelism to max(recordRDD partitions, total partitions) * 23405c5 Config name changes * 5e673ea Implementing more CLI commands * 918cfce Moving to 0.1.1-SNAPSHOT * afad497 Change the master branch to 0.2-SNAPSHOT * 832c1a7 Make sure the bloom filter reading and tagging has a parellel factor >= group by parallelism * 0a6a6d3 Prepare the v0.1 version * 72cfbe2 The snapshoter should also copy hoodie.properties file * 3b0ee45 Add one more metric * 488f1c7 Add switch for cleaning out inflight commits * a259b6f Adding textutils jar to hoodie build * 36e3118 Fix Hoodie CLI - ClassNotFound and added more logging to JDBC Incremental pull * 2c8f554 Fix Predicate pushdown during incremental pull * 888ec20 Add one more graphite metrics * a671dfc Ensure files picked for cleaning are part of some valid commit * ba5cd65 Adding cleaning based on last X commits * 7dc76d3 Organize config values by category * 9da6474 Move cleaning logic into HoodieCleaner class * 7becba9 Change the update metric name * d32b1f3 Fix some graphite issues * 365ee14 hot fix a stupid bug I made * 93eab43 Adding a hoodie.table.type value to hoodie.properties on init * 075c646 Add the database name to the sync * 3bae059 Adding HoodieKey as metadata field into Record * 61513fa Add stats and more cli commands * b0cb112 New Hoodie CLI Framework. Implement CLI function parity with the current CLI * aaa1bf8 New Hoodie CLI Framework. Implement CLI function parity with the current CLI * 3a3db73 New Hoodie CLI Framework. Implement CLI function parity with the current CLI * c413342 Fail the job if exception during writing old records * 7304d3d Exclude javax.servlet from hive-jdbc * 3d65b50 Add the datestr <> '0000-00-00' back to the incremental sql * 0577661 HoodieIncrementalConfig not used anymore * 5338004 Fixing multiple minor issues we found during the SQLStreamer demo preperation * 0744283 Fix the Hive server and Spark Hive client mismatch by setting userClassPathFirst=true and creating a assembly jar with all hadoop related dependencies excluded * c189dc0 Kickoff hdrone sync after SQLStreamer finishing committing to target hoodie dataset * 1eb8da0 Check if the .commit file is empty * f95386a Add support for rollbacking .inflight commit in Admin CLI * 97595ea Update the record count when upserting * 49139cd Remove table config and add _SUCCESS tag * 8500a48 Catch the exception when upserting * 10bcc19 Merge branch 'sqlload' |\ | * 10fcc88 More log statements | * ca6b71d Merge with master | |\ | | * b33db25 Merge remote-tracking branch 'origin/sqlload' into sqlload | | |\ | | | * 8fca7c6 insert() takes a JavaRDD<HoodieRecord> again | | * | 63db8c6 Fix test breakage from javax.servlet pom dependency | | * | b2cff33 insert() takes a JavaRDD<HoodieRecord> again | | * | 0162930 Minor Fixes | | * | a0eb0b8 Minor Fixes | | * | 5853e7c Minor fixed to HoodieSQLStreamer | | * | 379bbed HoodieSQLStreamer improvements | | * | 22bf816 Remove setJsonPayload() and other non-generic calls from HoodieRecordPayload | | * | 4cacde6 Remove setJsonPayload() and other non-generic calls from HoodieRecordPayload | | * | 5f985f3 Refactor of AvroParquetIO and create proper abstraction for StorageWriter | | * | 6b90bb0 Refactor to introduce proper abstractions for RawTripPayload and implement HoodieSQLStreamer | | * | ff24ce8 Implementation of HoodieSQLStreamer | | * | abae08a Implementation of HoodieSQLStreamer | * | | c2d306d Fixes to HoodieSQLStreamer | | |/ | |/| | * | 70bad72 Minor Fixes | * | 8da6abf Minor Fixes | * | 6b9d16b Minor fixed to HoodieSQLStreamer | * | f76f5b8 HoodieSQLStreamer improvements | * | 5f1425e Remove setJsonPayload() and other non-generic calls from HoodieRecordPayload | * | 616e2ee Remove setJsonPayload() and other non-generic calls from HoodieRecordPayload | * | 9e77ef9 Refactor of AvroParquetIO and create proper abstraction for StorageWriter | * | 14e4812 Refactor to introduce proper abstractions for RawTripPayload and implement HoodieSQLStreamer | * | 3b05f04 Implementation of HoodieSQLStreamer | * | 1484c34 Implementation of HoodieSQLStreamer * | | b3b9754 Standardize UTF-8 for getBytes() calls | |/ |/| * | 8cde079 Add graphite metrics to HoodieClient * | b94afad Add testcase for the snapshot copy |/ * 8567225 T417977. WriteStatus for failed records * 11d7cd2 Add code to deflate the HoodieRecord after writing it to storage * 9edafb4 Add a daily snapshot job * 2962bf6 Fix the last file non-closed issue * d995b6b SizeAwareParquetWriter will now have a fixed compression ratio * 6b5f67f HoodieWrapperFileSystem should initialize the underlying filesystem with default uri * 2a607c2 Merging conflicts with master * ac9852d Auto size parquet files to just under block size based on incoming records size * 3c4c0d0 Remove client code leaks & add parallelism config for sorting * 1e51e30 Add UpsertHandle * 685ca1f Add hoodie cli * ded7f6c CR feedback incorporated * d532089 Change the return type to a RDD * 22533c1 Fix bug in cleanup logic by using TaskContext.getPartitionId() in place of unitNumber * 86532fb Implement insert() using sorting, to align file sizes easily * 0967e1c Add hook to compare old record with new incoming record * f48b048 Merge branch 'sort-based-dag' |\ | * 3614cec Rename write() -> upsert() and load() -> insert() * | 65cf631 Parquet version mismatch in HoodieInputFormat * | 160303b Formatting change * | 2c079c8 Formatting change |/ * e4eb658 Fix formatting * 025114a Add test for HoodieAvroWriteSupport * 6fd11ef Fix small bug in HoodieCommits & correct doc to reflect exclusivity of findCommitsInRange - Added simple unit test * 05659c9 Add tests around HoodieClient apis * 8d3f73e Fix some small bugs * 7f1c4bc Modify HoodieInputFormatTest to make it certain that incremental pull is only pulling the required records * 2b73ba0 Remove direct versioning in pom * dd5695f Comment change * f62eef7 Unit test for predicate pushdown * 9941dad Fixing an issue which results in unsorted commits * 5e71506 Update README * 219e103 InputFormat unit tests * 8f1c7ba Enable cobertura coverage to be run with mvn test * 01f76e3 Call out self-join limitation in README * 4284a73 Defaulting to Google Java Style and reformatting existing code * de2cbda Making sure that incremental does not send duplicate records * f6a3833 Implement Review comments * 1de5025 Refactor in HoodieTableMetadata, HoodieInputFormat * 549ad9a Fixing broken test schemas * fbb2190 update the unit number * 9353ba9 Change the io number to 1 for old load data * e28f0cf Add commit metadata fields to create_table.sql * d06e93d Pull avroFn & dedupeFn into a single HoodieClientHooks class * b6d387f Changes to sequence_no/commit metadata addition * 212d237 Add some benchmark results to the code * 70d7715 Add commit rollback logic * 54a4d0f Use FSUtils helper to detemine fileId * 4b672ad Core classes refactoring * f705fab Move partitionPath back into HoodieKey * 39b3ff3 Cleanup Sample job & add a detailed quickstart * 981c6f7 fix the hoodie-query-meta pom * 371ab34 Publish hoodie to uber internal artifactory * b4e83bc improvement on the bloom index tag job * 779b502 Change to use hadoop's bloom filter * cfbd9e6 Add bloom filter indexing mechanism * f519c47 Initial Implementation of storing the client metadata for hoodie queries * d5eccea Initial Implementation of storing the client metadata for hoodie queries * ef34482 Pass on the HDrone configuration profile as an argument * 5578cd3 Implement initial incremental tailing support in InputFormat and provide a seperate module for Hdrone registration to be created as a oozie trigger * b08e5ff Merge branch 'master' into AddBloomFilterWriteSupport * 20b7e8e fix a typo * 4c39407 Quick fix for the HBASE indx duplicates records issue * 6dca38f Adding code to sync to hive using hdrone * 55a1d44 Fixes to InputFormat. Created a placeholder OutputFormat. * beda7ed Revise the globPartitions to avoid the bad partition paths * 5d889c0 Fix a wrong config * a60fbdf First version to add load function * 4b90944 Adding detailed metadata to each commit * 4a97a6c Changes to backfill script + enabling spark event log * ada2b79 Discard records without partition path & move parquet writer to snappy * 954c933 Adding backfill script - Cleanups & additional cmd line options to job - Changed iounit logic to special case 2010-2014 again * 8b5e288 Breaking apart backfill job & single run into two classes * ebdcbea Handle partial failures in update() * 4bf6ffe Fixing an issue where file name is not present * e468bff Fix couple of issues with Hbase indexing and commit ts checks * 17da30c Changing de-dupe implementation to be a Spark reduceByKey * 248c725 removed coalescing which was put in there for testing * 1b3f929 Implement compression when storing large json strings in memory * 5bada98 Changes to accomodate task failure handling, on top of cleaner * 66f895a Clean out files generated by previous failed attempts * 9cbe370 Implementing a rudimentary cleaner & avro conversion rewrite * 3606658 Adding configs for iounits & reduce parallelism * 066c2f5 Registering the Hoodie classes with Kryo * 342eed1 Implementing a rudimentary cleaner * 0d20d1d Merge branch 'trip-test-run' |\ | * 6eafdbb Adding de-dupe step before writing/shuffling * | 34baba7 Packaging hadoop-common with the hadoop-mr InputFormat JAR |/ * d5856db Merge HoodieInputFormat with existing code. Factor out common logic into hadoop-common. Tune the partitions, spark executors, parquet parameters to be able to run on a single day of input data * e8885ce Introduce IOUnit to split parallelize inserts * ab1977a Pushing in a real Spark job that works off real data * 0c86645 HoodirInputFormat with TestDataSimulator * 6af483c Initial checkin for HoodieInputFormat * 99c58f2 Implementing HBase backed index * 4177529 First major chunk of Hoodie Spark Client Impl * 29fad70 Benchmark bloom filter file read performance * 18f52a4 Checking in the simulation code, measuring cost of trip's file-level updates * 885f444 Adding basic datastructures for Client, key & record. * 72e7b4d Initial commit
This commit is contained in:
@@ -0,0 +1,634 @@
|
||||
/*
|
||||
* 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 com.google.common.collect.Iterables;
|
||||
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.HoodieCleaner;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.SparkConf;
|
||||
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.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
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.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestHoodieClient implements Serializable {
|
||||
private transient JavaSparkContext jsc = null;
|
||||
private transient SQLContext sqlContext;
|
||||
private String basePath = null;
|
||||
private transient HoodieTestDataGenerator
|
||||
dataGen = null;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
SparkConf sparkConf = new SparkConf().setAppName("TestHoodieClient").setMaster("local[4]");
|
||||
jsc = new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
|
||||
private HoodieWriteConfig getConfig() {
|
||||
return getConfigBuilder().build();
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
private void assertNoWriteErrors(List<WriteStatus> statuses) {
|
||||
// Verify there are no errors
|
||||
for (WriteStatus status : statuses) {
|
||||
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
|
||||
}
|
||||
}
|
||||
|
||||
private void checkTaggedRecords(List<HoodieRecord> 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
|
||||
public void testFilterExist() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, config.getBasePath());
|
||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||
|
||||
// Should not find any files
|
||||
assertTrue(filteredRDD.collect().size() == 100);
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
List<WriteStatus> statuses = writeClient.insert(smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
readClient = new HoodieReadClient(jsc, config.getBasePath());
|
||||
filteredRDD = readClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
assertTrue(result.size() == 25);
|
||||
}
|
||||
|
||||
@Test
|
||||
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)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting a single commit.", readClient.listCommitsSince("000").size(), 1);
|
||||
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
|
||||
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), records.size());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, "001");
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
records = dataGen.generateUpdates(newCommitTime, 100);
|
||||
LinkedHashMap<HoodieKey, HoodieRecord> recordsMap = new LinkedHashMap<>();
|
||||
for (HoodieRecord rec : records) {
|
||||
if (!recordsMap.containsKey(rec.getKey())) {
|
||||
recordsMap.put(rec.getKey(), rec);
|
||||
}
|
||||
}
|
||||
List<HoodieRecord> 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
|
||||
readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting two commits.", readClient.listCommitsSince("000").size(), 2);
|
||||
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
|
||||
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetadata(fs, basePath)).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", readClient.read(fullPartitionPaths).count(), 200);
|
||||
|
||||
|
||||
// Check that the incremental consumption from time 000
|
||||
assertEquals("Incremental consumption from time 002, should give all records in commit 004",
|
||||
readClient.readCommit(newCommitTime).count(),
|
||||
readClient.readSince("002").count());
|
||||
assertEquals("Incremental consumption from time 001, should give all records in commit 004",
|
||||
readClient.readCommit(newCommitTime).count(),
|
||||
readClient.readSince("001").count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertAndCleanByVersions() throws Exception {
|
||||
int maxVersions = 2; // keep upto 2 versions for each file
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.retainFileVersions(maxVersions).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* 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<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 500);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 5);
|
||||
|
||||
List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).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);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
SortedMap<String, HoodieCommitMetadata> commitMetadata = metadata.getAllCommitMetadata();
|
||||
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
// compute all the versions of all files, from time 0
|
||||
HashMap<String, TreeSet<String>> fileIdToVersions = new HashMap<>();
|
||||
for (Map.Entry<String, HoodieCommitMetadata> entry : commitMetadata.entrySet()) {
|
||||
for (HoodieWriteStat wstat : entry.getValue().getWriteStats(partitionPath)) {
|
||||
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<String>());
|
||||
}
|
||||
fileIdToVersions.get(wstat.getFileId()).add(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileVersions.entrySet()) {
|
||||
List<FileStatus> versions = entry.getValue();
|
||||
// No file has no more than max versions
|
||||
assertTrue("fileId " + entry.getKey() + " has more than " + maxVersions + " versions",
|
||||
versions.size() <= maxVersions);
|
||||
|
||||
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(entry.getKey()));
|
||||
for (int i = 0; i < versions.size(); i++) {
|
||||
assertEquals("File " + entry.getKey() + " does not have latest versions" + versions + " on commits" + commitedVersions,
|
||||
FSUtils.getCommitTime(Iterables.get(versions, i).getPath().getName()),
|
||||
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(HoodieCleaner.CleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.retainCommits(maxCommits).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* 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<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 500);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 5);
|
||||
|
||||
List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).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);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
String earliestRetainedCommit = commits.lastCommit(maxCommits - 1);
|
||||
Set<String> acceptableCommits = new HashSet<>(commits.getCommitList());
|
||||
if (earliestRetainedCommit != null) {
|
||||
acceptableCommits.removeAll(commits.findCommitsInRange("000", earliestRetainedCommit));
|
||||
acceptableCommits.add(earliestRetainedCommit);
|
||||
}
|
||||
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileVersions.entrySet()) {
|
||||
Set<String> commitTimes = new HashSet<>(entry.getValue().size());
|
||||
for(FileStatus value:entry.getValue()) {
|
||||
commitTimes.add(FSUtils.getCommitTime(value.getPath().getName()));
|
||||
}
|
||||
assertEquals("Only contain acceptable versions of file should be present",
|
||||
acceptableCommits, 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();
|
||||
|
||||
// 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();
|
||||
|
||||
// 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));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSmallInsertHandling() throws Exception {
|
||||
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder();
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
|
||||
final String TEST_PARTITION_PATH = "2016/09/26";
|
||||
final int INSERT_SPLIT_LIMIT = 10;
|
||||
// based on examination of sample file, the schema produces the following per record size
|
||||
final int SIZE_PER_RECORD = 50 * 1024;
|
||||
// setup the small file handling params
|
||||
HoodieWriteConfig config = builder.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(SIZE_PER_RECORD * 15)
|
||||
.insertSplitSize(INSERT_SPLIT_LIMIT).build()) // tolerate upto 15 records
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(SIZE_PER_RECORD * 20)
|
||||
.build()).build(); // hold upto 20 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {TEST_PARTITION_PATH});
|
||||
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
|
||||
|
||||
// Inserts => will write file1
|
||||
String commitTime1 = "001";
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses= client.upsert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals("file should contain 10 records",
|
||||
ParquetUtils.readRowKeysFromParquet(new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
|
||||
10);
|
||||
|
||||
// Update + Inserts such that they just expand file1
|
||||
String commitTime2 = "002";
|
||||
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 4);
|
||||
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
|
||||
List<HoodieRecord> insertsAndUpdates2 = new ArrayList<>();
|
||||
insertsAndUpdates2.addAll(inserts2);
|
||||
insertsAndUpdates2.addAll(dataGen.generateUpdates(commitTime2, inserts1));
|
||||
|
||||
JavaRDD<HoodieRecord> insertAndUpdatesRDD2 = jsc.parallelize(insertsAndUpdates2, 1);
|
||||
statuses = client.upsert(insertAndUpdatesRDD2, commitTime2).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
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("file should contain 14 records", ParquetUtils.readRowKeysFromParquet(newFile).size(), 14);
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(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));
|
||||
}
|
||||
|
||||
// update + inserts such that file1 is updated and expanded, a new file2 is created.
|
||||
String commitTime3 = "003";
|
||||
List<HoodieRecord> insertsAndUpdates3 = dataGen.generateInserts(commitTime3, 20);
|
||||
Set<String> keys3 = HoodieClientTestUtils.getRecordKeys(insertsAndUpdates3);
|
||||
List<HoodieRecord> updates3 = dataGen.generateUpdates(commitTime3, inserts2);
|
||||
insertsAndUpdates3.addAll(updates3);
|
||||
|
||||
JavaRDD<HoodieRecord> insertAndUpdatesRDD3 = jsc.parallelize(insertsAndUpdates3, 1);
|
||||
statuses = client.upsert(insertAndUpdatesRDD3, commitTime3).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
FileStatus[] files = metadata.getLatestVersionInPartition(fs, TEST_PARTITION_PATH, commitTime3);
|
||||
int numTotalInsertsInCommit3 = 0;
|
||||
for (FileStatus file: files) {
|
||||
if (file.getPath().getName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(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();
|
||||
if (recordCommitTime.equals(commitTime3)) {
|
||||
if (keys2.contains(recordKey)) {
|
||||
assertEquals("only expect commit3", commitTime3, recordCommitTime);
|
||||
keys2.remove(recordKey);
|
||||
} else {
|
||||
numTotalInsertsInCommit3++;
|
||||
}
|
||||
}
|
||||
}
|
||||
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, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(file.getPath());
|
||||
for (GenericRecord record: records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit3", commitTime3, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
assertTrue("key expected to be part of commit3", keys3.contains(recordKey));
|
||||
}
|
||||
numTotalInsertsInCommit3 += records.size();
|
||||
}
|
||||
}
|
||||
assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3);
|
||||
}
|
||||
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
*/
|
||||
public class HoodieClientTestUtils {
|
||||
|
||||
|
||||
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
while (statusListItr.hasNext()) {
|
||||
statuses.addAll(statusListItr.next());
|
||||
}
|
||||
return statuses;
|
||||
}
|
||||
|
||||
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
|
||||
Set<String> keys = new HashSet<>();
|
||||
for (HoodieRecord rec: hoodieRecords) {
|
||||
keys.add(rec.getRecordKey());
|
||||
}
|
||||
return keys;
|
||||
}
|
||||
|
||||
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
|
||||
String parentPath = basePath + "/"+ HoodieTableMetadata.METAFOLDER_NAME;
|
||||
new File(parentPath).mkdirs();
|
||||
new File(parentPath + "/" + commitTime + suffix).createNewFile();
|
||||
}
|
||||
|
||||
|
||||
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTableMetadata.COMMIT_FILE_SUFFIX);
|
||||
}
|
||||
|
||||
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTableMetadata.INFLIGHT_FILE_SUFFIX);
|
||||
}
|
||||
|
||||
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 {
|
||||
String parentPath = String.format("%s/%s", basePath, partitionPath);
|
||||
new File(parentPath).mkdirs();
|
||||
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
|
||||
new File(path).createNewFile();
|
||||
new RandomAccessFile(path, "rw").setLength(length);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,169 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* Class to be used in tests to keep generating test inserts & updates against a corpus.
|
||||
*
|
||||
* Test data uses a toy Uber trips, data model.
|
||||
*/
|
||||
public class HoodieTestDataGenerator {
|
||||
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieTestDataGenerator.class);
|
||||
|
||||
static class KeyPartition {
|
||||
HoodieKey key;
|
||||
String partitionPath;
|
||||
}
|
||||
|
||||
public static String TRIP_EXAMPLE_SCHEMA = "{\"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\"}]}";
|
||||
|
||||
|
||||
private List<KeyPartition> existingKeysList = new ArrayList<>();
|
||||
private static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
|
||||
private static Random rand = new Random(46474747);
|
||||
private String[] partitionPaths = {"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"});
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Generates new inserts, uniformly across the partition paths above. It also updates the list
|
||||
* of existing keys.
|
||||
*/
|
||||
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
|
||||
List<HoodieRecord> inserts = new ArrayList<>();
|
||||
for (int i = 0; i < n; i++) {
|
||||
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
|
||||
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
|
||||
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
|
||||
inserts.add(record);
|
||||
|
||||
KeyPartition kp = new KeyPartition();
|
||||
kp.key = key;
|
||||
kp.partitionPath = partitionPath;
|
||||
existingKeysList.add(kp);
|
||||
|
||||
logger.info(" GENERATING INSERT FOR :" + key + "," + record.getPartitionPath());
|
||||
}
|
||||
return inserts;
|
||||
}
|
||||
|
||||
|
||||
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
|
||||
List<HoodieRecord> updates = new ArrayList<>();
|
||||
for (HoodieRecord baseRecord: baseRecords) {
|
||||
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime));
|
||||
updates.add(record);
|
||||
logger.info(" GENERATING UPDATE FOR :" + baseRecord.getKey());
|
||||
}
|
||||
return updates;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates new updates, randomly distributed across the keys above.
|
||||
*/
|
||||
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
|
||||
List<HoodieRecord> updates = new ArrayList<>();
|
||||
for (int i = 0; i < n; i++) {
|
||||
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
|
||||
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
|
||||
updates.add(record);
|
||||
logger.info(" GENERATING UPDATE FOR :" + kp.key);
|
||||
}
|
||||
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 = new GenericData.Record(avroSchema);
|
||||
rec.put("_row_key", key.getRecordKey());
|
||||
rec.put("rider", "rider-" + commitTime);
|
||||
rec.put("driver", "driver-" + commitTime);
|
||||
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);
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
|
||||
public static void createCommitFile(String basePath, String commitTime) throws IOException {
|
||||
Path commitFile =
|
||||
new Path(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + FSUtils.makeCommitFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import com.uber.hoodie.avro.MercifulJsonConverter;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.Map;
|
||||
import java.util.zip.Deflater;
|
||||
import java.util.zip.DeflaterOutputStream;
|
||||
import java.util.zip.InflaterInputStream;
|
||||
|
||||
/**
|
||||
* Example row change event based on some example data used by testcases. The data avro schema is
|
||||
* src/test/resources/schema1.
|
||||
*/
|
||||
public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayload> {
|
||||
private transient static final ObjectMapper mapper = new ObjectMapper();
|
||||
private String partitionPath;
|
||||
private String rowKey;
|
||||
private byte[] jsonDataCompressed;
|
||||
private int dataSize;
|
||||
|
||||
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath, String schemaStr) throws IOException {
|
||||
this.jsonDataCompressed = compressData(jsonData);
|
||||
this.dataSize = jsonData.length();
|
||||
this.rowKey = rowKey;
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public TestRawTripPayload(String jsonData) throws IOException {
|
||||
this.jsonDataCompressed = compressData(jsonData);
|
||||
this.dataSize = jsonData.length();
|
||||
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
|
||||
this.rowKey = jsonRecordMap.get("_row_key").toString();
|
||||
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
|
||||
@Override public TestRawTripPayload preCombine(TestRawTripPayload another) {
|
||||
return another;
|
||||
}
|
||||
|
||||
@Override public IndexedRecord combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
|
||||
return this.getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override public IndexedRecord getInsertValue(Schema schema) throws IOException {
|
||||
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
|
||||
return jsonConverter.convert(getJsonData());
|
||||
}
|
||||
|
||||
public String getRowKey() {
|
||||
return rowKey;
|
||||
}
|
||||
|
||||
public String getJsonData() throws IOException {
|
||||
return unCompressData(jsonDataCompressed);
|
||||
}
|
||||
|
||||
|
||||
private byte[] compressData(String jsonData) throws IOException {
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DeflaterOutputStream dos =
|
||||
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
|
||||
try {
|
||||
dos.write(jsonData.getBytes());
|
||||
} finally {
|
||||
dos.flush();
|
||||
dos.close();
|
||||
}
|
||||
return baos.toByteArray();
|
||||
}
|
||||
|
||||
|
||||
private String unCompressData(byte[] data) throws IOException {
|
||||
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
|
||||
StringWriter sw = new StringWriter(dataSize);
|
||||
IOUtils.copy(iis, sw);
|
||||
return sw.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
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.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestUpdateMapFunction {
|
||||
private String basePath = null;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
||||
// Create a bunch of records with a old version of schema
|
||||
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("100", config, metadata);
|
||||
|
||||
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<HoodieRecord> records = new ArrayList<>();
|
||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||
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));
|
||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||
records.add(
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
|
||||
rowChange3));
|
||||
Iterator<List<WriteStatus>> insertResult = table.handleInsert(records.iterator());
|
||||
Path commitFile =
|
||||
new Path(config.getBasePath() + "/.hoodie/" + FSUtils.makeCommitFileName("100"));
|
||||
FSUtils.getFs().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 HoodieTableMetadata(FSUtils.getFs(), basePath);
|
||||
String fileId = insertResult.next().get(0).getFileId();
|
||||
System.out.println(fileId);
|
||||
|
||||
|
||||
table = new HoodieCopyOnWriteTable("101", config, metadata);
|
||||
// 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}";
|
||||
records = new ArrayList<>();
|
||||
rowChange1 = new TestRawTripPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
|
||||
rowChange1);
|
||||
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
|
||||
records.add(record1);
|
||||
|
||||
try {
|
||||
table.handleUpdate(fileId, records.iterator());
|
||||
} catch (ClassCastException e) {
|
||||
fail(
|
||||
"UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt");
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,439 @@
|
||||
/*
|
||||
* 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.google.common.base.Optional;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
|
||||
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.SparkConf;
|
||||
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;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.*;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class TestHoodieBloomIndex {
|
||||
private JavaSparkContext jsc = null;
|
||||
private String basePath = null;
|
||||
private transient final FileSystem fs;
|
||||
|
||||
public TestHoodieBloomIndex() throws Exception {
|
||||
fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
SparkConf sparkConf = new SparkConf().setAppName("TestHoodieBloomIndex").setMaster("local[4]");
|
||||
jsc = new JavaSparkContext(sparkConf);
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
}
|
||||
|
||||
@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 schemaStr =
|
||||
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||
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);
|
||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||
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<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
|
||||
Map<String, Iterable<String>> map = index.getPartitionToRowKeys(recordRDD);
|
||||
assertEquals(map.size(), 2);
|
||||
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
|
||||
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
|
||||
assertEquals(list1.size(), 3);
|
||||
assertEquals(list2.size(), 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLoadInvolvedFiles() throws IOException {
|
||||
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)
|
||||
new File(basePath + "/2016/01/21").mkdirs();
|
||||
new File(basePath + "/2016/04/01").mkdirs();
|
||||
new File(basePath + "/2015/03/12").mkdirs();
|
||||
new File(basePath + "/2016/04/01/2_0_20160401010101.parquet").createNewFile();
|
||||
new File(basePath + "/2015/03/12/1_0_20150312101010.parquet").createNewFile();
|
||||
new File(basePath + "/2015/03/12/3_0_20150312101010.parquet").createNewFile();
|
||||
new File(basePath + "/2015/03/12/4_0_20150312101010.parquet").createNewFile();
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
JavaPairRDD<String, String> rdd = index.loadInvolvedFiles(partitions, metadata);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(rdd.count(), 0);
|
||||
|
||||
// Add some commits
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
rdd = index.loadInvolvedFiles(partitions, metadata);
|
||||
final List<Tuple2<String, String>> filesList = rdd.collect();
|
||||
assertEquals(filesList.size(), 4);
|
||||
|
||||
// no longer sorted, but should have same files.
|
||||
Set<String> actualFiles = new HashSet<String>(){{
|
||||
add(filesList.get(0)._1 + "/" + filesList.get(0)._2);
|
||||
add(filesList.get(1)._1 + "/" + filesList.get(1)._2);
|
||||
add(filesList.get(2)._1 + "/" + filesList.get(2)._2);
|
||||
add(filesList.get(3)._1 + "/" + filesList.get(3)._2);
|
||||
}};
|
||||
|
||||
Set<String> expected = new HashSet<String>() {{
|
||||
add("2016/04/01/2_0_20160401010101.parquet");
|
||||
add("2015/03/12/1_0_20150312101010.parquet");
|
||||
add("2015/03/12/3_0_20150312101010.parquet");
|
||||
add("2015/03/12/4_0_20150312101010.parquet");
|
||||
}};
|
||||
assertEquals(expected, actualFiles);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException, ClassNotFoundException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
|
||||
|
||||
String schemaStr =
|
||||
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
// 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}";
|
||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||
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);
|
||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||
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);
|
||||
|
||||
|
||||
// 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);
|
||||
|
||||
// The bloom filter contains 3 records
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record3.getRecordKey()));
|
||||
assertFalse(filter.mightContain(record4.getRecordKey()));
|
||||
|
||||
// Compare with file
|
||||
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
|
||||
record3.getRecordKey(), record4.getRecordKey());
|
||||
|
||||
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(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"));
|
||||
// TODO(vc): Need more coverage on actual filenames
|
||||
//assertTrue(results.get(0)._2().equals(filename));
|
||||
//assertTrue(results.get(1)._2().equals(filename));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocationWithEmptyRDD() throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||
|
||||
try {
|
||||
bloomIndex.tagLocation(recordRDD, metadata);
|
||||
} catch (IllegalArgumentException e) {
|
||||
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testTagLocation() throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String schemaStr =
|
||||
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
|
||||
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);
|
||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||
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);
|
||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
assertTrue(!record.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
// 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);
|
||||
|
||||
// We do the tag again
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
|
||||
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
|
||||
} else if (record.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(!record.isCurrentLocationKnown());
|
||||
} else if (record.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckExists() throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String schemaStr =
|
||||
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
|
||||
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);
|
||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
|
||||
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
|
||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
|
||||
HoodieRecord record3 = new HoodieRecord(key3, rowChange3);
|
||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
|
||||
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
|
||||
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, metadata);
|
||||
|
||||
// Should not find any files
|
||||
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
}
|
||||
|
||||
// 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);
|
||||
|
||||
// We do the tag again
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, metadata);
|
||||
|
||||
// Check results
|
||||
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
||||
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
Path path1 = new Path(record._2.get());
|
||||
assertEquals(FSUtils.getFileId(filename1), FSUtils.getFileId(path1.getName()));
|
||||
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
Path path2 = new Path(record._2.get());
|
||||
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
|
||||
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
} else if (record._1.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
Path path3 = new Path(record._2.get());
|
||||
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path3.getName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@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}";
|
||||
|
||||
// We write record1 to a parquet file, using a bloom filter having both records
|
||||
String schemaStr =
|
||||
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
|
||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||
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);
|
||||
|
||||
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
||||
filter.add(record2.getRecordKey());
|
||||
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, filter, true);
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
|
||||
// We do the tag
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename)));
|
||||
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private String writeParquetFile(String partitionPath, List<HoodieRecord> 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<HoodieRecord> 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);
|
||||
ParquetWriter writer = new ParquetWriter(new Path(basePath + "/" + partitionPath + "/" + filename),
|
||||
writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
|
||||
int seqId = 1;
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema);
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
|
||||
writer.write(avroRecord);
|
||||
filter.add(record.getRecordKey());
|
||||
}
|
||||
writer.close();
|
||||
|
||||
if (createCommitTime) {
|
||||
// Also make sure the commit is valid
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME).mkdirs();
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
|
||||
}
|
||||
return filename;
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,44 @@
|
||||
/*
|
||||
* 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.config.HoodieWriteConfig;
|
||||
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class TestHoodieIndex {
|
||||
@Test
|
||||
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();
|
||||
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex);
|
||||
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();
|
||||
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex);
|
||||
}
|
||||
}
|
||||
@@ -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.io;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import java.io.IOException;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
/**
|
||||
* Tests around Cleaning logic in Hoodie
|
||||
*/
|
||||
public class TestHoodieCleaner {
|
||||
|
||||
private String basePath = null;
|
||||
private String[] partitionPaths = {"2016/01/01", "2016/02/02"};
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
this.basePath = HoodieTestUtils.initializeTempHoodieBasePath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepLatestFileVersions() throws IOException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.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");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
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");
|
||||
|
||||
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
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[1]));
|
||||
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");
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
|
||||
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must clean two files" , 2, cleaner.clean(partitionPaths[0]));
|
||||
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
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testKeepLatestCommits() throws IOException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.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");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
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");
|
||||
|
||||
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
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
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");
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
|
||||
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals(
|
||||
"Must not clean any file. We have to keep 1 version before the latest commit time to keep",
|
||||
0, cleaner.clean(partitionPaths[0]));
|
||||
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "003");
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update
|
||||
String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals(
|
||||
"Must not clean one old file", 1, cleaner.clean(partitionPaths[0]));
|
||||
|
||||
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
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,164 @@
|
||||
/*
|
||||
* 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 com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestHoodieCommitArchiveLog {
|
||||
private String basePath;
|
||||
private FileSystem fs;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
@Test
|
||||
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);
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveDatasetWithNoArchival() 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();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertEquals("Loaded 4 commits and the count should match", 4,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveDatasetWithArchival() 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();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
SortedMap<String, HoodieCommitMetadata> originalCommits = new TreeMap<>(metadata.getAllCommitMetadata());
|
||||
|
||||
assertEquals("Loaded 6 commits and the count should match", 6,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertEquals(
|
||||
"Should archive commits when maxCommitsToKeep is 5 and now the commits length should be minCommitsToKeep which is 2",
|
||||
2, metadata.getAllCommits().getCommitList().size());
|
||||
assertEquals("Archive should not archive the last 2 commits",
|
||||
Lists.newArrayList("104", "105"), metadata.getAllCommits().getCommitList());
|
||||
|
||||
// Remove all the commits from the original commits, make it ready to be checked against the read map
|
||||
for(String key:metadata.getAllCommitMetadata().keySet()) {
|
||||
originalCommits.remove(key);
|
||||
}
|
||||
|
||||
// Read back the commits to make sure
|
||||
SequenceFile.Reader reader = new SequenceFile.Reader(fs.getConf(),
|
||||
SequenceFile.Reader.file(archiveLog.getArchiveFilePath()));
|
||||
Text key = new Text();
|
||||
Text val = new Text();
|
||||
SortedMap<String, HoodieCommitMetadata> readCommits = new TreeMap<>();
|
||||
while (reader.next(key, val)) {
|
||||
HoodieCommitMetadata meta = HoodieCommitMetadata.fromJsonString(val.toString());
|
||||
readCommits.put(key.toString(), meta);
|
||||
}
|
||||
|
||||
assertEquals(
|
||||
"Read commits map should match the originalCommits - commitsLoadedAfterArchival",
|
||||
originalCommits, readCommits);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveCommitSafety() 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();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertEquals("Loaded 6 commits and the count should match", 6,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("100"));
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("101"));
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("102"));
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("103"));
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.commons.configuration.ConfigurationException;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestHoodieMetrics {
|
||||
private HoodieMetrics metrics = null;
|
||||
|
||||
@Before
|
||||
public void start() throws ConfigurationException {
|
||||
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
|
||||
when(config.isMetricsOn()).thenReturn(true);
|
||||
when(config.getMetricsReporterType()).thenReturn(MetricsReporterType.INMEMORY);
|
||||
metrics = new HoodieMetrics(config, "raw_table");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRegisterGauge() {
|
||||
metrics.registerGauge("metric1", 123L);
|
||||
assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,431 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
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.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.io.HoodieInsertHandle;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
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.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.SparkConf;
|
||||
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 java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestCopyOnWriteTable {
|
||||
private String basePath = null;
|
||||
private transient JavaSparkContext jsc = null;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
|
||||
// Initialize a local spark env
|
||||
SparkConf sparkConf = new SparkConf().setAppName("TestCopyOnWriteTable").setMaster("local[4]");
|
||||
jsc = new JavaSparkContext(sparkConf);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeNewPath() throws Exception {
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String partitionPath = "2016/05/04";
|
||||
int unitNumber = (int) (Math.random() * 10);
|
||||
HoodieRecord record = mock(HoodieRecord.class);
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime, null, partitionPath);
|
||||
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
|
||||
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, unitNumber, fileName)));
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
|
||||
return makeHoodieClientConfigBuilder().build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
|
||||
}
|
||||
|
||||
// TODO (weiy): Add testcases for crossing file writing.
|
||||
@Test
|
||||
public void testUpdateRecords() throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String firstCommitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
String partitionPath = "/2016/01/31";
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(firstCommitTime, 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}";
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||
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));
|
||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
|
||||
// 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")) {
|
||||
parquetFile = file;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(parquetFile != null);
|
||||
|
||||
// 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);
|
||||
for (HoodieRecord record : records) {
|
||||
assertTrue(filter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
// Create a commit file
|
||||
new File(this.basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/"
|
||||
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
|
||||
|
||||
// Read the parquet file, check the record content
|
||||
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(parquetFilePath);
|
||||
GenericRecord newRecord;
|
||||
int index = 0;
|
||||
for (GenericRecord record: fileRecords) {
|
||||
assertTrue(record.get("_row_key").toString().equals(records.get(index).getRecordKey()));
|
||||
index++;
|
||||
}
|
||||
|
||||
// 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}";
|
||||
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())));
|
||||
|
||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||
HoodieRecord insertedRecord1 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
|
||||
|
||||
Thread.sleep(1000);
|
||||
String newCommitTime = HoodieTestUtils.getNewCommitTime();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
table = new HoodieCopyOnWriteTable(newCommitTime, config, metadata);
|
||||
Iterator<List<WriteStatus>> iter = table.handleUpdate(updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
|
||||
|
||||
// Check the updated file
|
||||
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()))
|
||||
&& HoodieCommits
|
||||
.isCommit1After(FSUtils.getCommitTime(file.getName()), FSUtils.getCommitTime(parquetFile.getName()))) {
|
||||
updatedParquetFile = file;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
assertTrue(updatedParquetFile != null);
|
||||
// Check whether the record has been updated
|
||||
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
|
||||
BloomFilter updatedFilter = ParquetUtils.readBloomFilterFromParquetMetadata(updatedParquetFilePath);
|
||||
for (HoodieRecord record : records) {
|
||||
// No change to the _row_key
|
||||
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
|
||||
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
|
||||
records.add(insertedRecord1);// add this so it can further check below
|
||||
|
||||
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()));
|
||||
if (index == 0) {
|
||||
assertTrue(newRecord.get("number").toString().equals("15"));
|
||||
}
|
||||
index++;
|
||||
}
|
||||
updatedReader.close();
|
||||
// Also check the numRecordsWritten
|
||||
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(iter);
|
||||
WriteStatus writeStatus = statuses.get(0);
|
||||
assertTrue("Should be only one file generated", statuses.size() == 1);
|
||||
assertEquals(4, writeStatus.getStat().getNumWrites());//3 rewritten records + 1 new record
|
||||
}
|
||||
|
||||
|
||||
private List<HoodieRecord> newHoodieRecords(int n, String time) throws Exception {
|
||||
List<HoodieRecord> 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);
|
||||
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
|
||||
records.add(new HoodieRecord(
|
||||
new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
|
||||
rowChange));
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
@Test public void testInsertWithPartialFailures() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
|
||||
// Write a few records, and get atleast one file
|
||||
// 10 records for partition 1, 1 record for partition 2.
|
||||
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
|
||||
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
|
||||
|
||||
// Simulate crash after first file
|
||||
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(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()))
|
||||
);
|
||||
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(records.iterator()));
|
||||
status = statuses.get(0);
|
||||
|
||||
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));
|
||||
}
|
||||
|
||||
|
||||
@Test public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
|
||||
// Case 1:
|
||||
// 10 records for partition 1, 1 record for partition 2.
|
||||
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
|
||||
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
|
||||
|
||||
// Insert new records
|
||||
List<WriteStatus> returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
|
||||
|
||||
|
||||
// TODO: check the actual files and make sure 11 records, total were written.
|
||||
assertEquals(2, returnedStatuses.size());
|
||||
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
|
||||
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
|
||||
assertEquals(10, returnedStatuses.get(0).getWrittenRecords().size());
|
||||
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
|
||||
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
|
||||
assertEquals(1, returnedStatuses.get(1).getWrittenRecords().size());
|
||||
|
||||
// Case 2:
|
||||
// 1 record for partition 1, 5 record for partition 2, 1 records for partition 3.
|
||||
records = newHoodieRecords(1, "2016-01-31T03:16:41.415Z");
|
||||
records.addAll(newHoodieRecords(5, "2016-02-01T03:16:41.415Z"));
|
||||
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
|
||||
|
||||
// Insert new records
|
||||
returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
|
||||
|
||||
assertEquals(3, returnedStatuses.size());
|
||||
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
|
||||
assertEquals(1, returnedStatuses.get(0).getWrittenRecords().size());
|
||||
|
||||
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
|
||||
assertEquals(5, returnedStatuses.get(1).getWrittenRecords().size());
|
||||
|
||||
assertEquals("2016/02/02", returnedStatuses.get(2).getPartitionPath());
|
||||
assertEquals(1, returnedStatuses.get(2).getWrittenRecords().size());
|
||||
|
||||
}
|
||||
|
||||
@Test public void testFileSizeUpsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
|
||||
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
|
||||
.parquetPageSize(64 * 1024).build()).build();
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
|
||||
List<HoodieRecord> 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 + "}";
|
||||
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
|
||||
rowChange));
|
||||
}
|
||||
|
||||
// Insert new records
|
||||
HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
|
||||
|
||||
// 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)) {
|
||||
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);
|
||||
}
|
||||
|
||||
|
||||
|
||||
private List<HoodieCopyOnWriteTable.InsertBucket> 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();
|
||||
|
||||
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("001", config, metadata);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH});
|
||||
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
|
||||
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
|
||||
for (HoodieRecord updateRec: updateRecords) {
|
||||
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
|
||||
}
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
records.addAll(insertRecords);
|
||||
records.addAll(updateRecords);
|
||||
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(TEST_PARTITION_PATH);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitioner() throws Exception {
|
||||
// Inserts + Updates... Check all updates go together & inserts subsplit
|
||||
List<HoodieCopyOnWriteTable.InsertBucket> 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<HoodieCopyOnWriteTable.InsertBucket> 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);
|
||||
|
||||
// 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);
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user