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,224 @@
|
||||
/*
|
||||
* 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.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||
*
|
||||
* 1) It provides sufficient time for existing queries running on older versions, to finish
|
||||
*
|
||||
* 2) It bounds the growth of the files in the file system
|
||||
*
|
||||
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
|
||||
*
|
||||
*
|
||||
*/
|
||||
public class HoodieCleaner {
|
||||
|
||||
public enum CleaningPolicy {
|
||||
KEEP_LATEST_FILE_VERSIONS,
|
||||
KEEP_LATEST_COMMITS
|
||||
}
|
||||
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
|
||||
|
||||
|
||||
private HoodieTableMetadata metadata;
|
||||
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCleaner(HoodieTableMetadata metadata,
|
||||
HoodieWriteConfig config,
|
||||
FileSystem fs) {
|
||||
this.metadata = metadata;
|
||||
this.config = config;
|
||||
this.fs = fs;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* Selects the older versions of files for cleaning, such that it bounds the number of versions of each file.
|
||||
* This policy is useful, if you are simply interested in querying the table, and you don't want too many
|
||||
* versions for a single file (i.e run it with versionsRetained = 1)
|
||||
*
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
|
||||
logger.info("Cleaning "+ partitionPath+", retaining latest "+ config.getCleanerFileVersionsRetained()+" file versions. ");
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
for (String file : fileVersions.keySet()) {
|
||||
List<FileStatus> commitList = fileVersions.get(file);
|
||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||
Iterator<FileStatus> commitItr = commitList.iterator();
|
||||
while (commitItr.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
commitItr.next();
|
||||
keepVersions--;
|
||||
}
|
||||
// Delete the remaining files
|
||||
while (commitItr.hasNext()) {
|
||||
deletePaths.add(String.format("%s/%s/%s",
|
||||
config.getBasePath(),
|
||||
partitionPath,
|
||||
commitItr.next().getPath().getName()));
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Selects the versions for file for cleaning, such that it
|
||||
*
|
||||
* - Leaves the latest version of the file untouched
|
||||
* - For older versions,
|
||||
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
|
||||
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
|
||||
* This is essential to leave the file used by the query thats running for the max time.
|
||||
*
|
||||
* This provides the effect of having lookback into all changes that happened in the last X
|
||||
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
|
||||
*
|
||||
* This policy is the default.
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger.info(
|
||||
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
if (commits.getNumCommits() > commitsRetained) {
|
||||
String earliestCommitToRetain =
|
||||
commits.nthCommit(commits.getNumCommits() - commitsRetained);
|
||||
Map<String, List<FileStatus>> fileVersions =
|
||||
metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (String file : fileVersions.keySet()) {
|
||||
List<FileStatus> fileList = fileVersions.get(file);
|
||||
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getPath().getName());
|
||||
String lastVersionBeforeEarliestCommitToRetain =
|
||||
getLatestVersionBeforeCommit(fileList, earliestCommitToRetain);
|
||||
|
||||
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||
// i.e always spare the last commit.
|
||||
for (FileStatus afile : fileList) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(afile.getPath().getName());
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (
|
||||
lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime
|
||||
.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (HoodieCommits.isCommit1After(earliestCommitToRetain, fileCommitTime)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
deletePaths.add(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutTaskPartitionId(fileCommitTime, file)));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileStatus> fileList, String commitTime) {
|
||||
for (FileStatus file : fileList) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(file.getPath().getName());
|
||||
if (HoodieCommits.isCommit1After(commitTime, fileCommitTime)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
// There is no version of this file which is <= commitTime
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Performs cleaning of the partition path according to cleaning policy and returns the number
|
||||
* of files cleaned.
|
||||
*
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
public int clean(String partitionPath) throws IOException {
|
||||
CleaningPolicy policy = config.getCleanerPolicy();
|
||||
List<String> deletePaths;
|
||||
if (policy == CleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||
} else if (policy == CleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
|
||||
// perform the actual deletes
|
||||
for (String deletePath : deletePaths) {
|
||||
logger.info("Working on delete path :" + deletePath);
|
||||
FileStatus[] deleteVersions = fs.globStatus(new Path(deletePath));
|
||||
if (deleteVersions != null) {
|
||||
for (FileStatus deleteVersion : deleteVersions) {
|
||||
if (fs.delete(deleteVersion.getPath(), false)) {
|
||||
logger.info("Cleaning file at path :" + deleteVersion.getPath());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
logger.info(deletePaths.size() + " files deleted for partition path:" + partitionPath);
|
||||
return deletePaths.size();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,144 @@
|
||||
/*
|
||||
* 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.file.HoodieAppendLog;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.BZip2Codec;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Log to hold older historical commits, to bound the growth of .commit files
|
||||
*/
|
||||
public class HoodieCommitArchiveLog {
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final FileSystem fs;
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config) {
|
||||
this.archiveFilePath =
|
||||
new Path(config.getBasePath(),
|
||||
HoodieTableMetadata.METAFOLDER_NAME + "/" +HOODIE_COMMIT_ARCHIVE_LOG_FILE);
|
||||
this.fs = FSUtils.getFs();
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired() {
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath());
|
||||
List<String> commitsToArchive = getCommitsToArchive(metadata);
|
||||
if (!commitsToArchive.isEmpty()) {
|
||||
log.info("Archiving commits " + commitsToArchive);
|
||||
archive(metadata, commitsToArchive);
|
||||
return deleteCommits(metadata, commitsToArchive);
|
||||
} else {
|
||||
log.info("No Commits to archive");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private List<String> getCommitsToArchive(HoodieTableMetadata metadata) {
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
List<String> commits = metadata.getAllCommits().getCommitList();
|
||||
List<String> commitsToArchive = new ArrayList<String>();
|
||||
if (commits.size() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
commitsToArchive = commits.subList(0, commits.size() - minCommitsToKeep);
|
||||
}
|
||||
return commitsToArchive;
|
||||
}
|
||||
|
||||
private boolean deleteCommits(HoodieTableMetadata metadata, List<String> commitsToArchive) {
|
||||
log.info("Deleting commits " + commitsToArchive);
|
||||
boolean success = true;
|
||||
for(String commitToArchive:commitsToArchive) {
|
||||
Path commitFile =
|
||||
new Path(metadata.getBasePath() + "/" +
|
||||
HoodieTableMetadata.METAFOLDER_NAME + "/" +
|
||||
FSUtils.makeCommitFileName(commitToArchive));
|
||||
try {
|
||||
if (fs.exists(commitFile)) {
|
||||
success &= fs.delete(commitFile, false);
|
||||
log.info("Archived and deleted commit file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to delete archived commit " + commitToArchive, e);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
private HoodieAppendLog.Writer openWriter() throws IOException {
|
||||
log.info("Opening archive file at path: " + archiveFilePath);
|
||||
return HoodieAppendLog
|
||||
.createWriter(fs.getConf(), HoodieAppendLog.Writer.file(archiveFilePath),
|
||||
HoodieAppendLog.Writer.keyClass(Text.class),
|
||||
HoodieAppendLog.Writer.appendIfExists(true),
|
||||
HoodieAppendLog.Writer.valueClass(Text.class), HoodieAppendLog.Writer
|
||||
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
|
||||
}
|
||||
|
||||
private void archive(HoodieTableMetadata metadata, List<String> commits)
|
||||
throws HoodieCommitException {
|
||||
HoodieAppendLog.Writer writer = null;
|
||||
try {
|
||||
writer = openWriter();
|
||||
for (String commitTime : commits) {
|
||||
Text k = new Text(commitTime);
|
||||
Text v = new Text(metadata.getCommitMetadata(commitTime).toJsonString());
|
||||
writer.append(k, v);
|
||||
log.info("Wrote " + k);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("Could not archive commits " + commits, e);
|
||||
} finally {
|
||||
if (writer != null) {
|
||||
try {
|
||||
writer.hsync();
|
||||
writer.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException(
|
||||
"Could not close the archive commits writer " + commits, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public Path getArchiveFilePath() {
|
||||
return archiveFilePath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,92 @@
|
||||
/*
|
||||
* 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.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
|
||||
protected final String commitTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTableMetadata metadata;
|
||||
protected final Schema schema;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTableMetadata metadata) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.metadata = metadata;
|
||||
this.schema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
||||
Path path = new Path(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
|
||||
return new Path(path.toString(),
|
||||
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes any new tmp files written during the current commit, into the partition
|
||||
*/
|
||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
String partitionPath,
|
||||
int taskPartitionId) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
try {
|
||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
||||
if (prevFailedFiles != null) {
|
||||
logger.info("Deleting " + prevFailedFiles.length
|
||||
+ " files generated by previous failed attempts.");
|
||||
for (FileStatus status : prevFailedFiles) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
public Schema getSchema() {
|
||||
return schema;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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.WriteStatus;
|
||||
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.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
|
||||
public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieInsertHandle.class);
|
||||
|
||||
private final WriteStatus status;
|
||||
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private final Path path;
|
||||
private int recordsWritten = 0;
|
||||
|
||||
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTableMetadata metadata, String partitionPath) {
|
||||
super(config, commitTime, metadata);
|
||||
this.status = new WriteStatus();
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
status.setPartitionPath(partitionPath);
|
||||
|
||||
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
|
||||
try {
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, metadata, config, schema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
* <p/>
|
||||
* - Whether it belongs to the same partitionPath as existing records
|
||||
* - Whether the current file written bytes < max file size
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath()
|
||||
.equals(status.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*
|
||||
* @param record
|
||||
*/
|
||||
public void write(HoodieRecord record) {
|
||||
try {
|
||||
IndexedRecord avroRecord = record.getData().getInsertValue(schema);
|
||||
storageWriter.writeAvroWithMetadata(avroRecord, record);
|
||||
status.markSuccess(record);
|
||||
// update the new location of record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
|
||||
record.deflate();
|
||||
recordsWritten++;
|
||||
} catch (Throwable t) {
|
||||
status.markFailure(record, t);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public WriteStatus close() {
|
||||
logger.info(
|
||||
"Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
storageWriter.close();
|
||||
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(status.getFileId());
|
||||
stat.setFullPath(path.toString());
|
||||
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
|
||||
stat.setTotalWriteErrors(status.getFailedRecords().size());
|
||||
status.setStat(stat);
|
||||
|
||||
return status;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,193 @@
|
||||
/*
|
||||
* 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.WriteStatus;
|
||||
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.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
||||
@SuppressWarnings("Duplicates") public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieUpdateHandle.class);
|
||||
|
||||
private final WriteStatus writeStatus;
|
||||
private final HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
private long recordsWritten = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
private String fileId;
|
||||
|
||||
public HoodieUpdateHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTableMetadata metadata,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
super(config, commitTime, metadata);
|
||||
WriteStatus writeStatus = new WriteStatus();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.fileId = fileId;
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
init(recordItr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map, and extract the old file path.
|
||||
*/
|
||||
private void init(Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
try {
|
||||
// Load the new records in a map
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// If the first record, we need to extract some info out
|
||||
if (oldFilePath == null) {
|
||||
String latestValidFilePath = metadata.getFilenameForRecord(fs, record, fileId);
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + record.getPartitionPath() + "/"
|
||||
+ latestValidFilePath);
|
||||
newFilePath = new Path(
|
||||
config.getBasePath() + "/" + record.getPartitionPath() + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
|
||||
// handle cases of partial failures, for update task
|
||||
if (fs.exists(newFilePath)) {
|
||||
fs.delete(newFilePath, false);
|
||||
}
|
||||
|
||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||
oldFilePath.toString(), newFilePath.toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
writeStatus.getStat().setFullPath(newFilePath.toString());
|
||||
}
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
}
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, newFilePath, metadata, config, schema);
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void writeUpdateRecord(HoodieRecord<T> hoodieRecord, IndexedRecord indexedRecord) {
|
||||
try {
|
||||
storageWriter.writeAvroWithMetadata(indexedRecord, hoodieRecord);
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord);
|
||||
recordsWritten ++;
|
||||
updatedRecordsWritten ++;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record "+ hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
if (keyToNewRecords.containsKey(key)) {
|
||||
try {
|
||||
IndexedRecord avroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
|
||||
writeUpdateRecord(hoodieRecord, avroRecord);
|
||||
keyToNewRecords.remove(key);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieUpsertException("Failed to combine/merge new record with old value in storage, for new record {"
|
||||
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
||||
}
|
||||
} else {
|
||||
// this should work as it is, since this is an existing record
|
||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath;
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error(
|
||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
|
||||
.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath, e);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
recordsWritten ++;
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
||||
while (pendingRecordsItr.hasNext()) {
|
||||
String key = pendingRecordsItr.next();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
|
||||
if (storageWriter != null) {
|
||||
storageWriter.close();
|
||||
}
|
||||
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getOldFilePath() {
|
||||
return oldFilePath;
|
||||
}
|
||||
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
public class HoodieParquetConfig {
|
||||
private HoodieAvroWriteSupport writeSupport;
|
||||
private CompressionCodecName compressionCodecName;
|
||||
private int blockSize;
|
||||
private int pageSize;
|
||||
private int maxFileSize;
|
||||
private Configuration hadoopConf;
|
||||
|
||||
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
||||
CompressionCodecName compressionCodecName, int blockSize, int pageSize, int maxFileSize,
|
||||
Configuration hadoopConf) {
|
||||
this.writeSupport = writeSupport;
|
||||
this.compressionCodecName = compressionCodecName;
|
||||
this.blockSize = blockSize;
|
||||
this.pageSize = pageSize;
|
||||
this.maxFileSize = maxFileSize;
|
||||
this.hadoopConf = hadoopConf;
|
||||
}
|
||||
|
||||
public HoodieAvroWriteSupport getWriteSupport() {
|
||||
return writeSupport;
|
||||
}
|
||||
|
||||
public CompressionCodecName getCompressionCodecName() {
|
||||
return compressionCodecName;
|
||||
}
|
||||
|
||||
public int getBlockSize() {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
public int getPageSize() {
|
||||
return pageSize;
|
||||
}
|
||||
|
||||
public int getMaxFileSize() {
|
||||
return maxFileSize;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
}
|
||||
@@ -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.io.storage;
|
||||
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file.
|
||||
* Provides a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
|
||||
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
private static double STREAM_COMPRESSION_RATIO = 0.1;
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String commitTime;
|
||||
private final Schema schema;
|
||||
|
||||
|
||||
private static Configuration registerFileSystem(Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||
HoodieWrapperFileSystem.class.getName());
|
||||
return returnConf;
|
||||
}
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file,
|
||||
HoodieParquetConfig parquetConfig, Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
this.file =
|
||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO);
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord,
|
||||
record.getRecordKey(),
|
||||
record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
|
||||
@Override public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||
super.write(object);
|
||||
writeSupport.add(key);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface HoodieStorageWriter<R extends IndexedRecord> {
|
||||
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
||||
boolean canWrite();
|
||||
void close() throws IOException;
|
||||
void writeAvro(String key, R oldRecord) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTableMetadata metadata, HoodieWriteConfig config, Schema schema)
|
||||
throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
|
||||
BloomFilter filter =
|
||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,677 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.fs.permission.AclStatus;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
/**
|
||||
* HoodieWrapperFileSystem wraps the default file system.
|
||||
* It holds state about the open streams in the file system to support getting the
|
||||
* written size to each of the open streams.
|
||||
*/
|
||||
public class HoodieWrapperFileSystem extends FileSystem {
|
||||
private static final Set<String> SUPPORT_SCHEMES;
|
||||
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
|
||||
|
||||
static {
|
||||
SUPPORT_SCHEMES = new HashSet<>(2);
|
||||
SUPPORT_SCHEMES.add("file");
|
||||
SUPPORT_SCHEMES.add("hdfs");
|
||||
}
|
||||
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams =
|
||||
new ConcurrentHashMap<>();
|
||||
private FileSystem fileSystem;
|
||||
private URI uri;
|
||||
|
||||
@Override public void initialize(URI uri, Configuration conf) throws IOException {
|
||||
// Get the default filesystem to decorate
|
||||
fileSystem = FileSystem.get(conf);
|
||||
// Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
|
||||
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
|
||||
// fileSystem.setConf(conf);
|
||||
this.uri = uri;
|
||||
}
|
||||
|
||||
@Override public URI getUri() {
|
||||
return uri;
|
||||
}
|
||||
|
||||
@Override public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
||||
return fileSystem.open(convertToDefaultPath(f), bufferSize);
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
final Path translatedPath = convertToDefaultPath(f);
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize,
|
||||
progress));
|
||||
}
|
||||
|
||||
private FSDataOutputStream wrapOutputStream(final Path path,
|
||||
FSDataOutputStream fsDataOutputStream) throws IOException {
|
||||
if (fsDataOutputStream instanceof SizeAwareFSDataOutputStream) {
|
||||
return fsDataOutputStream;
|
||||
}
|
||||
|
||||
SizeAwareFSDataOutputStream os =
|
||||
new SizeAwareFSDataOutputStream(fsDataOutputStream, new Runnable() {
|
||||
@Override public void run() {
|
||||
openStreams.remove(path.getName());
|
||||
}
|
||||
});
|
||||
openStreams.put(path.getName(), os);
|
||||
return os;
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, boolean overwrite) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite));
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)));
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, Progressable progress) throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), progress);
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, short replication) throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), replication);
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, short replication, Progressable progress)
|
||||
throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), replication, progress);
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize)
|
||||
throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize);
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
|
||||
Progressable progress) throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress,
|
||||
Options.ChecksumOpt checksumOpt) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
|
||||
progress, checksumOpt);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize);
|
||||
}
|
||||
|
||||
|
||||
@Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
return fileSystem.append(convertToDefaultPath(f), bufferSize, progress);
|
||||
}
|
||||
|
||||
@Override public boolean rename(Path src, Path dst) throws IOException {
|
||||
return fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public boolean delete(Path f, boolean recursive) throws IOException {
|
||||
return fileSystem.delete(convertToDefaultPath(f), recursive);
|
||||
}
|
||||
|
||||
@Override public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public void setWorkingDirectory(Path new_dir) {
|
||||
fileSystem.setWorkingDirectory(convertToDefaultPath(new_dir));
|
||||
}
|
||||
|
||||
@Override public Path getWorkingDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getWorkingDirectory());
|
||||
}
|
||||
|
||||
@Override public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
||||
return fileSystem.mkdirs(convertToDefaultPath(f), permission);
|
||||
}
|
||||
|
||||
@Override public FileStatus getFileStatus(Path f) throws IOException {
|
||||
return fileSystem.getFileStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public String getScheme() {
|
||||
return uri.getScheme();
|
||||
}
|
||||
|
||||
@Override public String getCanonicalServiceName() {
|
||||
return fileSystem.getCanonicalServiceName();
|
||||
}
|
||||
|
||||
@Override public String getName() {
|
||||
return fileSystem.getName();
|
||||
}
|
||||
|
||||
@Override public Path makeQualified(Path path) {
|
||||
return convertToHoodiePath(fileSystem.makeQualified(convertToDefaultPath(path)));
|
||||
}
|
||||
|
||||
@Override public Token<?> getDelegationToken(String renewer) throws IOException {
|
||||
return fileSystem.getDelegationToken(renewer);
|
||||
}
|
||||
|
||||
@Override public Token<?>[] addDelegationTokens(String renewer, Credentials credentials)
|
||||
throws IOException {
|
||||
return fileSystem.addDelegationTokens(renewer, credentials);
|
||||
}
|
||||
|
||||
@Override public FileSystem[] getChildFileSystems() {
|
||||
return fileSystem.getChildFileSystems();
|
||||
}
|
||||
|
||||
@Override public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len)
|
||||
throws IOException {
|
||||
return fileSystem.getFileBlockLocations(file, start, len);
|
||||
}
|
||||
|
||||
@Override public BlockLocation[] getFileBlockLocations(Path p, long start, long len)
|
||||
throws IOException {
|
||||
return fileSystem.getFileBlockLocations(convertToDefaultPath(p), start, len);
|
||||
}
|
||||
|
||||
@Override public FsServerDefaults getServerDefaults() throws IOException {
|
||||
return fileSystem.getServerDefaults();
|
||||
}
|
||||
|
||||
@Override public FsServerDefaults getServerDefaults(Path p) throws IOException {
|
||||
return fileSystem.getServerDefaults(convertToDefaultPath(p));
|
||||
}
|
||||
|
||||
@Override public Path resolvePath(Path p) throws IOException {
|
||||
return convertToHoodiePath(fileSystem.resolvePath(convertToDefaultPath(p)));
|
||||
}
|
||||
|
||||
@Override public FSDataInputStream open(Path f) throws IOException {
|
||||
return fileSystem.open(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication,
|
||||
blockSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize,
|
||||
replication, blockSize, progress);
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
|
||||
EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
|
||||
Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), permission, flags, bufferSize, replication,
|
||||
blockSize, progress);
|
||||
}
|
||||
|
||||
@Override public boolean createNewFile(Path f) throws IOException {
|
||||
return fileSystem.createNewFile(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream append(Path f) throws IOException {
|
||||
return fileSystem.append(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public FSDataOutputStream append(Path f, int bufferSize) throws IOException {
|
||||
return fileSystem.append(convertToDefaultPath(f), bufferSize);
|
||||
}
|
||||
|
||||
@Override public void concat(Path trg, Path[] psrcs) throws IOException {
|
||||
Path[] psrcsNew = convertDefaults(psrcs);
|
||||
fileSystem.concat(convertToDefaultPath(trg), psrcsNew);
|
||||
}
|
||||
|
||||
@Override public short getReplication(Path src) throws IOException {
|
||||
return fileSystem.getReplication(convertToDefaultPath(src));
|
||||
}
|
||||
|
||||
@Override public boolean setReplication(Path src, short replication) throws IOException {
|
||||
return fileSystem.setReplication(convertToDefaultPath(src), replication);
|
||||
}
|
||||
|
||||
@Override public boolean delete(Path f) throws IOException {
|
||||
return fileSystem.delete(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public boolean deleteOnExit(Path f) throws IOException {
|
||||
return fileSystem.deleteOnExit(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public boolean cancelDeleteOnExit(Path f) {
|
||||
return fileSystem.cancelDeleteOnExit(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public boolean exists(Path f) throws IOException {
|
||||
return fileSystem.exists(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public boolean isDirectory(Path f) throws IOException {
|
||||
return fileSystem.isDirectory(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public boolean isFile(Path f) throws IOException {
|
||||
return fileSystem.isFile(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public long getLength(Path f) throws IOException {
|
||||
return fileSystem.getLength(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public ContentSummary getContentSummary(Path f) throws IOException {
|
||||
return fileSystem.getContentSummary(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public RemoteIterator<Path> listCorruptFileBlocks(Path path) throws IOException {
|
||||
return fileSystem.listCorruptFileBlocks(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public FileStatus[] listStatus(Path f, PathFilter filter)
|
||||
throws FileNotFoundException, IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
}
|
||||
|
||||
@Override public FileStatus[] listStatus(Path[] files)
|
||||
throws FileNotFoundException, IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
}
|
||||
|
||||
@Override public FileStatus[] listStatus(Path[] files, PathFilter filter)
|
||||
throws FileNotFoundException, IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
}
|
||||
|
||||
@Override public FileStatus[] globStatus(Path pathPattern) throws IOException {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern));
|
||||
}
|
||||
|
||||
@Override public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
|
||||
throws IOException {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
|
||||
}
|
||||
|
||||
@Override public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
|
||||
throws FileNotFoundException, IOException {
|
||||
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
|
||||
throws FileNotFoundException, IOException {
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
}
|
||||
|
||||
@Override public Path getHomeDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getHomeDirectory());
|
||||
}
|
||||
|
||||
@Override public boolean mkdirs(Path f) throws IOException {
|
||||
return fileSystem.mkdirs(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public void copyFromLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.copyFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void moveFromLocalFile(Path[] srcs, Path dst) throws IOException {
|
||||
fileSystem.moveFromLocalFile(convertDefaults(srcs), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void moveFromLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.moveFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
|
||||
fileSystem.copyFromLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)
|
||||
throws IOException {
|
||||
fileSystem
|
||||
.copyFromLocalFile(delSrc, overwrite, convertDefaults(srcs), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
|
||||
throws IOException {
|
||||
fileSystem.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src),
|
||||
convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void copyToLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void moveToLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
|
||||
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)
|
||||
throws IOException {
|
||||
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst),
|
||||
useRawLocalFileSystem);
|
||||
}
|
||||
|
||||
@Override public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
|
||||
throws IOException {
|
||||
return convertToHoodiePath(fileSystem.startLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile)));
|
||||
}
|
||||
|
||||
@Override public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
|
||||
throws IOException {
|
||||
fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile));
|
||||
}
|
||||
|
||||
@Override public void close() throws IOException {
|
||||
fileSystem.close();
|
||||
}
|
||||
|
||||
@Override public long getUsed() throws IOException {
|
||||
return fileSystem.getUsed();
|
||||
}
|
||||
|
||||
@Override public long getBlockSize(Path f) throws IOException {
|
||||
return fileSystem.getBlockSize(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public long getDefaultBlockSize() {
|
||||
return fileSystem.getDefaultBlockSize();
|
||||
}
|
||||
|
||||
@Override public long getDefaultBlockSize(Path f) {
|
||||
return fileSystem.getDefaultBlockSize(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public short getDefaultReplication() {
|
||||
return fileSystem.getDefaultReplication();
|
||||
}
|
||||
|
||||
@Override public short getDefaultReplication(Path path) {
|
||||
return fileSystem.getDefaultReplication(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public void access(Path path, FsAction mode)
|
||||
throws AccessControlException, FileNotFoundException, IOException {
|
||||
fileSystem.access(convertToDefaultPath(path), mode);
|
||||
}
|
||||
|
||||
@Override public void createSymlink(Path target, Path link, boolean createParent)
|
||||
throws AccessControlException, FileAlreadyExistsException, FileNotFoundException,
|
||||
ParentNotDirectoryException, UnsupportedFileSystemException, IOException {
|
||||
fileSystem
|
||||
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
|
||||
}
|
||||
|
||||
@Override public FileStatus getFileLinkStatus(Path f)
|
||||
throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException,
|
||||
IOException {
|
||||
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public boolean supportsSymlinks() {
|
||||
return fileSystem.supportsSymlinks();
|
||||
}
|
||||
|
||||
@Override public Path getLinkTarget(Path f) throws IOException {
|
||||
return convertToHoodiePath(fileSystem.getLinkTarget(convertToDefaultPath(f)));
|
||||
}
|
||||
|
||||
@Override public FileChecksum getFileChecksum(Path f) throws IOException {
|
||||
return fileSystem.getFileChecksum(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override public FileChecksum getFileChecksum(Path f, long length) throws IOException {
|
||||
return fileSystem.getFileChecksum(convertToDefaultPath(f), length);
|
||||
}
|
||||
|
||||
@Override public void setVerifyChecksum(boolean verifyChecksum) {
|
||||
fileSystem.setVerifyChecksum(verifyChecksum);
|
||||
}
|
||||
|
||||
@Override public void setWriteChecksum(boolean writeChecksum) {
|
||||
fileSystem.setWriteChecksum(writeChecksum);
|
||||
}
|
||||
|
||||
@Override public FsStatus getStatus() throws IOException {
|
||||
return fileSystem.getStatus();
|
||||
}
|
||||
|
||||
@Override public FsStatus getStatus(Path p) throws IOException {
|
||||
return fileSystem.getStatus(convertToDefaultPath(p));
|
||||
}
|
||||
|
||||
@Override public void setPermission(Path p, FsPermission permission) throws IOException {
|
||||
fileSystem.setPermission(convertToDefaultPath(p), permission);
|
||||
}
|
||||
|
||||
@Override public void setOwner(Path p, String username, String groupname) throws IOException {
|
||||
fileSystem.setOwner(convertToDefaultPath(p), username, groupname);
|
||||
}
|
||||
|
||||
@Override public void setTimes(Path p, long mtime, long atime) throws IOException {
|
||||
fileSystem.setTimes(convertToDefaultPath(p), mtime, atime);
|
||||
}
|
||||
|
||||
@Override public Path createSnapshot(Path path, String snapshotName) throws IOException {
|
||||
return convertToHoodiePath(
|
||||
fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
|
||||
}
|
||||
|
||||
@Override public void renameSnapshot(Path path, String snapshotOldName, String snapshotNewName)
|
||||
throws IOException {
|
||||
fileSystem.renameSnapshot(convertToDefaultPath(path), snapshotOldName, snapshotNewName);
|
||||
}
|
||||
|
||||
@Override public void deleteSnapshot(Path path, String snapshotName) throws IOException {
|
||||
fileSystem.deleteSnapshot(convertToDefaultPath(path), snapshotName);
|
||||
}
|
||||
|
||||
@Override public void modifyAclEntries(Path path, List<AclEntry> aclSpec) throws IOException {
|
||||
fileSystem.modifyAclEntries(convertToDefaultPath(path), aclSpec);
|
||||
}
|
||||
|
||||
@Override public void removeAclEntries(Path path, List<AclEntry> aclSpec) throws IOException {
|
||||
fileSystem.removeAclEntries(convertToDefaultPath(path), aclSpec);
|
||||
}
|
||||
|
||||
@Override public void removeDefaultAcl(Path path) throws IOException {
|
||||
fileSystem.removeDefaultAcl(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public void removeAcl(Path path) throws IOException {
|
||||
fileSystem.removeAcl(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
|
||||
fileSystem.setAcl(convertToDefaultPath(path), aclSpec);
|
||||
}
|
||||
|
||||
@Override public AclStatus getAclStatus(Path path) throws IOException {
|
||||
return fileSystem.getAclStatus(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public void setXAttr(Path path, String name, byte[] value) throws IOException {
|
||||
fileSystem.setXAttr(convertToDefaultPath(path), name, value);
|
||||
}
|
||||
|
||||
@Override public void setXAttr(Path path, String name, byte[] value, EnumSet<XAttrSetFlag> flag)
|
||||
throws IOException {
|
||||
fileSystem.setXAttr(convertToDefaultPath(path), name, value, flag);
|
||||
}
|
||||
|
||||
@Override public byte[] getXAttr(Path path, String name) throws IOException {
|
||||
return fileSystem.getXAttr(convertToDefaultPath(path), name);
|
||||
}
|
||||
|
||||
@Override public Map<String, byte[]> getXAttrs(Path path) throws IOException {
|
||||
return fileSystem.getXAttrs(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public Map<String, byte[]> getXAttrs(Path path, List<String> names)
|
||||
throws IOException {
|
||||
return fileSystem.getXAttrs(convertToDefaultPath(path), names);
|
||||
}
|
||||
|
||||
@Override public List<String> listXAttrs(Path path) throws IOException {
|
||||
return fileSystem.listXAttrs(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override public void removeXAttr(Path path, String name) throws IOException {
|
||||
fileSystem.removeXAttr(convertToDefaultPath(path), name);
|
||||
}
|
||||
|
||||
@Override public void setConf(Configuration conf) {
|
||||
// ignore this. we will set conf on init
|
||||
}
|
||||
|
||||
@Override public Configuration getConf() {
|
||||
return fileSystem.getConf();
|
||||
}
|
||||
|
||||
@Override public int hashCode() {
|
||||
return fileSystem.hashCode();
|
||||
}
|
||||
|
||||
@Override public boolean equals(Object obj) {
|
||||
return fileSystem.equals(obj);
|
||||
}
|
||||
|
||||
@Override public String toString() {
|
||||
return fileSystem.toString();
|
||||
}
|
||||
|
||||
public Path convertToHoodiePath(Path oldPath) {
|
||||
return convertPathWithScheme(oldPath, getHoodieScheme(fileSystem.getScheme()));
|
||||
}
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
}
|
||||
|
||||
private Path convertToDefaultPath(Path oldPath) {
|
||||
return convertPathWithScheme(oldPath, fileSystem.getScheme());
|
||||
}
|
||||
|
||||
private Path[] convertDefaults(Path[] psrcs) {
|
||||
Path[] psrcsNew = new Path[psrcs.length];
|
||||
for (int i = 0; i < psrcs.length; i++) {
|
||||
psrcsNew[i] = convertToDefaultPath(psrcs[i]);
|
||||
}
|
||||
return psrcsNew;
|
||||
}
|
||||
|
||||
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
|
||||
URI oldURI = oldPath.toUri();
|
||||
URI newURI;
|
||||
try {
|
||||
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
|
||||
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
|
||||
return new Path(newURI);
|
||||
} catch (URISyntaxException e) {
|
||||
// TODO - Better Exception handling
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHoodieScheme(String scheme) {
|
||||
String newScheme;
|
||||
if (SUPPORT_SCHEMES.contains(scheme)) {
|
||||
newScheme = HOODIE_SCHEME_PREFIX + scheme;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"BlockAlignedAvroParquetWriter does not support schema " + scheme);
|
||||
}
|
||||
return newScheme;
|
||||
}
|
||||
|
||||
public long getBytesWritten(Path file) {
|
||||
if (openStreams.containsKey(file.getName())) {
|
||||
return openStreams.get(file.getName()).getBytesWritten();
|
||||
}
|
||||
// When the file is first written, we do not have a track of it
|
||||
throw new IllegalArgumentException(file.toString()
|
||||
+ " does not have a open stream. Cannot get the bytes written on the stream");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,59 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes.
|
||||
* This gives a cheap way to check on the underlying file size.
|
||||
*/
|
||||
public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||
// A callback to call when the output stream is closed.
|
||||
private final Runnable closeCallback;
|
||||
// Keep track of the bytes written
|
||||
private final AtomicLong bytesWritten = new AtomicLong(0L);
|
||||
|
||||
public SizeAwareFSDataOutputStream(FSDataOutputStream out, Runnable closeCallback)
|
||||
throws IOException {
|
||||
super(out);
|
||||
this.closeCallback = closeCallback;
|
||||
}
|
||||
|
||||
@Override public synchronized void write(byte[] b, int off, int len) throws IOException {
|
||||
bytesWritten.addAndGet(len);
|
||||
super.write(b, off, len);
|
||||
}
|
||||
|
||||
@Override public void write(byte[] b) throws IOException {
|
||||
bytesWritten.addAndGet(b.length);
|
||||
super.write(b);
|
||||
}
|
||||
|
||||
@Override public void close() throws IOException {
|
||||
super.close();
|
||||
closeCallback.run();
|
||||
}
|
||||
|
||||
public long getBytesWritten() {
|
||||
return bytesWritten.get();
|
||||
}
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user