1
0

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:
Vinoth Chandar
2016-12-16 14:34:42 -08:00
parent 0512da094b
commit 81874a8406
69 changed files with 10464 additions and 11 deletions

View File

@@ -0,0 +1,451 @@
/*
* 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.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.util.FSUtils;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.io.HoodieUpdateHandle;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.spark.Partitioner;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import scala.Option;
import scala.Tuple2;
/**
* Implementation of a very heavily read-optimized Hoodie Table where
*
* INSERTS - Produce new files, block aligned to desired size (or)
* Merge with the smallest existing file, to expand it
*
* UPDATES - Produce a new version of the file containing the invalidated records
*
*/
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable {
// seed for random number generator. No particular significance, just makes testing deterministic
private static final long RANDOM_NUMBER_SEED = 356374L;
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
enum BucketType {
UPDATE,
INSERT
}
/**
* Helper class for a small file's location and its actual size on disk
*/
class SmallFile implements Serializable {
HoodieRecordLocation location;
long sizeBytes;
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("SmallFile {");
sb.append("location=").append(location).append(", ");
sb.append("sizeBytes=").append(sizeBytes);
sb.append('}');
return sb.toString();
}
}
/**
* Helper class for an insert bucket along with the weight [0.0, 0.1]
* that defines the amount of incoming inserts that should be allocated to
* the bucket
*/
class InsertBucket implements Serializable {
int bucketNumber;
// fraction of total inserts, that should go into this bucket
double weight;
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadStat {");
sb.append("bucketNumber=").append(bucketNumber).append(", ");
sb.append("weight=").append(weight);
sb.append('}');
return sb.toString();
}
}
/**
* Helper class for a bucket's type (INSERT & UPDATE) and its file location
*/
class BucketInfo implements Serializable {
BucketType bucketType;
String fileLoc;
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("BucketInfo {");
sb.append("bucketType=").append(bucketType).append(", ");
sb.append("fileLoc=").append(fileLoc);
sb.append('}');
return sb.toString();
}
}
public HoodieCopyOnWriteTable(String commitTime, HoodieWriteConfig config, HoodieTableMetadata metadata) {
super(commitTime, config, metadata);
}
/**
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition)
*/
class UpsertPartitioner extends Partitioner {
/**
* Total number of RDD partitions, is determined by total buckets we want to
* pack the incoming workload into
*/
private int totalBuckets = 0;
/**
* Helps decide which bucket an incoming update should go to.
*/
private HashMap<String, Integer> updateLocationToBucket;
/**
* Helps us pack inserts into 1 or more buckets depending on number of
* incoming records.
*/
private HashMap<String, List<InsertBucket>> partitionPathToInsertBuckets;
/**
* Remembers what type each bucket is for later.
*/
private HashMap<Integer, BucketInfo> bucketInfoMap;
/**
* Random number generator to use for splitting inserts into buckets by weight
*/
private Random rand = new Random(RANDOM_NUMBER_SEED);
UpsertPartitioner(WorkloadProfile profile) {
updateLocationToBucket = new HashMap<>();
partitionPathToInsertBuckets = new HashMap<>();
bucketInfoMap = new HashMap<>();
assignUpdates(profile);
assignInserts(profile);
logger.info("Total Buckets :" + totalBuckets + ", " +
"buckets info => " + bucketInfoMap + ", \n" +
"Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n" +
"UpdateLocations mapped to buckets =>" + updateLocationToBucket);
}
private void assignUpdates(WorkloadProfile profile) {
// each update location gets a partition
WorkloadStat gStat = profile.getGlobalStat();
for (Map.Entry<String, Long> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) {
addUpdateBucket(updateLocEntry.getKey());
}
}
private int addUpdateBucket(String fileLoc) {
int bucket = totalBuckets;
updateLocationToBucket.put(fileLoc, bucket);
BucketInfo bucketInfo = new BucketInfo();
bucketInfo.bucketType = BucketType.UPDATE;
bucketInfo.fileLoc = fileLoc;
bucketInfoMap.put(totalBuckets, bucketInfo);
totalBuckets++;
return bucket;
}
private void assignInserts(WorkloadProfile profile) {
// for new inserts, compute buckets depending on how many records we have for each partition
Set<String> partitionPaths = profile.getPartitionPaths();
long averageRecordSize = averageBytesPerRecord();
logger.info("AvgRecordSize => " + averageRecordSize);
for (String partitionPath : partitionPaths) {
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
if (pStat.getNumInserts() > 0) {
List<SmallFile> smallFiles = getSmallFiles(partitionPath);
logger.info("For partitionPath : "+ partitionPath + " Small Files => " + smallFiles);
long totalUnassignedInserts = pStat.getNumInserts();
List<Integer> bucketNumbers = new ArrayList<>();
List<Long> recordsPerBucket = new ArrayList<>();
// first try packing this into one of the smallFiles
for (SmallFile smallFile: smallFiles) {
long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes)/ averageRecordSize, totalUnassignedInserts);
if (recordsToAppend > 0 && totalUnassignedInserts > 0){
// create a new bucket or re-use an existing bucket
int bucket;
if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) {
bucket = updateLocationToBucket.get(smallFile.location.getFileId());
logger.info("Assigning " + recordsToAppend + " inserts to existing update bucket "+ bucket);
} else {
bucket = addUpdateBucket(smallFile.location.getFileId());
logger.info("Assigning " + recordsToAppend + " inserts to new update bucket "+ bucket);
}
bucketNumbers.add(bucket);
recordsPerBucket.add(recordsToAppend);
totalUnassignedInserts -= recordsToAppend;
}
}
// if we have anything more, create new insert buckets, like normal
if (totalUnassignedInserts > 0) {
long insertRecordsPerBucket = config.getCopyOnWriteInsertSplitSize();
if (config.shouldAutoTuneInsertSplits()) {
insertRecordsPerBucket = config.getParquetMaxFileSize()/averageRecordSize;
}
int insertBuckets = (int) Math.max(totalUnassignedInserts / insertRecordsPerBucket, 1L);
logger.info("After small file assignment: unassignedInserts => " + totalUnassignedInserts
+ ", totalInsertBuckets => " + insertBuckets
+ ", recordsPerBucket => " + insertRecordsPerBucket);
for (int b = 0; b < insertBuckets; b++) {
bucketNumbers.add(totalBuckets);
recordsPerBucket.add(totalUnassignedInserts/insertBuckets);
BucketInfo bucketInfo = new BucketInfo();
bucketInfo.bucketType = BucketType.INSERT;
bucketInfoMap.put(totalBuckets, bucketInfo);
totalBuckets++;
}
}
// Go over all such buckets, and assign weights as per amount of incoming inserts.
List<InsertBucket> insertBuckets = new ArrayList<>();
for (int i = 0; i < bucketNumbers.size(); i++) {
InsertBucket bkt = new InsertBucket();
bkt.bucketNumber = bucketNumbers.get(i);
bkt.weight = (1.0 * recordsPerBucket.get(i))/pStat.getNumInserts();
insertBuckets.add(bkt);
}
logger.info("Total insert buckets for partition path "+ partitionPath + " => " + insertBuckets);
partitionPathToInsertBuckets.put(partitionPath, insertBuckets);
}
}
}
/**
* Returns a list of small files in the given partition path
*
* @param partitionPath
* @return
*/
private List<SmallFile> getSmallFiles(String partitionPath) {
FileSystem fs = FSUtils.getFs();
List<SmallFile> smallFileLocations = new ArrayList<>();
if (metadata.getAllCommits().getNumCommits() > 0) { // if we have some commits
String latestCommitTime = metadata.getAllCommits().lastCommit();
FileStatus[] allFiles = metadata.getLatestVersionInPartition(fs, partitionPath, latestCommitTime);
if (allFiles != null && allFiles.length > 0) {
for (FileStatus fileStatus : allFiles) {
if (fileStatus.getLen() < config.getParquetSmallFileLimit()) {
String filename = fileStatus.getPath().getName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(
FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename));
sf.sizeBytes = fileStatus.getLen();
smallFileLocations.add(sf);
}
}
}
}
return smallFileLocations;
}
/**
* Obtains the average record size based on records written during last commit.
* Used for estimating how many records pack into one file.
*
* @return
*/
private long averageBytesPerRecord() {
long avgSize = 0L;
try {
if (metadata.getAllCommits().getNumCommits() > 0) {
String latestCommitTime = metadata.getAllCommits().lastCommit();
HoodieCommitMetadata commitMetadata = metadata.getCommitMetadata(latestCommitTime);
avgSize =(long) Math.ceil((1.0 * commitMetadata.fetchTotalBytesWritten())/commitMetadata.fetchTotalRecordsWritten());
}
} catch (Throwable t) {
// make this fail safe.
logger.error("Error trying to compute average bytes/record ", t);
}
return avgSize <= 0L ? config.getCopyOnWriteRecordSizeEstimate() : avgSize;
}
public BucketInfo getBucketInfo(int bucketNumber) {
return bucketInfoMap.get(bucketNumber);
}
public List<InsertBucket> getInsertBuckets(String partitionPath) {
return partitionPathToInsertBuckets.get(partitionPath);
}
@Override
public int numPartitions() {
return totalBuckets;
}
@Override
public int getPartition(Object key) {
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
if (keyLocation._2().isDefined()) {
HoodieRecordLocation location = keyLocation._2().get();
return updateLocationToBucket.get(location.getFileId());
} else {
List<InsertBucket> targetBuckets = partitionPathToInsertBuckets.get(keyLocation._1().getPartitionPath());
// pick the target bucket to use based on the weights.
double totalWeight = 0.0;
double r = rand.nextDouble();
for (InsertBucket insertBucket: targetBuckets) {
totalWeight += insertBucket.weight;
if (r <= totalWeight) {
return insertBucket.bucketNumber;
}
}
// return first one, by default
return targetBuckets.get(0).bucketNumber;
}
}
}
@Override
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
if (profile == null) {
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
}
return new UpsertPartitioner(profile);
}
@Override
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
return null;
}
@Override
public boolean isWorkloadProfileNeeded() {
return true;
}
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr) throws Exception {
// these are updates
HoodieUpdateHandle upsertHandle =
new HoodieUpdateHandle<>(config, commitTime, metadata, recordItr, fileLoc);
if (upsertHandle.getOldFilePath() == null) {
logger.error("Error in finding the old file path at commit " + commitTime);
} else {
Configuration conf = FSUtils.getFs().getConf();
AvroReadSupport.setAvroReadSchema(conf, upsertHandle.getSchema());
ParquetReader<IndexedRecord> reader =
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(conf).build();
try {
IndexedRecord record;
while ((record = reader.read()) != null) {
// Two types of writes here (new record, and old record).
// We have already catch the exception during writing new records.
// But for old records, we should fail if any exception happens.
upsertHandle.write((GenericRecord) record);
}
} catch (IOException e) {
throw new HoodieUpsertException(
"Failed to read record from " + upsertHandle.getOldFilePath()
+ " with new Schema " + upsertHandle.getSchema(), e);
} finally {
reader.close();
upsertHandle.close();
}
}
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
logger.info(
"Upsert Handle has partition path as null " + upsertHandle.getOldFilePath()
+ ", " + upsertHandle.getWriteStatus());
}
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
}
public Iterator<List<WriteStatus>> handleInsert(Iterator<HoodieRecord<T>> recordItr) throws Exception {
return new LazyInsertIterable<>(recordItr, config, commitTime, metadata);
}
@Override
public Iterator<List<WriteStatus>> handleUpsertPartition(Integer partition,
Iterator recordItr,
Partitioner partitioner) {
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
BucketType btype = binfo.bucketType;
try {
if (btype.equals(BucketType.INSERT)) {
return handleInsert(recordItr);
} else if (btype.equals(BucketType.UPDATE)) {
return handleUpdate(binfo.fileLoc, recordItr);
} else {
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
}
} catch (Throwable t) {
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
logger.error(msg, t);
throw new HoodieUpsertException(msg, t);
}
}
}