1
0
Files
hudi/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java
Vinoth Chandar 81874a8406 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
2016-12-16 14:34:42 -08:00

557 lines
25 KiB
Java

/*
* 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.codahale.metrics.Timer;
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.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.InsertMapFunction;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCleaner;
import com.uber.hoodie.io.HoodieCommitArchiveLog;
import com.uber.hoodie.metrics.HoodieMetrics;
import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.WorkloadProfile;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Accumulator;
import org.apache.spark.Partitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.api.java.function.VoidFunction;
import org.apache.spark.storage.StorageLevel;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Collections;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import scala.Option;
import scala.Tuple2;
/**
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then
* perform efficient mutations on a HDFS dataset [upsert()]
*
* Note that, at any given time, there can only be one Spark job performing
* these operatons on a Hoodie dataset.
*
*/
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
private static Logger logger = LogManager.getLogger(HoodieWriteClient.class);
private transient final FileSystem fs;
private transient final JavaSparkContext jsc;
private final HoodieWriteConfig config;
private transient final HoodieMetrics metrics;
private transient final HoodieIndex<T> index;
private transient final HoodieCommitArchiveLog archiveLog;
private transient Timer.Context writeContext = null;
private final SimpleDateFormat FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
/**
* @param jsc
* @param clientConfig
* @throws Exception
*/
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception {
this(jsc, clientConfig, false);
}
/**
* @param jsc
* @param clientConfig
* @param rollbackInFlight
* @throws Exception
*/
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) {
this.fs = FSUtils.getFs();
this.jsc = jsc;
this.config = clientConfig;
this.index = HoodieIndex.createIndex(config, jsc);
this.metrics = new HoodieMetrics(config, config.getTableName());
this.archiveLog = new HoodieCommitArchiveLog(clientConfig);
if (rollbackInFlight) {
rollbackInflightCommits();
}
}
/**
* Filter out HoodieRecords that already exists in the output folder. This is useful in
* deduplication.
*
* @param hoodieRecords Input RDD of Hoodie records.
* @return A subset of hoodieRecords RDD, with existing records filtered out.
*/
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, metadata);
return recordsWithLocation.filter(new Function<HoodieRecord<T>, Boolean>() {
@Override
public Boolean call(HoodieRecord<T> v1) throws Exception {
return !v1.isCurrentLocationKnown();
}
});
}
/**
* Upserts a bunch of new records into the Hoodie table, at the supplied commitTime
*/
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
writeContext = metrics.getCommitCtx();
final HoodieTable table =
HoodieTable.getHoodieTable(metadata.getTableType(), commitTime, config, metadata);
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeUpsert(), records,
config.getUpsertShuffleParallelism());
// perform index loop up to get existing location of records
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, metadata);
// Cache the tagged records, so we don't end up computing both
taggedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
WorkloadProfile profile = null;
if (table.isWorkloadProfileNeeded()) {
profile = new WorkloadProfile(taggedRecords);
logger.info("Workload profile :" + profile);
}
// obtain the upsert partitioner, and the run the tagger records through that & get a partitioned RDD.
final Partitioner upsertPartitioner = table.getUpsertPartitioner(profile);
JavaRDD<HoodieRecord<T>> partitionedRecords = taggedRecords.mapToPair(
new PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>() {
@Override
public Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> call(
HoodieRecord<T> record) throws Exception {
return new Tuple2<>(new Tuple2<>(record.getKey(),
Option.apply(record.getCurrentLocation())), record);
}
}).partitionBy(upsertPartitioner).map(
new Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(
Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> tuple)
throws Exception {
return tuple._2();
}
});
// Perform the actual writing.
JavaRDD<WriteStatus> upsertStatusRDD = partitionedRecords.mapPartitionsWithIndex(
new Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>>() {
@Override
public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
return table.handleUpsertPartition(partition, recordItr, upsertPartitioner);
}
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses;
}
});
// Update the index back.
JavaRDD<WriteStatus> resultRDD = index.updateLocation(upsertStatusRDD, metadata);
resultRDD = resultRDD.persist(config.getWriteStatusStorageLevel());
boolean commitResult = commit(commitTime, resultRDD);
if (!commitResult) {
throw new HoodieCommitException("Failed to commit " + commitTime);
}
return resultRDD;
} catch (Throwable e) {
if (e instanceof HoodieUpsertException) {
throw (HoodieUpsertException) e;
}
throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e);
}
}
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition,
JavaRDD<HoodieRecord<T>> records, int parallelism) {
if(condition) {
return deduplicateRecords(records, parallelism);
}
return records;
}
/**
* Loads the given HoodieRecords, as inserts into the table.
* (This implementation uses sortBy and attempts to control the numbers of files with less memory)
*
* @param records HoodieRecords to insert
* @param commitTime Commit Time handle
* @return JavaRDD<WriteStatus> - RDD of WriteStatus to inspect errors and counts
*
*/
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
writeContext = metrics.getCommitCtx();
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeInsert(), records,
config.getInsertShuffleParallelism());
// Now, sort the records and line them up nicely for loading.
JavaRDD<HoodieRecord<T>> sortedRecords =
dedupedRecords.sortBy(new Function<HoodieRecord<T>, String>() {
@Override
public String call(HoodieRecord<T> record) {
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
// the records split evenly across RDD partitions, such that small partitions fit
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
return String
.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
}
}, true, config.getInsertShuffleParallelism());
JavaRDD<WriteStatus> writeStatusRDD = sortedRecords
.mapPartitionsWithIndex(new InsertMapFunction<T>(commitTime, config, metadata),
true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses;
}
});
// Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, metadata);
// Trigger the insert and collect statuses
statuses = statuses.persist(config.getWriteStatusStorageLevel());
boolean commitResult = commit(commitTime, statuses);
if (!commitResult) {
throw new HoodieCommitException("Failed to commit " + commitTime);
}
return statuses;
} catch (Throwable e) {
if (e instanceof HoodieInsertException) {
throw e;
}
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
}
}
/**
* Commit changes performed at the given commitTime marker
*/
private boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + FSUtils.makeCommitFileName(commitTime));
try {
if (fs.exists(commitFile)) {
throw new HoodieCommitException("Duplicate commit found. " + commitTime);
}
List<Tuple2<String, HoodieWriteStat>> stats =
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
@Override
public Tuple2<String, HoodieWriteStat> call(WriteStatus writeStatus)
throws Exception {
return new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat());
}
}).collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
for (Tuple2<String, HoodieWriteStat> stat : stats) {
metadata.addWriteStat(stat._1(), stat._2());
}
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(config.getBasePath() + "/.hoodie/" + FSUtils
.makeInflightCommitFileName(commitTime));
FSDataOutputStream fsout = fs.create(inflightCommitFile, true);
fsout.writeBytes(new String(metadata.toJsonString().getBytes(StandardCharsets.UTF_8),
StandardCharsets.UTF_8));
fsout.close();
boolean success = fs.rename(inflightCommitFile, commitFile);
if (success) {
// We cannot have unbounded commit files. Archive commits if we have to archive
archiveLog.archiveIfRequired();
// Call clean to cleanup if there is anything to cleanup after the commit,
clean();
if(writeContext != null) {
long durationInMs = metrics.getDurationInMs(writeContext.stop());
metrics.updateCommitMetrics(FORMATTER.parse(commitTime).getTime(), durationInMs,
metadata);
writeContext = null;
}
}
return success;
} catch (IOException e) {
throw new HoodieCommitException(
"Failed to commit " + config.getBasePath() + " at time " + commitTime, e);
} catch (ParseException e) {
throw new HoodieCommitException(
"Commit time is not of valid format.Failed to commit " + config.getBasePath()
+ " at time " + commitTime, e);
}
}
/**
* Rollback the (inflight/committed) record changes with the given commit time.
* Three steps:
* (0) Obtain the commit or rollback file
* (1) clean indexing data,
* (2) clean new generated parquet files.
* (3) Finally delete .commit or .inflight file,
*/
public boolean rollback(final String commitTime) throws HoodieRollbackException {
final Timer.Context context = metrics.getRollbackCtx();
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
final String metaPath = config.getBasePath() + "/" + HoodieTableMetadata.METAFOLDER_NAME;
try {
// 0. Obtain the commit/.inflight file, to work on
FileStatus[] commitFiles =
fs.globStatus(new Path(metaPath + "/" + commitTime + ".*"));
if (commitFiles.length != 1) {
throw new HoodieRollbackException("Expected exactly one .commit or .inflight file for commitTime: " + commitTime);
}
// we first need to unpublish the commit by making it .inflight again. (this will ensure no future queries see this data)
Path filePath = commitFiles[0].getPath();
if (filePath.getName().endsWith(HoodieTableMetadata.COMMIT_FILE_SUFFIX)) {
if (metadata.findCommitsAfter(commitTime, Integer.MAX_VALUE).size() > 0) {
throw new HoodieRollbackException("Found commits after time :" + commitTime +
", please rollback greater commits first");
}
Path newInflightPath = new Path(metaPath + "/" + commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX);
if (!fs.rename(filePath, newInflightPath)) {
throw new HoodieRollbackException("Unable to rename .commit file to .inflight for commitTime:" + commitTime);
}
filePath = newInflightPath;
}
// 1. Revert the index changes
logger.info("Clean out index changes at time: " + commitTime);
if (!index.rollbackCommit(commitTime)) {
throw new HoodieRollbackException("Clean out index changes failed, for time :" + commitTime);
}
// 2. Delete the new generated parquet files
logger.info("Clean out all parquet files generated at time: " + commitTime);
final Accumulator<Integer> numFilesDeletedAccu = jsc.accumulator(0);
jsc.parallelize(FSUtils.getAllPartitionPaths(fs, metadata.getBasePath()))
.foreach(new VoidFunction<String>() {
@Override
public void call(String partitionPath) throws Exception {
// Scan all partitions files with this commit time
FileSystem fs = FSUtils.getFs();
FileStatus[] toBeDeleted =
fs.listStatus(new Path(config.getBasePath(), partitionPath),
new PathFilter() {
@Override
public boolean accept(Path path) {
return commitTime
.equals(FSUtils.getCommitTime(path.getName()));
}
});
for (FileStatus file : toBeDeleted) {
boolean success = fs.delete(file.getPath(), false);
logger.info("Delete file " + file.getPath() + "\t" + success);
if (success) {
numFilesDeletedAccu.add(1);
}
}
}
});
// 3. Clean out metadata (.commit or .tmp)
logger.info("Clean out metadata files at time: " + commitTime);
if (!fs.delete(filePath, false)) {
logger.error("Deleting file " + filePath + " failed.");
throw new HoodieRollbackException("Delete file " + filePath + " failed.");
}
if (context != null) {
long durationInMs = metrics.getDurationInMs(context.stop());
int numFilesDeleted = numFilesDeletedAccu.value();
metrics.updateRollbackMetrics(durationInMs, numFilesDeleted);
}
return true;
} catch (IOException e) {
throw new HoodieRollbackException("Failed to rollback " +
config.getBasePath() + " at commit time" + commitTime, e);
}
}
/**
* Releases any resources used by the client.
*/
public void close() {
// UNDER CONSTRUCTION
}
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage)
*/
private void clean() throws HoodieIOException {
try {
logger.info("Cleaner started");
final Timer.Context context = metrics.getCleanCtx();
final HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metadata.getBasePath());
// shuffle to distribute cleaning work across partitions evenly
Collections.shuffle(partitionsToClean);
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config.getCleanerPolicy());
if(partitionsToClean.isEmpty()) {
logger.info("Nothing to clean here mom. It is already clean");
return;
}
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
int numFilesDeleted = jsc.parallelize(partitionsToClean, cleanerParallelism)
.map(new Function<String, Integer>() {
@Override
public Integer call(String partitionPathToClean) throws Exception {
FileSystem fs = FSUtils.getFs();
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, fs);
return cleaner.clean(partitionPathToClean);
}
}).reduce(new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer v1, Integer v2) throws Exception {
return v1 + v2;
}
});
logger.info("Cleaned " + numFilesDeleted + " files");
// Emit metrics (duration, numFilesDeleted) if needed
if (context != null) {
long durationInMs = metrics.getDurationInMs(context.stop());
logger.info("cleanerElaspsedTime (Minutes): " + durationInMs / (1000 * 60));
metrics.updateCleanMetrics(durationInMs, numFilesDeleted);
}
} catch (IOException e) {
throw new HoodieIOException("Failed to clean up after commit", e);
}
}
/**
* Provides a new commit time for a write operation (insert/update)
*/
public String startCommit() {
String commitTime = FORMATTER.format(new Date());
startCommitWithTime(commitTime);
return commitTime;
}
public void startCommitWithTime(String commitTime) {
logger.info("Generate a new commit time " + commitTime);
// Create the in-flight commit file
Path inflightCommitFilePath = new Path(
config.getBasePath() + "/.hoodie/" + FSUtils.makeInflightCommitFileName(commitTime));
try {
if (fs.createNewFile(inflightCommitFilePath)) {
logger.info("Create an inflight commit file " + inflightCommitFilePath);
return;
}
throw new HoodieCommitException(
"Failed to create the inflight commit file " + inflightCommitFilePath);
} catch (IOException e) {
// handled below
throw new HoodieCommitException(
"Failed to create the inflight commit file " + inflightCommitFilePath, e);
}
}
public static SparkConf registerClasses(SparkConf conf) {
conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
return conf;
}
/**
* Deduplicate Hoodie records, using the given deduplication funciton.
*/
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records, int parallelism) {
return records.mapToPair(new PairFunction<HoodieRecord<T>, HoodieKey, HoodieRecord<T>>() {
@Override
public Tuple2<HoodieKey, HoodieRecord<T>> call(HoodieRecord<T> record) {
return new Tuple2<>(record.getKey(), record);
}
}).reduceByKey(new Function2<HoodieRecord<T>, HoodieRecord<T>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(HoodieRecord<T> rec1, HoodieRecord<T> rec2) {
@SuppressWarnings("unchecked")
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
// we cannot allow the user to change the key or partitionPath, since that will affect everything
// so pick it from one of the records.
return new HoodieRecord<T>(rec1.getKey(), reducedData);
}
}, parallelism).map(new Function<Tuple2<HoodieKey, HoodieRecord<T>>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(Tuple2<HoodieKey, HoodieRecord<T>> recordTuple) {
return recordTuple._2();
}
});
}
/**
* Cleanup all inflight commits
* @throws IOException
*/
private void rollbackInflightCommits() {
final HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
for (String commit : metadata.getAllInflightCommits()) {
rollback(commit);
}
}
}