1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -15,180 +15,182 @@
~ limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hoodie</artifactId>
<groupId>com.uber.hoodie</groupId>
<version>0.4.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hoodie</artifactId>
<groupId>com.uber.hoodie</groupId>
<version>0.4.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>hoodie-client</artifactId>
<packaging>jar</packaging>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
</plugins>
<artifactId>hoodie-client</artifactId>
<packaging>jar</packaging>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
</plugins>
<resources>
<resource>
<directory>src/main/resources</directory>
</resource>
<resource>
<directory>src/test/resources</directory>
</resource>
</resources>
</build>
<resources>
<resource>
<directory>src/main/resources</directory>
</resource>
<resource>
<directory>src/test/resources</directory>
</resource>
</resources>
</build>
<dependencies>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<classifier>tests</classifier>
<!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<classifier>tests</classifier>
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-common</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.dropwizard.metrics</groupId>
<artifactId>metrics-graphite</artifactId>
</dependency>
<dependency>
<groupId>io.dropwizard.metrics</groupId>
<artifactId>metrics-core</artifactId>
</dependency>
<dependency>
<groupId>com.beust</groupId>
<artifactId>jcommander</artifactId>
<version>1.48</version>
</dependency>
<dependencies>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<classifier>tests</classifier>
<!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<classifier>tests</classifier>
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-common</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.dropwizard.metrics</groupId>
<artifactId>metrics-graphite</artifactId>
</dependency>
<dependency>
<groupId>io.dropwizard.metrics</groupId>
<artifactId>metrics-core</artifactId>
</dependency>
<dependency>
<groupId>com.beust</groupId>
<artifactId>jcommander</artifactId>
<version>1.48</version>
</dependency>
<!-- Parent dependencies -->
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</dependency>
<!-- Parent dependencies -->
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<exclusions>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<exclusions>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<version>1.10.19</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-hadoop-mr</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<version>1.10.19</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.uber.hoodie</groupId>
<artifactId>hoodie-hadoop-mr</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</dependencies>
</project>

View File

@@ -17,25 +17,19 @@
package com.uber.hoodie;
import com.google.common.base.Optional;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.Serializable;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.SparkConf;
@@ -46,136 +40,126 @@ import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import scala.Tuple2;
/**
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
*
*/
public class HoodieReadClient implements Serializable {
private static Logger logger = LogManager.getLogger(HoodieReadClient.class);
private static Logger logger = LogManager.getLogger(HoodieReadClient.class);
private transient final JavaSparkContext jsc;
private transient final JavaSparkContext jsc;
private transient final FileSystem fs;
/**
* TODO: We need to persist the index type into hoodie.properties and be able to access the
* index just with a simple basepath pointing to the dataset. Until, then just always assume a
* BloomIndex
*/
private transient final HoodieBloomIndex index;
private final HoodieTimeline commitTimeline;
private HoodieTable hoodieTable;
private transient Optional<SQLContext> sqlContextOpt;
private transient final FileSystem fs;
/**
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
* just with a simple basepath pointing to the dataset. Until, then just always assume a
* BloomIndex
*/
private transient final HoodieBloomIndex index;
private final HoodieTimeline commitTimeline;
private HoodieTable hoodieTable;
private transient Optional<SQLContext> sqlContextOpt;
/**
* @param basePath path to Hoodie dataset
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc;
this.fs = FSUtils.getFs();
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
/**
* @param basePath path to Hoodie dataset
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc;
this.fs = FSUtils.getFs();
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
}
/**
*
* @param jsc
* @param basePath
* @param sqlContext
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
this(jsc, basePath);
this.sqlContextOpt = Optional.of(sqlContext);
}
/**
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
*
* @return SparkConf object to be used to construct the SparkContext by caller
*/
public static SparkConf addHoodieSupport(SparkConf conf) {
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
return conf;
}
private void assertSqlContext() {
if (!sqlContextOpt.isPresent()) {
throw new IllegalStateException(
"SQLContext must be set, when performing dataframe operations");
}
}
/**
*
* @param jsc
* @param basePath
* @param sqlContext
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
this(jsc, basePath);
this.sqlContextOpt = Optional.of(sqlContext);
}
/**
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame
*
* @return a dataframe
*/
public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism)
throws Exception {
/**
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
*
* @return SparkConf object to be used to construct the SparkContext by caller
*/
public static SparkConf addHoodieSupport(SparkConf conf) {
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
return conf;
}
assertSqlContext();
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
index.fetchRecordLocation(hoodieKeys, hoodieTable);
List<String> paths = keyToFileRDD
.filter(keyFileTuple -> keyFileTuple._2().isPresent())
.map(keyFileTuple -> keyFileTuple._2().get())
.collect();
private void assertSqlContext() {
if (!sqlContextOpt.isPresent()) {
throw new IllegalStateException("SQLContext must be set, when performing dataframe operations");
}
}
// record locations might be same for multiple keys, so need a unique list
Set<String> uniquePaths = new HashSet<>(paths);
Dataset<Row> originalDF = sqlContextOpt.get().read()
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
StructType schema = originalDF.schema();
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
.mapToPair(row -> {
HoodieKey key = new HoodieKey(
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
return new Tuple2<>(key, row);
});
/**
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame
*
* @return a dataframe
*/
public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism)
throws Exception {
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism)
.map(tuple -> tuple._2()._1());
assertSqlContext();
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
index.fetchRecordLocation(hoodieKeys, hoodieTable);
List<String> paths = keyToFileRDD
.filter(keyFileTuple -> keyFileTuple._2().isPresent())
.map(keyFileTuple -> keyFileTuple._2().get())
.collect();
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
}
// record locations might be same for multiple keys, so need a unique list
Set<String> uniquePaths = new HashSet<>(paths);
Dataset<Row> originalDF = sqlContextOpt.get().read()
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
StructType schema = originalDF.schema();
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
.mapToPair(row -> {
HoodieKey key = new HoodieKey(
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
return new Tuple2<>(key, row);
});
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
* If the optional FullFilePath value is not present, then the key is not found. If the
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
* file
*/
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
}
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism)
.map(tuple -> tuple._2()._1());
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key,
* Optional[FullFilePath]] If the optional FullFilePath value is not present, then the key is
* not found. If the FullFilePath value is present, it is the path component (without scheme) of
* the URI underlying file
*/
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
}
/**
* Filter out HoodieRecords that already exists in the output folder. This is useful in
* deduplication.
*
* @param hoodieRecords Input RDD of Hoodie records.
* @return A subset of hoodieRecords RDD, with existing records filtered out.
*/
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
/**
* Filter out HoodieRecords that already exists in the output folder. This is useful in
* deduplication.
*
* @param hoodieRecords Input RDD of Hoodie records.
* @return A subset of hoodieRecords RDD, with existing records filtered out.
*/
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
}

View File

@@ -50,10 +50,21 @@ import com.uber.hoodie.func.BulkInsertMapFunction;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCommitArchiveLog;
import com.uber.hoodie.metrics.HoodieMetrics;
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
import com.uber.hoodie.table.WorkloadProfile;
import com.uber.hoodie.table.WorkloadStat;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -66,25 +77,12 @@ import org.apache.spark.storage.StorageLevel;
import scala.Option;
import scala.Tuple2;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then
* perform efficient mutations on a HDFS dataset [upsert()]
*
* Note that, at any given time, there can only be one Spark job performing
* these operatons on a Hoodie dataset.
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
* mutations on a HDFS dataset [upsert()]
*
* Note that, at any given time, there can only be one Spark job performing these operatons on a
* Hoodie dataset.
*/
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
@@ -102,7 +100,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @param clientConfig
* @throws Exception
*/
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception {
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig)
throws Exception {
this(jsc, clientConfig, false);
}
@@ -111,7 +110,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @param clientConfig
* @param rollbackInFlight
*/
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) {
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
boolean rollbackInFlight) {
this.fs = FSUtils.getFs();
this.jsc = jsc;
this.config = clientConfig;
@@ -121,7 +121,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
if (rollbackInFlight) {
rollbackInflightCommits();
}
}
}
/**
@@ -163,17 +163,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
throw (HoodieUpsertException) e;
}
throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e);
}
}
}
/**
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal
* writes.
*
* This implementation skips the index check and is able to leverage benefits such as
* small file handling/blocking alignment, as with upsert(), by profiling the workload
* This implementation skips the index check and is able to leverage benefits such as small file
* handling/blocking alignment, as with upsert(), by profiling the workload
*
* @param records HoodieRecords to insert
* @param records HoodieRecords to insert
* @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/
@@ -194,7 +194,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
throw e;
}
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
}
}
}
/**
@@ -206,11 +206,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* attempts to control the numbers of files with less memory compared to the {@link
* HoodieWriteClient#insert(JavaRDD, String)}
*
* @param records HoodieRecords to insert
* @param records HoodieRecords to insert
* @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
final String commitTime) {
return bulkInsert(records, commitTime, Option.empty());
}
@@ -221,16 +222,18 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
* attempts to control the numbers of files with less memory compared to the {@link
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own partitioner. If
* specified then it will be used for repartitioning records. See {@link UserDefinedBulkInsertPartitioner}.
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own
* partitioner. If specified then it will be used for repartitioning records. See {@link
* UserDefinedBulkInsertPartitioner}.
*
* @param records HoodieRecords to insert
* @param records HoodieRecords to insert
* @param commitTime Commit Time handle
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are
* inserted into hoodie.
* @param bulkInsertPartitioner If specified then it will be used to partition input records
* before they are inserted into hoodie.
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
final String commitTime,
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible
@@ -240,7 +243,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
combineOnCondition(config.shouldCombineBeforeInsert(), records,
config.getInsertShuffleParallelism());
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
if (bulkInsertPartitioner.isDefined()) {
@@ -259,20 +263,22 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}, true, config.getBulkInsertShuffleParallelism());
}
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table), true)
.flatMap(writeStatuses -> writeStatuses.iterator());
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table),
true)
.flatMap(writeStatuses -> writeStatuses.iterator());
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
} catch (Throwable e) {
if (e instanceof HoodieInsertException) {
throw e;
}
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e);
}
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime,
e);
}
}
private void commitOnAutoCommit(String commitTime, JavaRDD<WriteStatus> resultRDD) {
if(config.shouldAutoCommit()) {
if (config.shouldAutoCommit()) {
logger.info("Auto commit enabled: Committing " + commitTime);
boolean commitResult = commit(commitTime, resultRDD);
if (!commitResult) {
@@ -280,30 +286,28 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
} else {
logger.info("Auto commit disabled for " + commitTime);
}
}
}
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition,
JavaRDD<HoodieRecord<T>> records,
int parallelism) {
if(condition) {
JavaRDD<HoodieRecord<T>> records,
int parallelism) {
if (condition) {
return deduplicateRecords(records, parallelism);
}
return records;
}
/**
*
* Save the workload profile in an intermediate file (here re-using commit files)
* This is useful when performing rollback for MOR datasets. Only updates are recorded
* in the workload profile metadata since updates to log blocks are unknown across batches
* Inserts (which are new parquet files) are rolled back based on commit time.
* // TODO : Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
* @param profile
* @param commitTime
* @throws HoodieCommitException
* Save the workload profile in an intermediate file (here re-using commit files) This is useful
* when performing rollback for MOR datasets. Only updates are recorded in the workload profile
* metadata since updates to log blocks are unknown across batches Inserts (which are new parquet
* files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata
* file instead of using HoodieCommitMetadata
*/
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table, String commitTime) throws HoodieCommitException {
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
HoodieTable<T> table,
String commitTime) throws HoodieCommitException {
try {
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
profile.getPartitionPaths().stream().forEach(path -> {
@@ -319,16 +323,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
Optional<HoodieInstant> instant = activeTimeline.filterInflights().lastInstant();
activeTimeline.saveToInflight(instant.get(),
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch(IOException io) {
throw new HoodieCommitException("Failed to commit " + commitTime + " unable to save inflight metadata ", io);
}
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException io) {
throw new HoodieCommitException(
"Failed to commit " + commitTime + " unable to save inflight metadata ", io);
}
}
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
String commitTime,
HoodieTable<T> hoodieTable,
final boolean isUpsert) {
String commitTime,
HoodieTable<T> hoodieTable,
final boolean isUpsert) {
// Cache the tagged records, so we don't end up computing both
preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
@@ -344,29 +349,31 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile);
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords
.mapPartitionsWithIndex((partition, recordItr) -> {
if (isUpsert) {
return hoodieTable
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
} else {
return hoodieTable
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
}
}, true)
.flatMap(writeStatuses -> writeStatuses.iterator());
.mapPartitionsWithIndex((partition, recordItr) -> {
if (isUpsert) {
return hoodieTable
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
} else {
return hoodieTable
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
}
}, true)
.flatMap(writeStatuses -> writeStatuses.iterator());
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
}
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert,
WorkloadProfile profile) {
if (isUpsert) {
return table.getUpsertPartitioner(profile);
} else {
return table.getInsertPartitioner(profile);
}
}
}
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> table, String commitTime) {
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> table, String commitTime) {
// Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
// Trigger the insert and collect statuses
@@ -375,12 +382,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
return statuses;
}
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords,
Partitioner partitioner) {
return dedupedRecords
.mapToPair(record ->
new Tuple2<>(new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
.partitionBy(partitioner)
.map(tuple -> tuple._2());
.mapToPair(record ->
new Tuple2<>(
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())),
record))
.partitionBy(partitioner)
.map(tuple -> tuple._2());
}
/**
@@ -394,8 +404,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* Commit changes performed at the given commitTime marker
*/
public boolean commit(String commitTime,
JavaRDD<WriteStatus> writeStatuses,
Optional<HashMap<String, String>> extraMetadata) {
JavaRDD<WriteStatus> writeStatuses,
Optional<HashMap<String, String>> extraMetadata) {
logger.info("Commiting " + commitTime);
// Create a Hoodie table which encapsulated the commits and files visible
@@ -405,9 +415,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
.collect();
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
.collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
for (Tuple2<String, HoodieWriteStat> stat : stats) {
@@ -438,7 +448,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// We cannot have unbounded commit files. Archive commits if we have to archive
archiveLog.archiveIfRequired();
if(config.isAutoClean()) {
if (config.isAutoClean()) {
// Call clean to cleanup if there is anything to cleanup after the commit,
logger.info("Auto cleaning is enabled. Running cleaner now");
clean(commitTime);
@@ -465,12 +475,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
/**
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
* will never be rolledback or archived.
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
* rolledback or archived.
*
* This gives an option to rollback the state to the savepoint anytime.
* Savepoint needs to be manually created and deleted.
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
* manually created and deleted.
*
* Savepoint should be on a commit that could not have been cleaned.
*
@@ -491,12 +501,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
/**
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
* will never be rolledback or archived.
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
* rolledback or archived.
*
* This gives an option to rollback the state to the savepoint anytime.
* Savepoint needs to be manually created and deleted.
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
* manually created and deleted.
*
* Savepoint should be on a commit that could not have been cleaned.
*
@@ -510,9 +520,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if(!table.getCompletedCommitTimeline().containsInstant(commitInstant)) {
throw new HoodieSavepointException("Could not savepoint non-existing commit " + commitInstant);
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
commitTime);
if (!table.getCompletedCommitTimeline().containsInstant(commitInstant)) {
throw new HoodieSavepointException(
"Could not savepoint non-existing commit " + commitInstant);
}
try {
@@ -534,7 +546,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
+ lastCommitRetained);
Map<String, List<String>> latestFilesMap = jsc.parallelize(
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning()))
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
// Scan all partitions files with this commit time
logger.info("Collecting latest files in partition path " + partitionPath);
@@ -555,12 +568,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
return true;
} catch (IOException e) {
throw new HoodieSavepointException("Failed to savepoint " + commitTime, e);
}
}
}
/**
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be rolledback
* and cleaner may clean up data files.
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be
* rolledback and cleaner may clean up data files.
*
* @param savepointTime - delete the savepoint
* @return true if the savepoint was deleted successfully
@@ -586,9 +599,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
/**
* Rollback the state to the savepoint.
* WARNING: This rollsback recent commits and deleted data files. Queries accessing the files
* will mostly fail. This should be done during a downtime.
* Rollback the state to the savepoint. WARNING: This rollsback recent commits and deleted data
* files. Queries accessing the files will mostly fail. This should be done during a downtime.
*
* @param savepointTime - savepoint time to rollback to
* @return true if the savepoint was rollecback to successfully
@@ -616,7 +628,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Make sure the rollback was successful
Optional<HoodieInstant> lastInstant =
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants()
.lastInstant();
Preconditions.checkArgument(lastInstant.isPresent());
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
savepointTime + "is not the last commit after rolling back " + commitsToRollback
@@ -625,12 +638,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
/**
* Rollback the (inflight/committed) record changes with the given commit time.
* Three steps:
* (1) Atomically unpublish this commit
* (2) clean indexing data,
* (3) clean new generated parquet files.
* (4) Finally delete .commit or .inflight file,
* Rollback the (inflight/committed) record changes with the given commit time. Three steps: (1)
* Atomically unpublish this commit (2) clean indexing data, (3) clean new generated parquet
* files. (4) Finally delete .commit or .inflight file,
*/
public boolean rollback(final String commitTime) throws HoodieRollbackException {
rollback(Lists.newArrayList(commitTime));
@@ -638,7 +648,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
private void rollback(List<String> commits) {
if(commits.isEmpty()) {
if (commits.isEmpty()) {
logger.info("List of commits to rollback is empty");
return;
}
@@ -702,7 +712,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
Optional<Long> durationInMs = Optional.empty();
if (context != null) {
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum();
Long numFilesDeleted = stats.stream()
.mapToLong(stat -> stat.getSuccessDeleteFiles().size())
.sum();
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
}
HoodieRollbackMetadata rollbackMetadata =
@@ -722,7 +734,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
} catch (IOException e) {
throw new HoodieRollbackException("Failed to rollback " +
config.getBasePath() + " commits " + commits, e);
}
}
}
/**
@@ -733,9 +745,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage)
* based on the configurations and CleaningPolicy used. (typically files that no longer can be used
* by a running query can be cleaned)
* Clean up any stale/old files/data lying around (either on file storage or index storage) based
* on the configurations and CleaningPolicy used. (typically files that no longer can be used by a
* running query can be cleaned)
*/
public void clean() throws HoodieIOException {
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
@@ -743,11 +755,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage)
* based on the configurations and CleaningPolicy used. (typically files that no longer can be used
* by a running query can be cleaned)
* Clean up any stale/old files/data lying around (either on file storage or index storage) based
* on the configurations and CleaningPolicy used. (typically files that no longer can be used by a
* running query can be cleaned)
*/
private void clean(String startCleanTime) throws HoodieIOException {
private void clean(String startCleanTime) throws HoodieIOException {
try {
logger.info("Cleaner started");
final Timer.Context context = metrics.getCleanCtx();
@@ -788,7 +800,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
} catch (IOException e) {
throw new HoodieIOException("Failed to clean up after commit", e);
}
}
}
/**
@@ -811,30 +823,30 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
public static SparkConf registerClasses(SparkConf conf) {
conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
conf.registerKryoClasses(
new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
return conf;
}
/**
* Deduplicate Hoodie records, using the given deduplication funciton.
*/
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records, int parallelism) {
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
int parallelism) {
return records
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
.reduceByKey((rec1, rec2) -> {
@SuppressWarnings("unchecked")
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
// we cannot allow the user to change the key or partitionPath, since that will affect everything
// so pick it from one of the records.
return new HoodieRecord<T>(rec1.getKey(), reducedData);
}, parallelism)
.map(recordTuple -> recordTuple._2());
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
.reduceByKey((rec1, rec2) -> {
@SuppressWarnings("unchecked")
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
// we cannot allow the user to change the key or partitionPath, since that will affect everything
// so pick it from one of the records.
return new HoodieRecord<T>(rec1.getKey(), reducedData);
}, parallelism)
.map(recordTuple -> recordTuple._2());
}
/**
* Cleanup all inflight commits
*
* @throws IOException
*/
private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable

View File

@@ -19,7 +19,6 @@ package com.uber.hoodie;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieWriteStat;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
@@ -32,124 +31,130 @@ import java.util.Optional;
*/
public class WriteStatus implements Serializable {
private final HashMap<HoodieKey, Throwable> errors = new HashMap<>();
private final HashMap<HoodieKey, Throwable> errors = new HashMap<>();
private final List<HoodieRecord> writtenRecords = new ArrayList<>();
private final List<HoodieRecord> writtenRecords = new ArrayList<>();
private final List<HoodieRecord> failedRecords = new ArrayList<>();
private final List<HoodieRecord> failedRecords = new ArrayList<>();
private Throwable globalError = null;
private Throwable globalError = null;
private String fileId = null;
private String fileId = null;
private String partitionPath = null;
private String partitionPath = null;
private HoodieWriteStat stat = null;
private HoodieWriteStat stat = null;
private long totalRecords = 0;
private long totalErrorRecords = 0;
private long totalRecords = 0;
private long totalErrorRecords = 0;
/**
* Mark write as success, optionally using given parameters for the purpose of calculating
* some aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver.
*
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link HoodieRecord} before deflation.
*/
public void markSuccess(HoodieRecord record,
Optional<Map<String, String>> optionalRecordMetadata) {
writtenRecords.add(record);
totalRecords++;
}
/**
* Mark write as success, optionally using given parameters for the purpose of calculating some
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver.
*
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
* it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link
* HoodieRecord} before deflation.
*/
public void markSuccess(HoodieRecord record,
Optional<Map<String, String>> optionalRecordMetadata) {
writtenRecords.add(record);
totalRecords++;
}
/**
* Mark write as failed, optionally using given parameters for the purpose of calculating
* some aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver.
*
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link HoodieRecord} before deflation.
*/
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> optionalRecordMetadata) {
failedRecords.add(record);
errors.put(record.getKey(), t);
totalRecords++;
totalErrorRecords++;
}
/**
* Mark write as failed, optionally using given parameters for the purpose of calculating some
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver.
*
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
* it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link
* HoodieRecord} before deflation.
*/
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> optionalRecordMetadata) {
failedRecords.add(record);
errors.put(record.getKey(), t);
totalRecords++;
totalErrorRecords++;
}
public String getFileId() {
return fileId;
}
public String getFileId() {
return fileId;
}
public void setFileId(String fileId) {
this.fileId = fileId;
}
public void setFileId(String fileId) {
this.fileId = fileId;
}
public boolean hasErrors() {
return totalErrorRecords > 0;
}
public boolean hasErrors() {
return totalErrorRecords > 0;
}
public boolean isErrored(HoodieKey key) {
return errors.containsKey(key);
}
public boolean isErrored(HoodieKey key) {
return errors.containsKey(key);
}
public HashMap<HoodieKey, Throwable> getErrors() {
return errors;
}
public HashMap<HoodieKey, Throwable> getErrors() {
return errors;
}
public boolean hasGlobalError() {
return globalError != null;
}
public boolean hasGlobalError() {
return globalError != null;
}
public void setGlobalError(Throwable t) {
this.globalError = t;
}
public void setGlobalError(Throwable t) {
this.globalError = t;
}
public Throwable getGlobalError() {
return this.globalError;
}
public Throwable getGlobalError() {
return this.globalError;
}
public List<HoodieRecord> getWrittenRecords() {
return writtenRecords;
}
public List<HoodieRecord> getWrittenRecords() {
return writtenRecords;
}
public List<HoodieRecord> getFailedRecords() {
return failedRecords;
}
public List<HoodieRecord> getFailedRecords() {
return failedRecords;
}
public HoodieWriteStat getStat() {
return stat;
}
public HoodieWriteStat getStat() {
return stat;
}
public void setStat(HoodieWriteStat stat) {
this.stat = stat;
}
public void setStat(HoodieWriteStat stat) {
this.stat = stat;
}
public String getPartitionPath() {
return partitionPath;
}
public String getPartitionPath() {
return partitionPath;
}
public void setPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
}
public void setPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
}
public long getTotalRecords() {
return totalRecords;
}
public long getTotalRecords() {
return totalRecords;
}
public long getTotalErrorRecords() { return totalErrorRecords; }
public long getTotalErrorRecords() {
return totalErrorRecords;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WriteStatus {");
sb.append("fileId=").append(fileId);
sb.append(", globalError='").append(globalError).append('\'');
sb.append(", hasErrors='").append(hasErrors()).append('\'');
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WriteStatus {");
sb.append("fileId=").append(fileId);
sb.append(", globalError='").append(globalError).append('\'');
sb.append(", hasErrors='").append(hasErrors()).append('\'');
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
sb.append('}');
return sb.toString();
}
}

View File

@@ -17,33 +17,35 @@
package com.uber.hoodie.config;
import java.io.Serializable;
import java.util.Map;
import java.util.Properties;
/**
* Default Way to load Hoodie config through a java.util.Properties
*/
public class DefaultHoodieConfig implements Serializable {
protected final Properties props;
public DefaultHoodieConfig(Properties props) {
this.props = props;
}
public Properties getProps() {
return props;
}
protected final Properties props;
public static void setDefaultOnCondition(Properties props, boolean condition, String propName,
String defaultValue) {
if (condition) {
props.setProperty(propName, defaultValue);
}
}
public DefaultHoodieConfig(Properties props) {
this.props = props;
}
public static void setDefaultOnCondition(Properties props, boolean condition, DefaultHoodieConfig config) {
if (condition) {
props.putAll(config.getProps());
}
public Properties getProps() {
return props;
}
public static void setDefaultOnCondition(Properties props, boolean condition, String propName,
String defaultValue) {
if (condition) {
props.setProperty(propName, defaultValue);
}
}
public static void setDefaultOnCondition(Properties props, boolean condition,
DefaultHoodieConfig config) {
if (condition) {
props.putAll(config.getProps());
}
}
}

View File

@@ -19,231 +19,239 @@ package com.uber.hoodie.config;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import javax.annotation.concurrent.Immutable;
/**
* Compaction related config
*/
@Immutable
public class HoodieCompactionConfig extends DefaultHoodieConfig {
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
private static final String DEFAULT_CLEANER_POLICY =
HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
private static final String DEFAULT_AUTO_CLEAN = "true";
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
private static final String DEFAULT_CLEANER_POLICY =
HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
// Turn on inline compaction - after fw delta commits a inline compaction will be run
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
private static final String DEFAULT_INLINE_COMPACT = "true";
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
private static final String DEFAULT_AUTO_CLEAN = "true";
// Run a compaction every N delta commits
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10";
// Turn on inline compaction - after fw delta commits a inline compaction will be run
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
private static final String DEFAULT_INLINE_COMPACT = "true";
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
"hoodie.cleaner.fileversions.retained";
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
// Run a compaction every N delta commits
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10";
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24";
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
"hoodie.cleaner.fileversions.retained";
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits";
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128);
public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits";
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96);
// Upsert uses this file size to compact new data onto existing files..
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
// Turned off by default
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0);
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24";
public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits";
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128);
public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits";
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96);
// Upsert uses this file size to compact new data onto existing files..
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
// Turned off by default
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0);
/** Configs related to specific table types **/
// Number of inserts, that will be put each partition/bucket for writing
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
/**
* Configs related to specific table types
**/
// Number of inserts, that will be put each partition/bucket for writing
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
// Config to control whether we control insert split sizes automatically based on average record sizes
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
// its off by default
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false);
// Config to control whether we control insert split sizes automatically based on average record sizes
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
// its off by default
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false);
// This value is used as a guessimate for the record size, if we can't determine this from previous commits
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
// Used to determine how much more can be packed into a small file, before it exceeds the size limit.
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String.valueOf(1024);
// This value is used as a guessimate for the record size, if we can't determine this from previous commits
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
// Used to determine how much more can be packed into a small file, before it exceeds the size limit.
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String
.valueOf(1024);
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
// 500GB of target IO per compaction (both read and write)
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
// 500GB of target IO per compaction (both read and write)
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
// 200GB of target IO per compaction
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class.getName();
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
// 200GB of target IO per compaction
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class
.getName();
// used to merge records written to log file
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class";
// used to merge records written to log file
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class";
private HoodieCompactionConfig(Properties props) {
super(props);
private HoodieCompactionConfig(Properties props) {
super(props);
}
public static HoodieCompactionConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public static HoodieCompactionConfig.Builder newBuilder() {
return new Builder();
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withAutoClean(Boolean autoClean) {
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
return this;
}
public Builder withInlineCompaction(Boolean inlineCompaction) {
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
return this;
}
public Builder inlineCompactionEvery(int deltaCommits) {
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(deltaCommits));
return this;
}
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
props.setProperty(CLEANER_POLICY_PROP, policy.name());
return this;
}
public Builder retainFileVersions(int fileVersionsRetained) {
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP,
String.valueOf(fileVersionsRetained));
return this;
}
public Builder retainCommits(int commitsRetained) {
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
return this;
}
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
props.setProperty(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep));
props.setProperty(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep));
return this;
}
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
return this;
}
public Builder insertSplitSize(int insertSplitSize) {
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
return this;
}
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
return this;
}
public Builder approxRecordSize(int recordSizeEstimate) {
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
return this;
}
public Builder withCleanerParallelism(int cleanerParallelism) {
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
return this;
}
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
return this;
}
public Builder withPayloadClass(String payloadClassName) {
props.setProperty(PAYLOAD_CLASS, payloadClassName);
return this;
}
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
return this;
}
public HoodieCompactionConfig build() {
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP),
INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT);
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP),
CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY);
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP),
MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP);
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP),
MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP);
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM),
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS),
PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS);
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
Preconditions.checkArgument(
Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer
.parseInt(props.getProperty(MIN_COMMITS_TO_KEEP)));
return config;
}
public Builder withAutoClean(Boolean autoClean) {
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
return this;
}
public Builder withInlineCompaction(Boolean inlineCompaction) {
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
return this;
}
public Builder inlineCompactionEvery(int deltaCommits) {
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(deltaCommits));
return this;
}
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
props.setProperty(CLEANER_POLICY_PROP, policy.name());
return this;
}
public Builder retainFileVersions(int fileVersionsRetained) {
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP,
String.valueOf(fileVersionsRetained));
return this;
}
public Builder retainCommits(int commitsRetained) {
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
return this;
}
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
props.setProperty(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep));
props.setProperty(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep));
return this;
}
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
return this;
}
public Builder insertSplitSize(int insertSplitSize) {
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
return this;
}
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS,
String.valueOf(autoTuneInsertSplits));
return this;
}
public Builder approxRecordSize(int recordSizeEstimate) {
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
String.valueOf(recordSizeEstimate));
return this;
}
public Builder withCleanerParallelism(int cleanerParallelism) {
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
return this;
}
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
return this;
}
public Builder withPayloadClass(String payloadClassName) {
props.setProperty(PAYLOAD_CLASS, payloadClassName);
return this;
}
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP,
String.valueOf(targetIOPerCompactionInMB));
return this;
}
public HoodieCompactionConfig build() {
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP),
INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT);
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP),
CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY);
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP),
MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP);
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP),
MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP);
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM),
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS),
PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS);
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
Preconditions.checkArgument(
Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer
.parseInt(props.getProperty(MIN_COMMITS_TO_KEEP)));
return config;
}
}
}

View File

@@ -16,14 +16,12 @@
package com.uber.hoodie.config;
import com.google.common.base.Preconditions;
import com.uber.hoodie.index.HoodieIndex;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import javax.annotation.concurrent.Immutable;
/**
* Indexing related config
@@ -31,123 +29,124 @@ import java.util.Properties;
@Immutable
public class HoodieIndexConfig extends DefaultHoodieConfig {
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
// ***** Bloom Index configs *****
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
// ***** Bloom Index configs *****
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
// ***** HBase Index Configs *****
public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
// ***** HBase Index Configs *****
public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
// ***** Bucketed Index Configs *****
public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets";
// ***** Bucketed Index Configs *****
public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets";
private HoodieIndexConfig(Properties props) {
super(props);
private HoodieIndexConfig(Properties props) {
super(props);
}
public static HoodieIndexConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public static HoodieIndexConfig.Builder newBuilder() {
return new Builder();
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withIndexType(HoodieIndex.IndexType indexType) {
props.setProperty(INDEX_TYPE_PROP, indexType.name());
return this;
}
public Builder bloomFilterNumEntries(int numEntries) {
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
return this;
}
public Builder bloomFilterFPP(double fpp) {
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
return this;
}
public Builder hbaseZkQuorum(String zkString) {
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
return this;
}
public Builder hbaseZkPort(int port) {
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
return this;
}
public Builder hbaseTableName(String tableName) {
props.setProperty(HBASE_TABLENAME_PROP, tableName);
return this;
}
public Builder bloomIndexParallelism(int parallelism) {
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
return this;
}
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
return this;
}
public Builder bloomIndexUseCaching(boolean useCaching) {
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
return this;
}
public Builder numBucketsPerPartition(int numBuckets) {
props.setProperty(BUCKETED_INDEX_NUM_BUCKETS_PROP, String.valueOf(numBuckets));
return this;
}
public HoodieIndexConfig build() {
HoodieIndexConfig config = new HoodieIndexConfig(props);
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP),
INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP),
BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
return config;
}
public Builder withIndexType(HoodieIndex.IndexType indexType) {
props.setProperty(INDEX_TYPE_PROP, indexType.name());
return this;
}
public Builder bloomFilterNumEntries(int numEntries) {
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
return this;
}
public Builder bloomFilterFPP(double fpp) {
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
return this;
}
public Builder hbaseZkQuorum(String zkString) {
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
return this;
}
public Builder hbaseZkPort(int port) {
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
return this;
}
public Builder hbaseTableName(String tableName) {
props.setProperty(HBASE_TABLENAME_PROP, tableName);
return this;
}
public Builder bloomIndexParallelism(int parallelism) {
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
return this;
}
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
return this;
}
public Builder bloomIndexUseCaching(boolean useCaching) {
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
return this;
}
public Builder numBucketsPerPartition(int numBuckets) {
props.setProperty(BUCKETED_INDEX_NUM_BUCKETS_PROP, String.valueOf(numBuckets));
return this;
}
public HoodieIndexConfig build() {
HoodieIndexConfig config = new HoodieIndexConfig(props);
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP),
INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP),
BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
return config;
}
}
}

View File

@@ -17,12 +17,11 @@
package com.uber.hoodie.config;
import com.uber.hoodie.metrics.MetricsReporterType;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import javax.annotation.concurrent.Immutable;
/**
* Fetch the configurations used by the Metrics system.
@@ -30,89 +29,90 @@ import java.util.Properties;
@Immutable
public class HoodieMetricsConfig extends DefaultHoodieConfig {
public final static String METRIC_PREFIX = "hoodie.metrics";
public final static String METRICS_ON = METRIC_PREFIX + ".on";
public final static boolean DEFAULT_METRICS_ON = false;
public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE =
MetricsReporterType.GRAPHITE;
public final static String METRIC_PREFIX = "hoodie.metrics";
public final static String METRICS_ON = METRIC_PREFIX + ".on";
public final static boolean DEFAULT_METRICS_ON = false;
public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE =
MetricsReporterType.GRAPHITE;
// Graphite
public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
// Graphite
public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
private HoodieMetricsConfig(Properties props) {
super(props);
private HoodieMetricsConfig(Properties props) {
super(props);
}
public static HoodieMetricsConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public static HoodieMetricsConfig.Builder newBuilder() {
return new Builder();
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder on(boolean metricsOn) {
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
return this;
}
public Builder withReporterType(String reporterType) {
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
return this;
}
public Builder toGraphiteHost(String host) {
props.setProperty(GRAPHITE_SERVER_HOST, host);
return this;
}
public Builder onGraphitePort(int port) {
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
return this;
}
public Builder usePrefix(String prefix) {
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
return this;
}
public HoodieMetricsConfig build() {
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
String.valueOf(DEFAULT_METRICS_ON));
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE),
METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name());
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST),
GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST);
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
return config;
}
public Builder on(boolean metricsOn) {
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
return this;
}
public Builder withReporterType(String reporterType) {
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
return this;
}
public Builder toGraphiteHost(String host) {
props.setProperty(GRAPHITE_SERVER_HOST, host);
return this;
}
public Builder onGraphitePort(int port) {
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
return this;
}
public Builder usePrefix(String prefix) {
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
return this;
}
public HoodieMetricsConfig build() {
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
String.valueOf(DEFAULT_METRICS_ON));
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE),
METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name());
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST),
GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST);
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
return config;
}
}
}

View File

@@ -16,75 +16,77 @@
package com.uber.hoodie.config;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import javax.annotation.concurrent.Immutable;
/**
* Storage related config
*/
@Immutable
public class HoodieStorageConfig extends DefaultHoodieConfig {
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
private HoodieStorageConfig(Properties props) {
super(props);
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
private HoodieStorageConfig(Properties props) {
super(props);
}
public static HoodieStorageConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public static HoodieStorageConfig.Builder newBuilder() {
return new Builder();
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder limitFileSize(int maxFileSize) {
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
return this;
}
public Builder parquetBlockSize(int blockSize) {
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
return this;
}
public Builder parquetPageSize(int pageSize) {
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
return this;
}
public HoodieStorageConfig build() {
HoodieStorageConfig config = new HoodieStorageConfig(props);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
return config;
}
public Builder limitFileSize(int maxFileSize) {
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
return this;
}
public Builder parquetBlockSize(int blockSize) {
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
return this;
}
public Builder parquetPageSize(int pageSize) {
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
return this;
}
public HoodieStorageConfig build() {
HoodieStorageConfig config = new HoodieStorageConfig(props);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
return config;
}
}
}

View File

@@ -24,395 +24,401 @@ import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.metrics.MetricsReporterType;
import org.apache.spark.storage.StorageLevel;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
import java.util.Properties;
import javax.annotation.concurrent.Immutable;
import org.apache.spark.storage.StorageLevel;
/**
* Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient}
*/
@Immutable
public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String BASE_PATH_PROP = "hoodie.base.path";
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
public static final String TABLE_NAME = "hoodie.table.name";
private static final String DEFAULT_PARALLELISM = "200";
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
private static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
private static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
private HoodieWriteConfig(Properties props) {
super(props);
}
private static final String BASE_PATH_PROP = "hoodie.base.path";
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
public static final String TABLE_NAME = "hoodie.table.name";
private static final String DEFAULT_PARALLELISM = "200";
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
private static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
private static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
/**
* base properties
**/
public String getBasePath() {
return props.getProperty(BASE_PATH_PROP);
}
private HoodieWriteConfig(Properties props) {
super(props);
}
public String getSchema() {
return props.getProperty(AVRO_SCHEMA);
}
/**
* base properties
**/
public String getBasePath() {
return props.getProperty(BASE_PATH_PROP);
}
public String getTableName() {
return props.getProperty(TABLE_NAME);
}
public String getSchema() {
return props.getProperty(AVRO_SCHEMA);
}
public Boolean shouldAutoCommit() {
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
}
public String getTableName() {
return props.getProperty(TABLE_NAME);
}
public Boolean shouldAssumeDatePartitioning() {
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
}
public Boolean shouldAutoCommit() {
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
}
public int getBulkInsertShuffleParallelism() {
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
}
public Boolean shouldAssumeDatePartitioning() {
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
}
public int getInsertShuffleParallelism() {
return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
}
public int getBulkInsertShuffleParallelism() {
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
}
public int getUpsertShuffleParallelism() {
return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
}
public int getInsertShuffleParallelism() {
return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
}
public boolean shouldCombineBeforeInsert() {
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
}
public int getUpsertShuffleParallelism() {
return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
}
public boolean shouldCombineBeforeUpsert() {
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
}
public boolean shouldCombineBeforeInsert() {
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
}
public StorageLevel getWriteStatusStorageLevel() {
return StorageLevel.fromString(props.getProperty(WRITE_STATUS_STORAGE_LEVEL));
}
public boolean shouldCombineBeforeUpsert() {
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
}
public String getWriteStatusClassName() {
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
}
public StorageLevel getWriteStatusStorageLevel() {
return StorageLevel.fromString(props.getProperty(WRITE_STATUS_STORAGE_LEVEL));
}
/**
* compaction properties
**/
public HoodieCleaningPolicy getCleanerPolicy() {
return HoodieCleaningPolicy
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
}
public String getWriteStatusClassName() {
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
}
public int getCleanerFileVersionsRetained() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
}
/**
* compaction properties
**/
public HoodieCleaningPolicy getCleanerPolicy() {
return HoodieCleaningPolicy
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
}
public int getCleanerCommitsRetained() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
}
public int getCleanerFileVersionsRetained() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
}
public int getMaxCommitsToKeep() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP));
}
public int getCleanerCommitsRetained() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
}
public int getMinCommitsToKeep() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP));
}
public int getMaxCommitsToKeep() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP));
}
public int getParquetSmallFileLimit() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
}
public int getMinCommitsToKeep() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP));
}
public int getCopyOnWriteInsertSplitSize() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
}
public int getParquetSmallFileLimit() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
}
public int getCopyOnWriteRecordSizeEstimate() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
}
public int getCopyOnWriteInsertSplitSize() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
}
public boolean shouldAutoTuneInsertSplits() {
return Boolean.parseBoolean(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
}
public int getCopyOnWriteRecordSizeEstimate() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
}
public int getCleanerParallelism() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
}
public boolean shouldAutoTuneInsertSplits() {
return Boolean.parseBoolean(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
}
public boolean isAutoClean() {
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
}
public int getCleanerParallelism() {
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
}
public boolean isInlineCompaction() {
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
}
public boolean isAutoClean() {
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
}
public int getInlineCompactDeltaCommitMax() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
}
public boolean isInlineCompaction() {
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
}
public CompactionStrategy getCompactionStrategy() {
return ReflectionUtils.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
}
public int getInlineCompactDeltaCommitMax() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
}
public Long getTargetIOPerCompactionInMB() {
return Long.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
}
public CompactionStrategy getCompactionStrategy() {
return ReflectionUtils
.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
}
/**
* index properties
**/
public HoodieIndex.IndexType getIndexType() {
return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
}
public Long getTargetIOPerCompactionInMB() {
return Long
.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
}
public int getBloomFilterNumEntries() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
}
/**
* index properties
**/
public HoodieIndex.IndexType getIndexType() {
return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
}
public double getBloomFilterFPP() {
return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
}
public int getBloomFilterNumEntries() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
}
public String getHbaseZkQuorum() {
return props.getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
}
public double getBloomFilterFPP() {
return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
}
public int getHbaseZkPort() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP));
}
public String getHbaseZkQuorum() {
return props.getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
}
public String getHbaseTableName() {
return props.getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
}
public int getHbaseZkPort() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP));
}
public int getBloomIndexParallelism() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
}
public String getHbaseTableName() {
return props.getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
}
public boolean getBloomIndexPruneByRanges() {
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
}
public int getBloomIndexParallelism() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
}
public boolean getBloomIndexUseCaching() {
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
}
public boolean getBloomIndexPruneByRanges() {
return Boolean
.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
}
public int getNumBucketsPerPartition() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BUCKETED_INDEX_NUM_BUCKETS_PROP));
}
public boolean getBloomIndexUseCaching() {
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
}
/**
* storage properties
**/
public int getParquetMaxFileSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
}
public int getNumBucketsPerPartition() {
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BUCKETED_INDEX_NUM_BUCKETS_PROP));
}
public int getParquetBlockSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
}
/**
* storage properties
**/
public int getParquetMaxFileSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
}
public int getParquetPageSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
}
public int getParquetBlockSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
}
/**
* metrics properties
**/
public boolean isMetricsOn() {
return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
}
public int getParquetPageSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
}
public MetricsReporterType getMetricsReporterType() {
return MetricsReporterType
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
}
/**
* metrics properties
**/
public boolean isMetricsOn() {
return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
}
public String getGraphiteServerHost() {
return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
}
public MetricsReporterType getMetricsReporterType() {
return MetricsReporterType
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
}
public int getGraphiteServerPort() {
return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
}
public String getGraphiteServerHost() {
return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
}
public String getGraphiteMetricPrefix() {
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
}
public int getGraphiteServerPort() {
return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
}
public static HoodieWriteConfig.Builder newBuilder() {
return new Builder();
}
public String getGraphiteMetricPrefix() {
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
}
public static HoodieWriteConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
private boolean isIndexConfigSet = false;
private boolean isStorageConfigSet = false;
private boolean isCompactionConfigSet = false;
private boolean isMetricsConfigSet = false;
private boolean isAutoCommit = true;
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
private final Properties props = new Properties();
private boolean isIndexConfigSet = false;
private boolean isStorageConfigSet = false;
private boolean isCompactionConfigSet = false;
private boolean isMetricsConfigSet = false;
private boolean isAutoCommit = true;
public Builder fromInputStream(InputStream inputStream) throws IOException {
try {
this.props.load(inputStream);
return this;
} finally {
inputStream.close();
}
}
public Builder withProps(Map kvprops) {
props.putAll(kvprops);
return this;
}
public Builder withPath(String basePath) {
props.setProperty(BASE_PATH_PROP, basePath);
return this;
}
public Builder withSchema(String schemaStr) {
props.setProperty(AVRO_SCHEMA, schemaStr);
return this;
}
public Builder forTable(String tableName) {
props.setProperty(TABLE_NAME, tableName);
return this;
}
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
return this;
}
public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
return this;
}
public Builder combineInput(boolean onInsert, boolean onUpsert) {
props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
return this;
}
public Builder withWriteStatusStorageLevel(String level) {
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
return this;
}
public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
props.putAll(indexConfig.getProps());
isIndexConfigSet = true;
return this;
}
public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
props.putAll(storageConfig.getProps());
isStorageConfigSet = true;
return this;
}
public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
props.putAll(compactionConfig.getProps());
isCompactionConfigSet = true;
return this;
}
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
props.putAll(metricsConfig.getProps());
isMetricsConfigSet = true;
return this;
}
public Builder withAutoCommit(boolean autoCommit) {
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
return this;
}
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
return this;
}
public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
return this;
}
public HoodieWriteConfig build() {
HoodieWriteConfig config = new HoodieWriteConfig(props);
// Check for mandatory properties
Preconditions.checkArgument(config.getBasePath() != null);
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
// Make sure the props is propagated
setDefaultOnCondition(props, !isIndexConfigSet,
HoodieIndexConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isStorageConfigSet,
HoodieStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isCompactionConfigSet,
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMetricsConfigSet,
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
return config;
}
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromInputStream(InputStream inputStream) throws IOException {
try {
this.props.load(inputStream);
return this;
} finally {
inputStream.close();
}
}
public Builder withProps(Map kvprops) {
props.putAll(kvprops);
return this;
}
public Builder withPath(String basePath) {
props.setProperty(BASE_PATH_PROP, basePath);
return this;
}
public Builder withSchema(String schemaStr) {
props.setProperty(AVRO_SCHEMA, schemaStr);
return this;
}
public Builder forTable(String tableName) {
props.setProperty(TABLE_NAME, tableName);
return this;
}
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
return this;
}
public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
return this;
}
public Builder combineInput(boolean onInsert, boolean onUpsert) {
props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
return this;
}
public Builder withWriteStatusStorageLevel(String level) {
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
return this;
}
public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
props.putAll(indexConfig.getProps());
isIndexConfigSet = true;
return this;
}
public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
props.putAll(storageConfig.getProps());
isStorageConfigSet = true;
return this;
}
public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
props.putAll(compactionConfig.getProps());
isCompactionConfigSet = true;
return this;
}
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
props.putAll(metricsConfig.getProps());
isMetricsConfigSet = true;
return this;
}
public Builder withAutoCommit(boolean autoCommit) {
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
return this;
}
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP,
String.valueOf(assumeDatePartitioning));
return this;
}
public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
return this;
}
public HoodieWriteConfig build() {
HoodieWriteConfig config = new HoodieWriteConfig(props);
// Check for mandatory properties
Preconditions.checkArgument(config.getBasePath() != null);
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
BULKINSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
// Make sure the props is propagated
setDefaultOnCondition(props, !isIndexConfigSet,
HoodieIndexConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isStorageConfigSet,
HoodieStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isCompactionConfigSet,
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMetricsConfigSet,
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
return config;
}
}
}

View File

@@ -17,16 +17,16 @@
package com.uber.hoodie.exception;
/**
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta commit
* </p>
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta
* commit </p>
*/
public class HoodieAppendException extends HoodieException {
public HoodieAppendException(String msg, Throwable e) {
super(msg, e);
}
public HoodieAppendException(String msg) {
super(msg);
}
public HoodieAppendException(String msg, Throwable e) {
super(msg, e);
}
public HoodieAppendException(String msg) {
super(msg);
}
}

View File

@@ -17,16 +17,16 @@
package com.uber.hoodie.exception;
/**
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
* </p>
*/
public class HoodieCommitException extends HoodieException {
public HoodieCommitException(String msg) {
super(msg);
}
public HoodieCommitException(String msg, Throwable e) {
super(msg, e);
}
public HoodieCommitException(String msg) {
super(msg);
}
public HoodieCommitException(String msg, Throwable e) {
super(msg, e);
}
}

View File

@@ -17,6 +17,7 @@
package com.uber.hoodie.exception;
public class HoodieCompactionException extends HoodieException {
public HoodieCompactionException(String msg) {
super(msg);
}

View File

@@ -18,18 +18,17 @@ package com.uber.hoodie.exception;
/**
* <p>
* Exception thrown when dependent system is not available
* </p>
* <p> Exception thrown when dependent system is not available </p>
*/
public class HoodieDependentSystemUnavailableException extends HoodieException {
public static final String HBASE = "HBASE";
public HoodieDependentSystemUnavailableException(String system, String connectURL) {
super(getLogMessage(system, connectURL));
}
public static final String HBASE = "HBASE";
private static String getLogMessage(String system, String connectURL) {
return "System " + system + " unavailable. Tried to connect to " + connectURL;
}
public HoodieDependentSystemUnavailableException(String system, String connectURL) {
super(getLogMessage(system, connectURL));
}
private static String getLogMessage(String system, String connectURL) {
return "System " + system + " unavailable. Tried to connect to " + connectURL;
}
}

View File

@@ -16,15 +16,13 @@
package com.uber.hoodie.exception;
import java.io.IOException;
/**
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk insert
* </p>
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk
* insert </p>
*/
public class HoodieInsertException extends HoodieException {
public HoodieInsertException(String msg, Throwable e) {
super(msg, e);
}
public HoodieInsertException(String msg, Throwable e) {
super(msg, e);
}
}

View File

@@ -18,11 +18,11 @@ package com.uber.hoodie.exception;
public class HoodieRollbackException extends HoodieException {
public HoodieRollbackException(String msg, Throwable e) {
super(msg, e);
}
public HoodieRollbackException(String msg, Throwable e) {
super(msg, e);
}
public HoodieRollbackException(String msg) {
super(msg);
}
public HoodieRollbackException(String msg) {
super(msg);
}
}

View File

@@ -18,11 +18,11 @@ package com.uber.hoodie.exception;
public class HoodieSavepointException extends HoodieException {
public HoodieSavepointException(String msg, Throwable e) {
super(msg, e);
}
public HoodieSavepointException(String msg, Throwable e) {
super(msg, e);
}
public HoodieSavepointException(String msg) {
super(msg);
}
public HoodieSavepointException(String msg) {
super(msg);
}
}

View File

@@ -17,16 +17,16 @@
package com.uber.hoodie.exception;
/**
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a incremental upsert
* </p>
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a
* incremental upsert </p>
*/
public class HoodieUpsertException extends HoodieException {
public HoodieUpsertException(String msg, Throwable e) {
super(msg, e);
}
public class HoodieUpsertException extends HoodieException {
public HoodieUpsertException(String msg) {
super(msg);
}
public HoodieUpsertException(String msg, Throwable e) {
super(msg, e);
}
public HoodieUpsertException(String msg) {
super(msg);
}
}

View File

@@ -16,16 +16,14 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.spark.api.java.function.Function2;
import java.util.Iterator;
import java.util.List;
import org.apache.spark.api.java.function.Function2;
/**
@@ -34,20 +32,21 @@ import java.util.List;
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
private String commitTime;
private HoodieWriteConfig config;
private HoodieTable<T> hoodieTable;
private String commitTime;
private HoodieWriteConfig config;
private HoodieTable<T> hoodieTable;
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.hoodieTable = hoodieTable;
}
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.hoodieTable = hoodieTable;
}
@Override
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr)
throws Exception {
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
}
@Override
public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> sortedRecordItr)
throws Exception {
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
}
}

View File

@@ -16,99 +16,101 @@
package com.uber.hoodie.func;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.io.HoodieIOHandle;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieIOHandle;
import com.uber.hoodie.table.HoodieTable;
import org.apache.spark.TaskContext;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import org.apache.spark.TaskContext;
/**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath,
* into new files.
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* files.
*/
public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
public class LazyInsertIterable<T extends HoodieRecordPayload> extends
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
private final HoodieWriteConfig hoodieConfig;
private final String commitTime;
private final HoodieTable<T> hoodieTable;
private Set<String> partitionsCleaned;
private HoodieCreateHandle handle;
private final HoodieWriteConfig hoodieConfig;
private final String commitTime;
private final HoodieTable<T> hoodieTable;
private Set<String> partitionsCleaned;
private HoodieCreateHandle handle;
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) {
super(sortedRecordItr);
this.partitionsCleaned = new HashSet<>();
this.hoodieConfig = config;
this.commitTime = commitTime;
this.hoodieTable = hoodieTable;
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) {
super(sortedRecordItr);
this.partitionsCleaned = new HashSet<>();
this.hoodieConfig = config;
this.commitTime = commitTime;
this.hoodieTable = hoodieTable;
}
@Override
protected void start() {
}
@Override
protected List<WriteStatus> computeNext() {
List<WriteStatus> statuses = new ArrayList<>();
while (inputItr.hasNext()) {
HoodieRecord record = inputItr.next();
// clean up any partial failures
if (!partitionsCleaned.contains(record.getPartitionPath())) {
// This insert task could fail multiple times, but Spark will faithfully retry with
// the same data again. Thus, before we open any files under a given partition, we
// first delete any files in the same partitionPath written by same Spark partition
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
commitTime,
record.getPartitionPath(),
TaskContext.getPartitionId());
partitionsCleaned.add(record.getPartitionPath());
}
// lazily initialize the handle, for the first time
if (handle == null) {
handle =
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
record.getPartitionPath());
}
if (handle.canWrite(record)) {
// write the record, if the handle has capacity
handle.write(record);
} else {
// handle is full.
statuses.add(handle.close());
// Need to handle the rejected record & open new handle
handle =
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
record.getPartitionPath());
handle.write(record); // we should be able to write 1 record.
break;
}
}
@Override protected void start() {
// If we exited out, because we ran out of records, just close the pending handle.
if (!inputItr.hasNext()) {
if (handle != null) {
statuses.add(handle.close());
}
}
assert statuses.size() > 0; // should never return empty statuses
return statuses;
}
@Override protected List<WriteStatus> computeNext() {
List<WriteStatus> statuses = new ArrayList<>();
@Override
protected void end() {
while (inputItr.hasNext()) {
HoodieRecord record = inputItr.next();
// clean up any partial failures
if (!partitionsCleaned.contains(record.getPartitionPath())) {
// This insert task could fail multiple times, but Spark will faithfully retry with
// the same data again. Thus, before we open any files under a given partition, we
// first delete any files in the same partitionPath written by same Spark partition
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
commitTime,
record.getPartitionPath(),
TaskContext.getPartitionId());
partitionsCleaned.add(record.getPartitionPath());
}
// lazily initialize the handle, for the first time
if (handle == null) {
handle =
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
record.getPartitionPath());
}
if (handle.canWrite(record)) {
// write the record, if the handle has capacity
handle.write(record);
} else {
// handle is full.
statuses.add(handle.close());
// Need to handle the rejected record & open new handle
handle =
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
record.getPartitionPath());
handle.write(record); // we should be able to write 1 record.
break;
}
}
// If we exited out, because we ran out of records, just close the pending handle.
if (!inputItr.hasNext()) {
if (handle != null) {
statuses.add(handle.close());
}
}
assert statuses.size() > 0; // should never return empty statuses
return statuses;
}
@Override protected void end() {
}
}
}

View File

@@ -31,98 +31,99 @@ import java.util.Iterator;
* responsible for calling inputIterator.next() and doing the processing in computeNext()
*/
public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterator<O> {
protected Iterator<I> inputItr = null;
private boolean consumed = false;
private boolean startCalled = false;
private boolean endCalled = false;
public LazyIterableIterator(Iterator<I> in) {
inputItr = in;
protected Iterator<I> inputItr = null;
private boolean consumed = false;
private boolean startCalled = false;
private boolean endCalled = false;
public LazyIterableIterator(Iterator<I> in) {
inputItr = in;
}
/**
* Called once, before any elements are processed
*/
protected abstract void start();
/**
* Block computation to be overwritten by sub classes.
*/
protected abstract O computeNext();
/**
* Called once, after all elements are processed.
*/
protected abstract void end();
//////////////////
// iterable implementation
private void invokeStartIfNeeded() {
if (!startCalled) {
startCalled = true;
try {
start();
} catch (Exception e) {
throw new RuntimeException("Error in start()");
}
}
}
private void invokeEndIfNeeded() {
// make the calls out to begin() & end()
if (!endCalled) {
endCalled = true;
// if we are out of elements, and end has not been called yet
try {
end();
} catch (Exception e) {
throw new RuntimeException("Error in end()");
}
}
}
@Override
public Iterator<O> iterator() {
//check for consumed inputItr
if (consumed) {
throw new RuntimeException("Invalid repeated inputItr consumption.");
}
/**
* Called once, before any elements are processed
*/
protected abstract void start();
//hand out self as inputItr exactly once (note: do not hand out the input
//inputItr since it is consumed by the self inputItr implementation)
consumed = true;
return this;
}
/**
* Block computation to be overwritten by sub classes.
*/
protected abstract O computeNext();
//////////////////
// inputItr implementation
/**
* Called once, after all elements are processed.
*/
protected abstract void end();
//////////////////
// iterable implementation
private void invokeStartIfNeeded() {
if (!startCalled) {
startCalled = true;
try {
start();
} catch (Exception e) {
throw new RuntimeException("Error in start()");
}
}
@Override
public boolean hasNext() {
boolean ret = inputItr.hasNext();
// make sure, there is exactly one call to start()
invokeStartIfNeeded();
if (!ret) {
// if we are out of elements, and end has not been called yet
invokeEndIfNeeded();
}
private void invokeEndIfNeeded() {
// make the calls out to begin() & end()
if (!endCalled) {
endCalled = true;
// if we are out of elements, and end has not been called yet
try {
end();
} catch (Exception e) {
throw new RuntimeException("Error in end()");
}
}
return ret;
}
@Override
public O next() {
try {
return computeNext();
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
@Override
public Iterator<O> iterator() {
//check for consumed inputItr
if (consumed)
throw new RuntimeException("Invalid repeated inputItr consumption.");
//hand out self as inputItr exactly once (note: do not hand out the input
//inputItr since it is consumed by the self inputItr implementation)
consumed = true;
return this;
}
//////////////////
// inputItr implementation
@Override
public boolean hasNext() {
boolean ret = inputItr.hasNext();
// make sure, there is exactly one call to start()
invokeStartIfNeeded();
if (!ret) {
// if we are out of elements, and end has not been called yet
invokeEndIfNeeded();
}
return ret;
}
@Override
public O next() {
try {
return computeNext();
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
@Override
public void remove() {
throw new RuntimeException("Unsupported remove operation.");
}
@Override
public void remove() {
throw new RuntimeException("Unsupported remove operation.");
}
}

View File

@@ -17,118 +17,108 @@
package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.bucketed.BucketedIndex;
import com.uber.hoodie.index.hbase.HBaseIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.Serializable;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.Serializable;
/**
* Base class for different types of indexes to determine the mapping from uuid
*
*/
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
protected transient JavaSparkContext jsc = null;
public enum IndexType {
HBASE,
INMEMORY,
BLOOM,
BUCKETED
}
protected final HoodieWriteConfig config;
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config;
this.jsc = jsc;
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
* If the optional FullFilePath value is not present, then the key is not found. If the FullFilePath
* value is present, it is the path component (without scheme) of the URI underlying file
*
* @param hoodieKeys
* @param table
* @return
*/
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present)
*/
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Extracts the location of written records, and updates the index.
*
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.
*/
public abstract boolean rollbackCommit(String commitTime);
/**
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`.
* Such an implementation is able to obtain the same mapping, for two hoodie keys with same `recordKey`
* but different `partitionPath`
*
* @return whether or not, the index implementation is global in nature
*/
public abstract boolean isGlobal();
/**
* This is used by storage to determine, if its safe to send inserts, straight to the log,
* i.e having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file.
*
* @return Returns true/false depending on whether the impl has this capability
*/
public abstract boolean canIndexLogFiles();
/**
*
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
* updates the index as well. This is used by storage, to save memory footprint in
* certain cases.
*
* @return
*/
public abstract boolean isImplicitWithStorage();
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) {
case HBASE:
return new HBaseIndex<>(config, jsc);
case INMEMORY:
return new InMemoryHashIndex<>(config, jsc);
case BLOOM:
return new HoodieBloomIndex<>(config, jsc);
case BUCKETED:
return new BucketedIndex<>(config, jsc);
}
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
protected transient JavaSparkContext jsc = null;
public enum IndexType {
HBASE,
INMEMORY,
BLOOM,
BUCKETED
}
protected final HoodieWriteConfig config;
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config;
this.jsc = jsc;
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
* If the optional FullFilePath value is not present, then the key is not found. If the
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
* file
*/
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present)
*/
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Extracts the location of written records, and updates the index.
*
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.
*/
public abstract boolean rollbackCommit(String commitTime);
/**
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the
* `partitionPath`. Such an implementation is able to obtain the same mapping, for two hoodie keys
* with same `recordKey` but different `partitionPath`
*
* @return whether or not, the index implementation is global in nature
*/
public abstract boolean isGlobal();
/**
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
* having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file.
*
* @return Returns true/false depending on whether the impl has this capability
*/
public abstract boolean canIndexLogFiles();
/**
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
* updates the index as well. This is used by storage, to save memory footprint in certain cases.
*/
public abstract boolean isImplicitWithStorage();
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) {
case HBASE:
return new HBaseIndex<>(config, jsc);
case INMEMORY:
return new InMemoryHashIndex<>(config, jsc);
case BLOOM:
return new HoodieBloomIndex<>(config, jsc);
case BUCKETED:
return new BucketedIndex<>(config, jsc);
}
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
}
}

View File

@@ -17,129 +17,119 @@
package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
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.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* Hoodie Index implementation backed by an in-memory Hash map.
* <p>
* ONLY USE FOR LOCAL TESTING
* Hoodie Index implementation backed by an in-memory Hash map. <p> ONLY USE FOR LOCAL TESTING
*/
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
recordLocationMap = new ConcurrentHashMap<>();
}
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
recordLocationMap = new ConcurrentHashMap<>();
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
@Override
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();
if (recordLocationMap.containsKey(rec.getKey())) {
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
}
taggedRecords.add(rec);
}
return taggedRecords.iterator();
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();
if (recordLocationMap.containsKey(rec.getKey())) {
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
}
taggedRecords.add(rec);
}
return taggedRecords.iterator();
}
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override
public WriteStatus call(WriteStatus writeStatus) {
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(record.getKey())) {
HoodieKey key = record.getKey();
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
if (newLocation.isPresent()) {
recordLocationMap.put(key, newLocation.get());
} else {
//Delete existing index for a deleted record
recordLocationMap.remove(key);
}
}
}
return writeStatus;
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override
public WriteStatus call(WriteStatus writeStatus) {
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(record.getKey())) {
HoodieKey key = record.getKey();
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
if (newLocation.isPresent()) {
recordLocationMap.put(key, newLocation.get());
} else {
//Delete existing index for a deleted record
recordLocationMap.remove(key);
}
});
}
}
}
return writeStatus;
}
});
}
@Override
public boolean rollbackCommit(String commitTime) {
return true;
}
@Override
public boolean rollbackCommit(String commitTime) {
return true;
}
/**
* Only looks up by recordKey
*
* @return
*/
@Override
public boolean isGlobal() {
return true;
}
/**
* Only looks up by recordKey
*/
@Override
public boolean isGlobal() {
return true;
}
/**
* Mapping is available in HBase already.
*
* @return
*/
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Mapping is available in HBase already.
*/
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Index needs to be explicitly updated after storage write.
*
* @return
*/
@Override
public boolean isImplicitWithStorage() {
return false;
}
/**
* Index needs to be explicitly updated after storage write.
*/
@Override
public boolean isImplicitWithStorage() {
return false;
}
}

View File

@@ -19,7 +19,6 @@
package com.uber.hoodie.index.bloom;
import com.google.common.base.Objects;
import java.io.Serializable;
/**
@@ -27,73 +26,75 @@ import java.io.Serializable;
*/
public class BloomIndexFileInfo implements Serializable {
private final String fileName;
private final String fileName;
private final String minRecordKey;
private final String minRecordKey;
private final String maxRecordKey;
private final String maxRecordKey;
public BloomIndexFileInfo(String fileName, String minRecordKey, String maxRecordKey) {
this.fileName = fileName;
this.minRecordKey = minRecordKey;
this.maxRecordKey = maxRecordKey;
public BloomIndexFileInfo(String fileName, String minRecordKey, String maxRecordKey) {
this.fileName = fileName;
this.minRecordKey = minRecordKey;
this.maxRecordKey = maxRecordKey;
}
public BloomIndexFileInfo(String fileName) {
this.fileName = fileName;
this.minRecordKey = null;
this.maxRecordKey = null;
}
public String getFileName() {
return fileName;
}
public String getMinRecordKey() {
return minRecordKey;
}
public String getMaxRecordKey() {
return maxRecordKey;
}
public boolean hasKeyRanges() {
return minRecordKey != null && maxRecordKey != null;
}
/**
* Does the given key fall within the range (inclusive)
*/
public boolean isKeyInRange(String recordKey) {
return minRecordKey.compareTo(recordKey) <= 0 &&
maxRecordKey.compareTo(recordKey) >= 0;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
public BloomIndexFileInfo(String fileName) {
this.fileName = fileName;
this.minRecordKey = null;
this.maxRecordKey = null;
}
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
return Objects.equal(that.fileName, fileName) &&
Objects.equal(that.minRecordKey, minRecordKey) &&
Objects.equal(that.maxRecordKey, maxRecordKey);
public String getFileName() {
return fileName;
}
}
public String getMinRecordKey() {
return minRecordKey;
}
@Override
public int hashCode() {
return Objects.hashCode(fileName, minRecordKey, maxRecordKey);
}
public String getMaxRecordKey() {
return maxRecordKey;
}
public boolean hasKeyRanges() {
return minRecordKey != null && maxRecordKey != null;
}
/**
* Does the given key fall within the range (inclusive)
* @param recordKey
* @return
*/
public boolean isKeyInRange(String recordKey) {
return minRecordKey.compareTo(recordKey) <= 0 &&
maxRecordKey.compareTo(recordKey) >= 0;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
return Objects.equal(that.fileName, fileName) &&
Objects.equal(that.minRecordKey, minRecordKey) &&
Objects.equal(that.maxRecordKey, maxRecordKey);
}
@Override
public int hashCode() {
return Objects.hashCode(fileName, minRecordKey, maxRecordKey);
}
public String toString() {
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
sb.append(" fileName=").append(fileName);
sb.append(" minRecordKey=").append(minRecordKey);
sb.append(" maxRecordKey=").append(maxRecordKey);
sb.append('}');
return sb.toString();
}
public String toString() {
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
sb.append(" fileName=").append(fileName);
sb.append(" minRecordKey=").append(minRecordKey);
sb.append(" maxRecordKey=").append(maxRecordKey);
sb.append('}');
return sb.toString();
}
}

View File

@@ -18,9 +18,12 @@
package com.uber.hoodie.index.bloom;
import static java.util.stream.Collectors.groupingBy;
import static java.util.stream.Collectors.mapping;
import static java.util.stream.Collectors.toList;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
@@ -34,7 +37,10 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.MetadataNotFoundException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -42,369 +48,370 @@ import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.storage.StorageLevel;
import scala.Tuple2;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static java.util.stream.Collectors.*;
/**
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
* its metadata.
*/
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION = SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
final HoodieTable<T> hoodieTable) {
// Step 0: cache the input record RDD
if (config.getBloomIndexUseCaching()) {
recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTable<T> hoodieTable) {
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
// Step 0: cache the input record RDD
if (config.getBloomIndexUseCaching()) {
recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
}
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
hoodieTable);
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, hoodieTable);
// Cache the result, for subsequent stages.
if (config.getBloomIndexUseCaching()) {
rowKeyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
}
if (logger.isDebugEnabled()) {
long totalTaggedRecords = rowKeyFilenamePairRDD.count();
logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
}
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
// Cost: 4 sec.
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(rowKeyFilenamePairRDD, recordRDD);
if (config.getBloomIndexUseCaching()) {
recordRDD.unpersist(); // unpersist the input Record RDD
rowKeyFilenamePairRDD.unpersist();
}
return taggedRecordRDD;
// Cache the result, for subsequent stages.
if (config.getBloomIndexUseCaching()) {
rowKeyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
}
if (logger.isDebugEnabled()) {
long totalTaggedRecords = rowKeyFilenamePairRDD.count();
logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
}
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
// Cost: 4 sec.
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(rowKeyFilenamePairRDD,
recordRDD);
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, table);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD)
.mapToPair(keyPathTuple -> {
Optional<String> recordLocationPath;
if (keyPathTuple._2._2.isPresent()) {
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional.of(new Path(
new Path(table.getMetaClient().getBasePath(), partitionPath),
fileName).toUri().getPath());
} else {
recordLocationPath = Optional.absent();
}
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
});
if (config.getBloomIndexUseCaching()) {
recordRDD.unpersist(); // unpersist the input Record RDD
rowKeyFilenamePairRDD.unpersist();
}
/**
* Lookup the location for each record key and return the pair<record_key,location> for all
* record keys already present and drop the record keys if not present
*/
private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
// Obtain records per partition, in the incoming records
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
return taggedRecordRDD;
}
// Step 2: Load all involved files as <Partition, filename> pairs
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, parallelism);
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, table);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD)
.mapToPair(keyPathTuple -> {
Optional<String> recordLocationPath;
if (keyPathTuple._2._2.isPresent()) {
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional.of(new Path(
new Path(table.getMetaClient().getBasePath(), partitionPath),
fileName).toUri().getPath());
} else {
recordLocationPath = Optional.absent();
}
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
});
}
/**
* Lookup the location for each record key and return the pair<record_key,location> for all record
* keys already present and drop the record keys if not present
*/
private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
// Obtain records per partition, in the incoming records
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
// Step 2: Load all involved files as <Partition, filename> pairs
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(
affectedPartitionPathList, hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD,
parallelism);
}
/**
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
*
* To be able to smoothly handle skews, we need to compute how to split each partitions into
* subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to <
* 2GB.
*
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified
* as a NON-zero number, then that is used explicitly.
*/
private int autoComputeParallelism(final Map<String, Long> recordsPerPartition,
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
long totalComparisons = 0;
if (config.getBloomIndexPruneByRanges()) {
// we will just try exploding the input and then count to determine comparisons
totalComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo,
partitionRecordKeyPairRDD).count();
} else {
// if not pruning by ranges, then each file in a partition needs to compared against all
// records for a partition.
Map<String, Long> filesPerPartition = partitionToFileInfo.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size())));
long totalFiles = 0, totalRecords = 0;
for (String partitionPath : recordsPerPartition.keySet()) {
long numRecords = recordsPerPartition.get(partitionPath);
long numFiles =
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
: 1L;
totalComparisons += numFiles * numRecords;
totalFiles +=
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
: 0L;
totalRecords += numRecords;
}
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles
+ ", TotalAffectedPartitions:" + recordsPerPartition.size());
}
/**
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
*
* To be able to smoothly handle skews, we need to compute how to split each partitions into
* subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to
* < 2GB.
*
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified as a NON-zero number,
* then that is used explicitly.
*
*/
private int autoComputeParallelism(final Map<String, Long> recordsPerPartition,
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
// each partition will have an item per comparison.
int parallelism = (int) (totalComparisons / MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
logger.info(
"Auto computed parallelism :" + parallelism + ", totalComparisons: " + totalComparisons);
return parallelism;
}
long totalComparisons = 0;
if (config.getBloomIndexPruneByRanges()) {
// we will just try exploding the input and then count to determine comparisons
totalComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD).count();
} else {
// if not pruning by ranges, then each file in a partition needs to compared against all
// records for a partition.
Map<String, Long> filesPerPartition = partitionToFileInfo.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size())));
long totalFiles = 0, totalRecords = 0;
for (String partitionPath : recordsPerPartition.keySet()) {
long numRecords = recordsPerPartition.get(partitionPath);
long numFiles = filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) : 1L;
/**
* Its crucial to pick the right parallelism.
*
* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism :
* typically number of input file splits
*
* We pick the max such that, we are always safe, but go higher if say a there are a lot of input
* files. (otherwise, we will fallback to number of partitions in input and end up with slow
* performance)
*/
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
// If bloom index parallelism is set, use it to to check against the input parallelism and take the max
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
logger.info("InputParallelism: ${" + inputParallelism + "}, " +
"IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " +
"TotalSubParts: ${" + totalSubPartitions + "}, " +
"Join Parallelism set to : " + joinParallelism);
return joinParallelism;
}
totalComparisons += numFiles * numRecords;
totalFiles += filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) : 0L;
totalRecords += numRecords;
}
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles + ", TotalAffectedPartitions:" + recordsPerPartition.size());
}
// each partition will have an item per comparison.
int parallelism = (int) (totalComparisons/ MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
logger.info("Auto computed parallelism :" + parallelism + ", totalComparisons: " + totalComparisons);
return parallelism;
}
/**
* Its crucial to pick the right parallelism.
*
* totalSubPartitions : this is deemed safe limit, to be nice with Spark.
* inputParallelism : typically number of input file splits
*
* We pick the max such that, we are always safe, but go higher if say a there are a lot of
* input files. (otherwise, we will fallback to number of partitions in input and end up with
* slow performance)
*/
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
// If bloom index parallelism is set, use it to to check against the input parallelism and take the max
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
logger.info("InputParallelism: ${" + inputParallelism + "}, " +
"IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " +
"TotalSubParts: ${" + totalSubPartitions + "}, " +
"Join Parallelism set to : " + joinParallelism);
return joinParallelism;
}
/**
* Load all involved files as <Partition, filename> pair RDD.
*/
@VisibleForTesting
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieTable<T> hoodieTable) {
// Obtain the latest data files from all the partitions.
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc.parallelize(partitions, Math.max(partitions.size(), 1))
.flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime =
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
filteredFiles =
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
latestCommitTime.get().getTimestamp())
.map(f -> new Tuple2<>(partitionPath, f))
.collect(toList());
}
return filteredFiles.iterator();
}).collect();
if (config.getBloomIndexPruneByRanges()) {
// also obtain file ranges, if range pruning is enabled
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1))
.mapToPair(ft -> {
try {
String[] minMaxKeys = ParquetUtils.readMinMaxRecordKeys(ft._2().getFileStatus().getPath());
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
} catch (MetadataNotFoundException me) {
logger.warn("Unable to find range metadata in file :" + ft._2());
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
}
}).collect();
} else {
return dataFilesList.stream()
.map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())))
/**
* Load all involved files as <Partition, filename> pair RDD.
*/
@VisibleForTesting
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions,
final HoodieTable<T> hoodieTable) {
// Obtain the latest data files from all the partitions.
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
.parallelize(partitions, Math.max(partitions.size(), 1))
.flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime =
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
filteredFiles =
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
latestCommitTime.get().getTimestamp())
.map(f -> new Tuple2<>(partitionPath, f))
.collect(toList());
}
return filteredFiles.iterator();
}).collect();
if (config.getBloomIndexPruneByRanges()) {
// also obtain file ranges, if range pruning is enabled
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1))
.mapToPair(ft -> {
try {
String[] minMaxKeys = ParquetUtils
.readMinMaxRecordKeys(ft._2().getFileStatus().getPath());
return new Tuple2<>(ft._1(),
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
} catch (MetadataNotFoundException me) {
logger.warn("Unable to find range metadata in file :" + ft._2());
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
}
}).collect();
} else {
return dataFilesList.stream()
.map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())))
.collect(toList());
}
}
@Override
public boolean rollbackCommit(String commitTime) {
// Nope, don't need to do anything.
return true;
}
/**
* This is not global, since we depend on the partitionPath to do the lookup
*/
@Override
public boolean isGlobal() {
return false;
}
/**
* No indexes into log files yet.
*/
@Override
public boolean canIndexLogFiles() {
return false;
}
/**
* Bloom filters are stored, into the same data files.
*/
@Override
public boolean isImplicitWithStorage() {
return true;
}
/**
* if we dont have key ranges, then also we need to compare against the file. no other choice if
* we do, then only compare the file if the record key falls in range.
*/
private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
}
/**
* For each incoming record, produce N output records, 1 each for each file against which the
* record's key needs to be checked. For datasets, where the keys have a definite insert order
* (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range
* pruning.
*/
// sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey
// ranges in the index info.
@VisibleForTesting
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
return partitionRecordKeyPairRDD
.map(partitionRecordKeyPair -> {
String recordKey = partitionRecordKeyPair._2();
String partitionPath = partitionRecordKeyPair._1();
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
if (indexInfos
!= null) { // could be null, if there are no files in a given partition yet.
// for each candidate file in partition, that needs to be compared.
for (BloomIndexFileInfo indexInfo : indexInfos) {
if (shouldCompareWithFile(indexInfo, recordKey)) {
recordComparisons.add(
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
new Tuple2<>(indexInfo.getFileName(),
new HoodieKey(recordKey, partitionPath))));
}
}
}
return recordComparisons;
})
.flatMapToPair(t -> t.iterator());
}
/**
* Find out <RowKey, filename> pair. All workload grouped by file-level.
*
* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such
* that each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load
* rowKeys, (3) Tag rowKey
*
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
int totalSubpartitions) {
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
totalSubpartitions);
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons(
partitionToFileIndexInfo, partitionRecordKeyPairRDD)
// sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly
.sortByKey(true, joinParallelism);
return fileSortedTripletRDD
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> {
List<Tuple2<String, String>> vals = new ArrayList<>();
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
}
return vals.iterator();
});
}
/**
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
*/
private JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
JavaPairRDD<String, String> rowKeyFilenamePairRDD,
JavaRDD<HoodieRecord<T>> recordRDD) {
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
v1 -> {
HoodieRecord<T> record = v1._1();
if (v1._2().isPresent()) {
String filename = v1._2().get();
if (filename != null && !filename.isEmpty()) {
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename)));
}
}
return record;
}
}
);
}
@Override
public boolean rollbackCommit(String commitTime) {
// Nope, don't need to do anything.
return true;
}
/**
* This is not global, since we depend on the partitionPath to do the lookup
*
* @return
*/
@Override
public boolean isGlobal() {
return false;
}
/**
* No indexes into log files yet.
*
* @return
*/
@Override
public boolean canIndexLogFiles() {
return false;
}
/**
* Bloom filters are stored, into the same data files.
*
* @return
*/
@Override
public boolean isImplicitWithStorage() {
return true;
}
/**
* if we dont have key ranges, then also we need to compare against the file. no other choice
* if we do, then only compare the file if the record key falls in range.
* @param indexInfo
* @param recordKey
* @return
*/
private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
}
/**
* For each incoming record, produce N output records, 1 each for each file against which the record's key
* needs to be checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix),
* the number of files to be compared gets cut down a lot from range pruning.
*
*
* @param partitionToFileIndexInfo
* @param partitionRecordKeyPairRDD
* @return
*/
// sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey
// ranges in the index info.
@VisibleForTesting
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
return partitionRecordKeyPairRDD
.map(partitionRecordKeyPair -> {
String recordKey = partitionRecordKeyPair._2();
String partitionPath = partitionRecordKeyPair._1();
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
// for each candidate file in partition, that needs to be compared.
for (BloomIndexFileInfo indexInfo : indexInfos) {
if (shouldCompareWithFile(indexInfo, recordKey)) {
recordComparisons.add(
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
new Tuple2<>(indexInfo.getFileName(), new HoodieKey(recordKey, partitionPath))));
}
}
}
return recordComparisons;
})
.flatMapToPair(t -> t.iterator());
}
/**
* Find out <RowKey, filename> pair. All workload grouped by file-level.
*
* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition
* such that each RDD partition is a file, then for each file, we do
* (1) load bloom filter,
* (2) load rowKeys,
* (3) Tag rowKey
*
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
int totalSubpartitions) {
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions);
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
// sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly
.sortByKey(true, joinParallelism);
return fileSortedTripletRDD
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> {
List<Tuple2<String, String>> vals = new ArrayList<>();
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
}
return vals.iterator();
});
}
/**
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
*/
private JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(JavaPairRDD<String, String> rowKeyFilenamePairRDD,
JavaRDD<HoodieRecord<T>> recordRDD) {
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
v1 -> {
HoodieRecord<T> record = v1._1();
if (v1._2().isPresent()) {
String filename = v1._2().get();
if (filename != null && !filename.isEmpty()) {
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename)));
}
}
return record;
}
);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) {
return writeStatusRDD;
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) {
return writeStatusRDD;
}
}

View File

@@ -24,172 +24,182 @@ import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.function.Function2;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.function.Function2;
import scala.Tuple2;
/**
* Function performing actual checking of RDD parition containing (fileId, hoodieKeys) against the
* actual files
*/
public class HoodieBloomIndexCheckFunction implements Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
public class HoodieBloomIndexCheckFunction implements
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
private final String basePath;
private final String basePath;
public HoodieBloomIndexCheckFunction(String basePath) {
this.basePath = basePath;
public HoodieBloomIndexCheckFunction(String basePath) {
this.basePath = basePath;
}
/**
* Given a list of row keys and one file, return only row keys existing in that file.
*/
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys,
Path filePath) throws HoodieIndexException {
List<String> foundRecordKeys = new ArrayList<>();
try {
// Load all rowKeys from the file, to double-confirm
if (!candidateRecordKeys.isEmpty()) {
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
if (logger.isDebugEnabled()) {
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
}
for (String rowKey : candidateRecordKeys) {
if (fileRowKeys.contains(rowKey)) {
foundRecordKeys.add(rowKey);
}
}
logger.info("After checking with row keys, we have " + foundRecordKeys.size()
+ " results, for file " + filePath + " => " + foundRecordKeys);
if (logger.isDebugEnabled()) {
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
}
}
} catch (Exception e) {
throw new HoodieIndexException("Error checking candidate keys against file.", e);
}
return foundRecordKeys;
}
/**
* Given a list of row keys and one file, return only row keys existing in that file.
*/
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
List<String> foundRecordKeys = new ArrayList<>();
try {
// Load all rowKeys from the file, to double-confirm
if (!candidateRecordKeys.isEmpty()) {
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
if (logger.isDebugEnabled()) {
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
}
for (String rowKey : candidateRecordKeys) {
if (fileRowKeys.contains(rowKey)) {
foundRecordKeys.add(rowKey);
}
}
logger.info("After checking with row keys, we have " + foundRecordKeys.size() + " results, for file " + filePath + " => " + foundRecordKeys);
if (logger.isDebugEnabled()) {
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
}
}
} catch (Exception e){
throw new HoodieIndexException("Error checking candidate keys against file.", e);
}
return foundRecordKeys;
class LazyKeyCheckIterator extends
LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
private List<String> candidateRecordKeys;
private BloomFilter bloomFilter;
private String currentFile;
private String currentParitionPath;
LazyKeyCheckIterator(
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) {
super(fileParitionRecordKeyTripletItr);
currentFile = null;
candidateRecordKeys = new ArrayList<>();
bloomFilter = null;
currentParitionPath = null;
}
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
private List<String> candidateRecordKeys;
private BloomFilter bloomFilter;
private String currentFile;
private String currentParitionPath;
LazyKeyCheckIterator(Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) {
super(fileParitionRecordKeyTripletItr);
currentFile = null;
candidateRecordKeys = new ArrayList<>();
bloomFilter = null;
currentParitionPath = null;
}
@Override
protected void start() {
}
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
candidateRecordKeys = new ArrayList<>();
currentFile = fileName;
currentParitionPath = partitionPath;
} catch (Exception e) {
throw new HoodieIndexException("Error checking candidate keys against file.", e);
}
}
@Override
protected List<IndexLookupResult> computeNext() {
List<IndexLookupResult> ret = new ArrayList<>();
try {
// process one file in each go.
while (inputItr.hasNext()) {
Tuple2<String, Tuple2<String, HoodieKey>> currentTuple = inputItr.next();
String fileName = currentTuple._2._1;
String partitionPath = currentTuple._2._2.getPartitionPath();
String recordKey = currentTuple._2._2.getRecordKey();
// lazily init state
if (currentFile == null) {
initState(fileName, partitionPath);
}
// if continue on current file)
if (fileName.equals(currentFile)) {
// check record key against bloom filter of current file & add to possible keys if needed
if (bloomFilter.mightContain(recordKey)) {
if (logger.isDebugEnabled()) {
logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName);
}
candidateRecordKeys.add(recordKey);
}
} else {
// do the actual checking of file & break out
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info("#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys.size() + " for " + filePath);
if (logger.isDebugEnabled()) {
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) {
if (logger.isDebugEnabled()) {
logger.debug("#2 Adding " + recordKey + " as candidate for file " + fileName);
}
candidateRecordKeys.add(recordKey);
}
break;
}
}
// handle case, where we ran out of input, finish pending work, update return val
if (!inputItr.hasNext()) {
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info("#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys.size() + " for " + filePath);
if (logger.isDebugEnabled()) {
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
}
} catch (Throwable e) {
if (e instanceof HoodieException) {
throw e;
}
throw new HoodieIndexException("Error checking bloom filter index. ", e);
}
return ret;
}
@Override
protected void end() {
}
}
@Override
public Iterator<List<IndexLookupResult>> call(Integer partition,
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) throws Exception {
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
protected void start() {
}
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
candidateRecordKeys = new ArrayList<>();
currentFile = fileName;
currentParitionPath = partitionPath;
} catch (Exception e) {
throw new HoodieIndexException("Error checking candidate keys against file.", e);
}
}
@Override
protected List<IndexLookupResult> computeNext() {
List<IndexLookupResult> ret = new ArrayList<>();
try {
// process one file in each go.
while (inputItr.hasNext()) {
Tuple2<String, Tuple2<String, HoodieKey>> currentTuple = inputItr.next();
String fileName = currentTuple._2._1;
String partitionPath = currentTuple._2._2.getPartitionPath();
String recordKey = currentTuple._2._2.getRecordKey();
// lazily init state
if (currentFile == null) {
initState(fileName, partitionPath);
}
// if continue on current file)
if (fileName.equals(currentFile)) {
// check record key against bloom filter of current file & add to possible keys if needed
if (bloomFilter.mightContain(recordKey)) {
if (logger.isDebugEnabled()) {
logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName);
}
candidateRecordKeys.add(recordKey);
}
} else {
// do the actual checking of file & break out
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info(
"#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys
.size() + " for " + filePath);
if (logger.isDebugEnabled()) {
logger
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) {
if (logger.isDebugEnabled()) {
logger.debug("#2 Adding " + recordKey + " as candidate for file " + fileName);
}
candidateRecordKeys.add(recordKey);
}
break;
}
}
// handle case, where we ran out of input, finish pending work, update return val
if (!inputItr.hasNext()) {
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info(
"#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys
.size() + " for " + filePath);
if (logger.isDebugEnabled()) {
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
}
} catch (Throwable e) {
if (e instanceof HoodieException) {
throw e;
}
throw new HoodieIndexException("Error checking bloom filter index. ", e);
}
return ret;
}
@Override
protected void end() {
}
}
@Override
public Iterator<List<IndexLookupResult>> call(Integer partition,
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
throws Exception {
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
}
}

View File

@@ -25,21 +25,21 @@ import java.util.List;
*/
public class IndexLookupResult {
private String fileName;
private String fileName;
private List<String> matchingRecordKeys;
private List<String> matchingRecordKeys;
public IndexLookupResult(String fileName, List<String> matchingRecordKeys) {
this.fileName = fileName;
this.matchingRecordKeys = matchingRecordKeys;
}
public IndexLookupResult(String fileName, List<String> matchingRecordKeys) {
this.fileName = fileName;
this.matchingRecordKeys = matchingRecordKeys;
}
public String getFileName() {
return fileName;
}
public String getFileName() {
return fileName;
}
public List<String> getMatchingRecordKeys() {
return matchingRecordKeys;
}
public List<String> getMatchingRecordKeys() {
return matchingRecordKeys;
}
}

View File

@@ -19,7 +19,6 @@
package com.uber.hoodie.index.bucketed;
import com.google.common.base.Optional;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -29,96 +28,86 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import scala.Tuple2;
/**
* An `stateless` index implementation that will using a deterministic mapping function to
* determine the fileID for a given record.
*
* Pros:
* - Fast
*
* Cons :
* - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune this)
* - Could increase write amplification on copy-on-write storage since inserts always rewrite files
* - Not global.
*
* An `stateless` index implementation that will using a deterministic mapping function to determine
* the fileID for a given record.
*
* Pros: - Fast
*
* Cons : - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune
* this) - Could increase write amplification on copy-on-write storage since inserts always rewrite
* files - Not global.
*/
public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
private String getBucket(String recordKey) {
return String.valueOf(recordKey.hashCode() % config.getNumBucketsPerPartition());
}
private String getBucket(String recordKey) {
return String.valueOf(recordKey.hashCode() % config.getNumBucketsPerPartition());
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) {
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) throws HoodieIndexException {
return recordRDD.map(record -> {
String bucket = getBucket(record.getRecordKey());
//HACK(vc) a non-existent commit is provided here.
record.setCurrentLocation(new HoodieRecordLocation("000", bucket));
return record;
});
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException {
return recordRDD.map(record -> {
String bucket = getBucket(record.getRecordKey());
//HACK(vc) a non-existent commit is provided here.
record.setCurrentLocation(new HoodieRecordLocation("000", bucket));
return record;
});
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) throws HoodieIndexException {
return writeStatusRDD;
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException {
return writeStatusRDD;
}
@Override
public boolean rollbackCommit(String commitTime) {
// nothing to rollback in the index.
return true;
}
@Override
public boolean rollbackCommit(String commitTime) {
// nothing to rollback in the index.
return true;
}
/**
* Bucketing is still done within each partition.
*
* @return
*/
@Override
public boolean isGlobal() {
return false;
}
/**
* Bucketing is still done within each partition.
*/
@Override
public boolean isGlobal() {
return false;
}
/**
* Since indexing is just a deterministic hash, we can identify file group correctly even without an index
* on the actual log file.
*
* @return
*/
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Since indexing is just a deterministic hash, we can identify file group correctly even without
* an index on the actual log file.
*/
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Indexing is just a hash function.
*
* @return
*/
@Override
public boolean isImplicitWithStorage() {
return true;
}
/**
* Indexing is just a hash function.
*/
@Override
public boolean isImplicitWithStorage() {
return true;
}
}

View File

@@ -19,24 +19,33 @@
package com.uber.hoodie.index.hbase;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
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.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.*;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -45,230 +54,221 @@ import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function2;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/**
* Hoodie Index implementation backed by HBase
*/
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
private final String tableName;
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
private final String tableName;
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
}
private static Connection hbaseConnection = null;
private Connection getHBaseConnection() {
Configuration hbaseConfig = HBaseConfiguration.create();
String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP);
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
try {
return ConnectionFactory.createConnection(hbaseConfig);
} catch (IOException e) {
throw new HoodieDependentSystemUnavailableException(
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
}
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
private final HoodieTable<T> hoodieTable;
LocationTagFunction(HoodieTable<T> hoodieTable) {
this.hoodieTable = hoodieTable;
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
}
private static Connection hbaseConnection = null;
private Connection getHBaseConnection() {
Configuration hbaseConfig = HBaseConfiguration.create();
String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP);
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
try {
return ConnectionFactory.createConnection(hbaseConfig);
} catch (IOException e) {
throw new HoodieDependentSystemUnavailableException(
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null) {
hbaseConnection = getHBaseConnection();
}
}
}
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
// Do the tagging.
while (hoodieRecordIterator.hasNext()) {
HoodieRecord rec = hoodieRecordIterator.next();
// TODO(vc): This may need to be a multi get.
Result result = hTable.get(
new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1)
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
// first, attempt to grab location from HBase
if (result.getRow() != null) {
String commitTs =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
String fileId =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
private final HoodieTable<T> hoodieTable;
LocationTagFunction(HoodieTable<T> hoodieTable) {
this.hoodieTable = hoodieTable;
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
// if the last commit ts for this row is less than the system commit ts
if (!commitTimeline.empty() && commitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
}
}
taggedRecords.add(rec);
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Tag indexed locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
@Override
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null) {
hbaseConnection = getHBaseConnection();
}
}
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
// Do the tagging.
while (hoodieRecordIterator.hasNext()) {
HoodieRecord rec = hoodieRecordIterator.next();
// TODO(vc): This may need to be a multi get.
Result result = hTable.get(
new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1)
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
}
return taggedRecords.iterator();
}
}
// first, attempt to grab location from HBase
if (result.getRow() != null) {
String commitTs =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
String fileId =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
}
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
// if the last commit ts for this row is less than the system commit ts
if (!commitTimeline.empty() && commitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
}
}
taggedRecords.add(rec);
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Tag indexed locations because of exception with HBase Client", e);
}
class UpdateLocationTask implements
Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
@Override
public Iterator<WriteStatus> call(Integer partition, Iterator<WriteStatus> statusIterator) {
}
return taggedRecords.iterator();
List<WriteStatus> writeStatusList = new ArrayList<>();
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null) {
hbaseConnection = getHBaseConnection();
}
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
}
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
@Override
public Iterator<WriteStatus> call(Integer partition, Iterator<WriteStatus> statusIterator) {
List<WriteStatus> writeStatusList = new ArrayList<>();
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null) {
hbaseConnection = getHBaseConnection();
}
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
while (statusIterator.hasNext()) {
WriteStatus writeStatus = statusIterator.next();
List<Put> puts = new ArrayList<>();
List<Delete> deletes = new ArrayList<>();
try {
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(rec.getKey())) {
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
if (loc.isPresent()) {
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
Bytes.toBytes(loc.get().getCommitTime()));
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
Bytes.toBytes(loc.get().getFileId()));
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
Bytes.toBytes(rec.getPartitionPath()));
puts.add(put);
} else {
//Delete existing index for a deleted record
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
deletes.add(delete);
}
}
}
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
while (statusIterator.hasNext()) {
WriteStatus writeStatus = statusIterator.next();
List<Put> puts = new ArrayList<>();
List<Delete> deletes = new ArrayList<>();
try {
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(rec.getKey())) {
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
if(loc.isPresent()) {
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
Bytes.toBytes(loc.get().getCommitTime()));
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
Bytes.toBytes(loc.get().getFileId()));
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
Bytes.toBytes(rec.getPartitionPath()));
puts.add(put);
} else {
//Delete existing index for a deleted record
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
deletes.add(delete);
}
}
}
hTable.put(puts);
hTable.delete(deletes);
hTable.flushCommits();
} catch (Exception e) {
Exception we = new Exception("Error updating index for " + writeStatus, e);
logger.error(we);
writeStatus.setGlobalError(we);
}
writeStatusList.add(writeStatus);
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Update Index locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
}
return writeStatusList.iterator();
hTable.put(puts);
hTable.delete(deletes);
hTable.flushCommits();
} catch (Exception e) {
Exception we = new Exception("Error updating index for " + writeStatus, e);
logger.error(we);
writeStatus.setGlobalError(we);
}
writeStatusList.add(writeStatus);
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Update Index locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
}
return writeStatusList.iterator();
}
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
}
@Override
public boolean rollbackCommit(String commitTime) {
// Can't really rollback here. HBase only can let you go from recordKey to fileID,
// not the other way around
return true;
}
@Override
public boolean rollbackCommit(String commitTime) {
// Can't really rollback here. HBase only can let you go from recordKey to fileID,
// not the other way around
return true;
}
/**
* Only looks up by recordKey
*
* @return
*/
@Override
public boolean isGlobal() {
return true;
}
/**
* Only looks up by recordKey
*/
@Override
public boolean isGlobal() {
return true;
}
/**
* Mapping is available in HBase already.
*
* @return
*/
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Mapping is available in HBase already.
*/
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Index needs to be explicitly updated after storage write.
*
* @return
*/
@Override
public boolean isImplicitWithStorage() {
return false;
}
/**
* Index needs to be explicitly updated after storage write.
*/
@Override
public boolean isImplicitWithStorage() {
return false;
}
}

View File

@@ -36,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieAppendException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
@@ -50,155 +43,161 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
/**
* IO Operation to append data onto an existing file.
*
* @param <T>
*/
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
private static AtomicLong recordIndex = new AtomicLong(1);
private final WriteStatus writeStatus;
private final String fileId;
private String partitionPath;
private List<HoodieRecord<T>> records;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private HoodieLogFile currentLogFile;
private Writer writer;
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
private static AtomicLong recordIndex = new AtomicLong(1);
public HoodieAppendHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
String fileId,
Iterator<HoodieRecord<T>> recordItr) {
super(config, commitTime, hoodieTable);
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieDeltaWriteStat());
this.writeStatus = writeStatus;
this.fileId = fileId;
init(recordItr);
}
private final WriteStatus writeStatus;
private final String fileId;
private String partitionPath;
private List<HoodieRecord<T>> records;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private HoodieLogFile currentLogFile;
private Writer writer;
private void init(Iterator<HoodieRecord<T>> recordItr) {
List<HoodieRecord<T>> records = Lists.newArrayList();
recordItr.forEachRemaining(record -> {
records.add(record);
// extract some information from the first record
if (partitionPath == null) {
partitionPath = record.getPartitionPath();
// HACK(vc) This also assumes a base file. It will break, if appending without one.
String latestValidFilePath =
fileSystemView.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
writeStatus.getStat().setFileId(fileId);
public HoodieAppendHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
String fileId,
Iterator<HoodieRecord<T>> recordItr) {
super(config, commitTime, hoodieTable);
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieDeltaWriteStat());
this.writeStatus = writeStatus;
this.fileId = fileId;
init(recordItr);
}
try {
this.writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime)
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
this.currentLogFile = writer.getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogOffset(writer.getCurrentSize());
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
.getMetaClient().getBasePath() + partitionPath, e);
}
Path path = new Path(record.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPath(path.toString());
}
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
});
this.records = records;
}
private void init(Iterator<HoodieRecord<T>> recordItr) {
List<HoodieRecord<T>> records = Lists.newArrayList();
recordItr.forEachRemaining(record -> {
records.add(record);
// extract some information from the first record
if (partitionPath == null) {
partitionPath = record.getPartitionPath();
// HACK(vc) This also assumes a base file. It will break, if appending without one.
String latestValidFilePath =
fileSystemView.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
writeStatus.getStat().setFileId(fileId);
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
if(avroRecord.isPresent()) {
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
recordIndex.getAndIncrement());
HoodieAvroUtils
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
hoodieRecord.getPartitionPath(), fileId);
HoodieAvroUtils
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
recordsWritten++;
} else {
recordsDeleted++;
}
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return avroRecord;
this.writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime)
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
this.currentLogFile = writer.getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogOffset(writer.getCurrentSize());
} catch (Exception e) {
logger.error("Error writing record " + hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
.getMetaClient().getBasePath() + partitionPath, e);
}
return Optional.empty();
}
Path path = new Path(record.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPath(path.toString());
}
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
});
this.records = records;
}
public void doAppend() {
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
List<IndexedRecord> recordList = new ArrayList<>();
List<String> keysToDelete = new ArrayList<>();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
records.stream().forEach(record -> {
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if(indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
} else {
keysToDelete.add(record.getRecordKey());
}
});
try {
if(recordList.size() > 0) {
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
}
if(keysToDelete.size() > 0) {
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
}
} catch (Exception e) {
throw new HoodieAppendException(
"Failed while appeding records to " + currentLogFile.getPath(), e);
}
}
if (avroRecord.isPresent()) {
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
recordIndex.getAndIncrement());
HoodieAvroUtils
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
hoodieRecord.getPartitionPath(), fileId);
HoodieAvroUtils
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
recordsWritten++;
} else {
recordsDeleted++;
}
public void close() {
try {
if (writer != null) {
writer.close();
}
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return avroRecord;
} catch (Exception e) {
logger.error("Error writing record " + hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
}
return Optional.empty();
}
public WriteStatus getWriteStatus() {
return writeStatus;
public void doAppend() {
List<IndexedRecord> recordList = new ArrayList<>();
List<String> keysToDelete = new ArrayList<>();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
records.stream().forEach(record -> {
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if (indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
} else {
keysToDelete.add(record.getRecordKey());
}
});
try {
if (recordList.size() > 0) {
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
}
if (keysToDelete.size() > 0) {
writer = writer.appendBlock(
new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
}
} catch (Exception e) {
throw new HoodieAppendException(
"Failed while appeding records to " + currentLogFile.getPath(), e);
}
}
public void close() {
try {
if (writer != null) {
writer.close();
}
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
}
public WriteStatus getWriteStatus() {
return writeStatus;
}
}

View File

@@ -27,226 +27,212 @@ import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
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.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
* <p>
* 1) It provides sufficient time for existing queries running on older versions, to finish
* <p>
* 2) It bounds the growth of the files in the file system
* <p>
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
* <p> 1) It provides sufficient time for existing queries running on older versions, to finish <p>
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
* based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
*/
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTable<T> hoodieTable;
private HoodieWriteConfig config;
private FileSystem fs;
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
this.hoodieTable = hoodieTable;
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fs = hoodieTable.getFs();
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTable<T> hoodieTable;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
this.hoodieTable = hoodieTable;
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fs = hoodieTable.getFs();
}
/**
* 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)
*/
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
throws IOException {
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
.getCleanerFileVersionsRetained() + " file versions. ");
List<HoodieFileGroup> fileGroups =
fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
// Collect all the datafiles savepointed by all the savepoints
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
int keepVersions = config.getCleanerFileVersionsRetained();
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
while (fileSliceIterator.hasNext() && keepVersions > 0) {
// Skip this most recent version
FileSlice nextSlice = fileSliceIterator.next();
HoodieDataFile dataFile = nextSlice.getDataFile().get();
if (savepointedFiles.contains(dataFile.getFileName())) {
// do not clean up a savepoint data file
continue;
}
keepVersions--;
}
// Delete the remaining files
while (fileSliceIterator.hasNext()) {
FileSlice nextSlice = fileSliceIterator.next();
HoodieDataFile dataFile = nextSlice.getDataFile().get();
deletePaths.add(dataFile.getFileStatus().getPath().toString());
if (hoodieTable.getMetaClient().getTableType()
== HoodieTableType.MERGE_ON_READ) {
// If merge on read, then clean the log files for the commits as well
deletePaths.addAll(nextSlice.getLogFiles()
.map(file -> file.getPath().toString())
.collect(Collectors.toList()));
}
}
}
return deletePaths;
}
/**
* 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. ");
List<HoodieFileGroup> fileGroups =
fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
// Collect all the datafiles savepointed by all the savepoints
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
/**
* Selects the versions for file for cleaning, such that it <p> - 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. <p> 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) <p> This policy is the
* default.
*/
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<>();
for (HoodieFileGroup fileGroup : fileGroups) {
int keepVersions = config.getCleanerFileVersionsRetained();
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
while (fileSliceIterator.hasNext() && keepVersions > 0) {
// Skip this most recent version
FileSlice nextSlice = fileSliceIterator.next();
HoodieDataFile dataFile = nextSlice.getDataFile().get();
if(savepointedFiles.contains(dataFile.getFileName())) {
// do not clean up a savepoint data file
continue;
}
keepVersions--;
}
// Delete the remaining files
while (fileSliceIterator.hasNext()) {
FileSlice nextSlice = fileSliceIterator.next();
HoodieDataFile dataFile = nextSlice.getDataFile().get();
deletePaths.add(dataFile.getFileStatus().getPath().toString());
if (hoodieTable.getMetaClient().getTableType()
== HoodieTableType.MERGE_ON_READ) {
// If merge on read, then clean the log files for the commits as well
deletePaths.addAll(nextSlice.getLogFiles()
.map(file -> file.getPath().toString())
.collect(Collectors.toList()));
}
// Collect all the datafiles savepointed by all the savepoints
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
// determine if we have enough commits, to start cleaning.
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
List<HoodieFileGroup> fileGroups =
fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
String lastVersion = dataFile.getCommitTime();
String lastVersionBeforeEarliestCommitToRetain =
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
// Ensure there are more than 1 version of the file (we only clean old files from updates)
// i.e always spare the last commit.
for (FileSlice aSlice : fileSliceList) {
HoodieDataFile aFile = aSlice.getDataFile().get();
String fileCommitTime = aFile.getCommitTime();
if (savepointedFiles.contains(aFile.getFileName())) {
// do not clean up a savepoint data file
continue;
}
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
// The window of commit retain == max query run time. So a query could be running which still
// 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 (HoodieTimeline.compareTimestamps(
earliestCommitToRetain.getTimestamp(),
fileCommitTime,
HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
deletePaths.add(aFile.getFileStatus().getPath().toString());
if (hoodieTable.getMetaClient().getTableType()
== HoodieTableType.MERGE_ON_READ) {
// If merge on read, then clean the log files for the commits as well
deletePaths.addAll(aSlice.getLogFiles()
.map(file -> file.getPath().toString())
.collect(Collectors.toList()));
}
}
}
return deletePaths;
}
}
return deletePaths;
}
/**
* Selects the versions for file for cleaning, such that it
* <p>
* - 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.
* <p>
* 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)
* <p>
* 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<>();
// Collect all the datafiles savepointed by all the savepoints
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
// determine if we have enough commits, to start cleaning.
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
List<HoodieFileGroup> fileGroups =
fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
String lastVersion = dataFile.getCommitTime();
String lastVersionBeforeEarliestCommitToRetain =
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
// Ensure there are more than 1 version of the file (we only clean old files from updates)
// i.e always spare the last commit.
for (FileSlice aSlice : fileSliceList) {
HoodieDataFile aFile = aSlice.getDataFile().get();
String fileCommitTime = aFile.getCommitTime();
if(savepointedFiles.contains(aFile.getFileName())) {
// do not clean up a savepoint data file
continue;
}
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
// The window of commit retain == max query run time. So a query could be running which still
// 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 (HoodieTimeline.compareTimestamps(
earliestCommitToRetain.getTimestamp(),
fileCommitTime,
HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
deletePaths.add(aFile.getFileStatus().getPath().toString());
if (hoodieTable.getMetaClient().getTableType()
== HoodieTableType.MERGE_ON_READ) {
// If merge on read, then clean the log files for the commits as well
deletePaths.addAll(aSlice.getLogFiles()
.map(file -> file.getPath().toString())
.collect(Collectors.toList()));
}
}
}
}
}
return deletePaths;
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
HoodieInstant commitTime) {
for (FileSlice file : fileSliceList) {
String fileCommitTime = file.getDataFile().get().getCommitTime();
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
HoodieTimeline.GREATER)) {
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
return fileCommitTime;
}
}
// There is no version of this file which is <= commitTime
return null;
}
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
HoodieInstant commitTime) {
for (FileSlice file : fileSliceList) {
String fileCommitTime = file.getDataFile().get().getCommitTime();
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
HoodieTimeline.GREATER)) {
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
return fileCommitTime;
}
}
// There is no version of this file which is <= commitTime
return null;
/**
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
*/
public List<String> getDeletePaths(String partitionPath) throws IOException {
HoodieCleaningPolicy policy = config.getCleanerPolicy();
List<String> deletePaths;
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
} else {
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
}
logger.info(
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
/**
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
*/
public List<String> getDeletePaths(String partitionPath) throws IOException {
HoodieCleaningPolicy policy = config.getCleanerPolicy();
List<String> deletePaths;
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
} else {
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
}
logger.info(
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
return deletePaths;
}
return deletePaths;
}
/**
* Returns earliest commit to retain based on cleaning policy.
*/
public Optional<HoodieInstant> getEarliestCommitToRetain() {
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
int commitsRetained = config.getCleanerCommitsRetained();
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
&& commitTimeline.countInstants() > commitsRetained) {
earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
}
return earliestCommitToRetain;
/**
* Returns earliest commit to retain based on cleaning policy.
*/
public Optional<HoodieInstant> getEarliestCommitToRetain() {
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
int commitsRetained = config.getCleanerCommitsRetained();
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
&& commitTimeline.countInstants() > commitsRetained) {
earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
}
return earliestCommitToRetain;
}
}

View File

@@ -39,6 +39,12 @@ import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
@@ -46,225 +52,233 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Archiver to bound the growth of <action>.commit files
*/
public class HoodieCommitArchiveLog {
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
private final Path archiveFilePath;
private final FileSystem fs;
private final HoodieWriteConfig config;
private HoodieLogFormat.Writer writer;
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
this.fs = fs;
this.config = config;
this.archiveFilePath = HoodieArchivedTimeline
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
private final Path archiveFilePath;
private final FileSystem fs;
private final HoodieWriteConfig config;
private HoodieLogFormat.Writer writer;
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
this.fs = fs;
this.config = config;
this.archiveFilePath = HoodieArchivedTimeline
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
}
private HoodieLogFormat.Writer openWriter() {
try {
if (this.writer == null) {
return HoodieLogFormat.newWriterBuilder()
.onParentPath(archiveFilePath.getParent())
.withFileId(archiveFilePath.getName())
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
.withFs(fs)
.overBaseCommit("").build();
} else {
return this.writer;
}
} catch (InterruptedException | IOException e) {
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
}
}
private void close() {
try {
if (this.writer != null) {
this.writer.close();
}
} catch (IOException e) {
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
}
}
/**
* Check if commits need to be archived. If yes, archive commits.
*/
public boolean archiveIfRequired() {
try {
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
boolean success = true;
if (instantsToArchive.iterator().hasNext()) {
this.writer = openWriter();
log.info("Archiving instants " + instantsToArchive);
archive(instantsToArchive);
success = deleteArchivedInstants(instantsToArchive);
} else {
log.info("No Instants to archive");
}
return success;
} finally {
close();
}
}
private Stream<HoodieInstant> getInstantsToArchive() {
// TODO : rename to max/minInstantsToKeep
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
// GroupBy each action and limit each action timeline to maxCommitsToKeep
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
HoodieTimeline.ROLLBACK_ACTION));
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
.collect(Collectors.groupingBy(s -> s.getAction()))
.entrySet()
.stream()
.map(i -> {
if (i.getValue().size() > maxCommitsToKeep) {
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
} else {
return new ArrayList<HoodieInstant>();
}
})
.flatMap(i -> i.stream());
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
HoodieTimeline.LESSER_OR_EQUAL));
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
}
private HoodieLogFormat.Writer openWriter() {
try {
if(this.writer == null) {
return HoodieLogFormat.newWriterBuilder()
.onParentPath(archiveFilePath.getParent())
.withFileId(archiveFilePath.getName())
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
.withFs(fs)
.overBaseCommit("").build();
} else {
return this.writer;
}
} catch(InterruptedException | IOException e) {
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
return instants;
}
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
log.info("Deleting instants " + archivedInstants);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
boolean success = true;
for (HoodieInstant archivedInstant : archivedInstants) {
Path commitFile =
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
try {
if (fs.exists(commitFile)) {
success &= fs.delete(commitFile, false);
log.info("Archived and deleted instant file " + commitFile);
}
} catch (IOException e) {
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
e);
}
}
return success;
}
private void close() {
try {
if(this.writer != null) {
this.writer.close();
}
} catch(IOException e) {
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
}
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
log.info("Wrapper schema " + wrapperSchema.toString());
List<IndexedRecord> records = new ArrayList<>();
for (HoodieInstant hoodieInstant : instants) {
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
}
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
this.writer = writer.appendBlock(block);
} catch (Exception e) {
throw new HoodieCommitException("Failed to archive commits", e);
}
}
/**
* Check if commits need to be archived. If yes, archive commits.
*/
public boolean archiveIfRequired() {
try {
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
boolean success = true;
if (instantsToArchive.iterator().hasNext()) {
this.writer = openWriter();
log.info("Archiving instants " + instantsToArchive);
archive(instantsToArchive);
success = deleteArchivedInstants(instantsToArchive);
} else {
log.info("No Instants to archive");
}
return success;
} finally {
close();
}
public Path getArchiveFilePath() {
return archiveFilePath;
}
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline,
HoodieInstant hoodieInstant) throws IOException {
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
switch (hoodieInstant.getAction()) {
case HoodieTimeline.CLEAN_ACTION: {
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
HoodieCleanMetadata.class));
archivedMetaWrapper.setActionType(ActionType.clean.name());
break;
}
case HoodieTimeline.COMMIT_ACTION: {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
archivedMetaWrapper.setActionType(ActionType.commit.name());
break;
}
case HoodieTimeline.COMPACTION_ACTION: {
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper
.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
archivedMetaWrapper.setActionType(ActionType.compaction.name());
break;
}
case HoodieTimeline.ROLLBACK_ACTION: {
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
HoodieRollbackMetadata.class));
archivedMetaWrapper.setActionType(ActionType.rollback.name());
break;
}
case HoodieTimeline.SAVEPOINT_ACTION: {
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
HoodieSavepointMetadata.class));
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
break;
}
case HoodieTimeline.DELTA_COMMIT_ACTION: {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
archivedMetaWrapper.setActionType(ActionType.commit.name());
break;
}
}
return archivedMetaWrapper;
}
private Stream<HoodieInstant> getInstantsToArchive() {
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(
HoodieCommitMetadata hoodieCommitMetadata) {
ObjectMapper mapper = new ObjectMapper();
//Need this to ignore other public get() methods
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
mapper.convertValue(hoodieCommitMetadata,
com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
return avroMetaData;
}
// TODO : rename to max/minInstantsToKeep
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
// GroupBy each action and limit each action timeline to maxCommitsToKeep
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
HoodieTimeline.ROLLBACK_ACTION));
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
.collect(Collectors.groupingBy(s -> s.getAction()))
.entrySet()
.stream()
.map(i -> {
if (i.getValue().size() > maxCommitsToKeep) {
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
} else {
return new ArrayList<HoodieInstant>();
}
})
.flatMap(i -> i.stream());
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
HoodieTimeline.LESSER_OR_EQUAL));
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
}
return instants;
}
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
log.info("Deleting instants " + archivedInstants);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
boolean success = true;
for (HoodieInstant archivedInstant : archivedInstants) {
Path commitFile =
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
try {
if (fs.exists(commitFile)) {
success &= fs.delete(commitFile, false);
log.info("Archived and deleted instant file " + commitFile);
}
} catch (IOException e) {
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
e);
}
}
return success;
}
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
log.info("Wrapper schema " + wrapperSchema.toString());
List<IndexedRecord> records = new ArrayList<>();
for (HoodieInstant hoodieInstant : instants) {
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
}
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
this.writer = writer.appendBlock(block);
} catch(Exception e) {
throw new HoodieCommitException("Failed to archive commits", e);
}
}
public Path getArchiveFilePath() {
return archiveFilePath;
}
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant) throws IOException {
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
switch(hoodieInstant.getAction()) {
case HoodieTimeline.CLEAN_ACTION:{
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCleanMetadata.class));
archivedMetaWrapper.setActionType(ActionType.clean.name());
break;
}
case HoodieTimeline.COMMIT_ACTION:{
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
archivedMetaWrapper.setActionType(ActionType.commit.name());
break;
}
case HoodieTimeline.COMPACTION_ACTION:{
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
archivedMetaWrapper.setActionType(ActionType.compaction.name());
break;
}
case HoodieTimeline.ROLLBACK_ACTION:{
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class));
archivedMetaWrapper.setActionType(ActionType.rollback.name());
break;
}
case HoodieTimeline.SAVEPOINT_ACTION:{
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
break;
}
case HoodieTimeline.DELTA_COMMIT_ACTION:{
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
archivedMetaWrapper.setActionType(ActionType.commit.name());
break;
}
}
return archivedMetaWrapper;
}
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(HoodieCommitMetadata hoodieCommitMetadata) {
ObjectMapper mapper = new ObjectMapper();
//Need this to ignore other public get() methods
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
mapper.convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
return avroMetaData;
}
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(HoodieCompactionMetadata hoodieCompactionMetadata) {
ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper.convertValue(hoodieCompactionMetadata,
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
return avroMetaData;
}
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(
HoodieCompactionMetadata hoodieCompactionMetadata) {
ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper
.convertValue(hoodieCompactionMetadata,
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
return avroMetaData;
}
}

View File

@@ -29,116 +29,111 @@ import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.Optional;
import java.util.UUID;
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.Optional;
import java.util.UUID;
public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
private final WriteStatus status;
private final HoodieStorageWriter<IndexedRecord> storageWriter;
private final Path path;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable, String partitionPath) {
super(config, commitTime, hoodieTable);
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
status.setFileId(UUID.randomUUID().toString());
status.setPartitionPath(partitionPath);
private final WriteStatus status;
private final HoodieStorageWriter<IndexedRecord> storageWriter;
private final Path path;
private long recordsWritten = 0;
private long recordsDeleted = 0;
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
try {
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), partitionPath));
partitionMetadata.trySave(TaskContext.getPartitionId());
this.storageWriter =
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, schema);
} catch (IOException e) {
throw new HoodieInsertException(
"Failed to initialize HoodieStorageWriter for path " + path, e);
}
logger.info("New InsertHandle for partition :" + partitionPath);
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable, String partitionPath) {
super(config, commitTime, hoodieTable);
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
status.setFileId(UUID.randomUUID().toString());
status.setPartitionPath(partitionPath);
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
try {
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), partitionPath));
partitionMetadata.trySave(TaskContext.getPartitionId());
this.storageWriter =
HoodieStorageWriterFactory
.getStorageWriter(commitTime, path, hoodieTable, 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
*
* - Whether it belongs to the same partitionPath as existing records
* - Whether the current file written bytes lt max file size
*
* @return
*/
public boolean canWrite(HoodieRecord record) {
return storageWriter.canWrite() && record.getPartitionPath()
.equals(status.getPartitionPath());
/**
* Determines whether we can accept the incoming records, into the current file, depending on
*
* - Whether it belongs to the same partitionPath as existing records - Whether the current file
* written bytes lt max file size
*/
public boolean canWrite(HoodieRecord record) {
return storageWriter.canWrite() && record.getPartitionPath()
.equals(status.getPartitionPath());
}
/**
* Perform the actual writing of the given record into the backing file.
*/
public void write(HoodieRecord record) {
Optional recordMetadata = record.getData().getMetadata();
try {
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
if (avroRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
// update the new location of record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
recordsWritten++;
} else {
recordsDeleted++;
}
record.deflate();
status.markSuccess(record, recordMetadata);
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
status.markFailure(record, t, recordMetadata);
logger.error("Error writing record " + record, t);
}
}
/**
* Perform the actual writing of the given record into the backing file.
*
* @param record
*/
public void write(HoodieRecord record) {
Optional recordMetadata = record.getData().getMetadata();
try {
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
/**
* Performs actions to durably, persist the current changes and returns a WriteStatus object
*/
public WriteStatus close() {
logger.info(
"Closing the file " + status.getFileId() + " as we are done with all the records "
+ recordsWritten);
try {
storageWriter.close();
if(avroRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
// update the new location of record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
recordsWritten++;
} else {
recordsDeleted++;
}
record.deflate();
status.markSuccess(record, recordMetadata);
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
status.markFailure(record, t, recordMetadata);
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.setNumDeletes(recordsDeleted);
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
stat.setFileId(status.getFileId());
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
stat.setPath(relativePath);
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);
}
HoodieWriteStat stat = new HoodieWriteStat();
stat.setNumWrites(recordsWritten);
stat.setNumDeletes(recordsDeleted);
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
stat.setFileId(status.getFileId());
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
stat.setPath(relativePath);
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);
}
}
}

View File

@@ -24,6 +24,7 @@ import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -31,68 +32,67 @@ 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 HoodieTable<T> hoodieTable;
protected HoodieTimeline hoodieTimeline;
protected TableFileSystemView.ReadOptimizedView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView();
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
protected final String commitTime;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTable<T> hoodieTable;
protected HoodieTimeline hoodieTimeline;
protected TableFileSystemView.ReadOptimizedView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView();
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);
}
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);
}
return new Path(path.toString(),
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
}
} catch (IOException e) {
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
e);
}
}
/**
* 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;
}
public Schema getSchema() {
return schema;
}
}

View File

@@ -16,19 +16,23 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
@@ -36,197 +40,197 @@ 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;
import java.util.Optional;
@SuppressWarnings("Duplicates")
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
private WriteStatus writeStatus;
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
private HoodieStorageWriter<IndexedRecord> storageWriter;
private Path newFilePath;
private Path oldFilePath;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
public HoodieMergeHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, hoodieTable);
init(fileId, recordItr);
}
private WriteStatus writeStatus;
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
private HoodieStorageWriter<IndexedRecord> storageWriter;
private Path newFilePath;
private Path oldFilePath;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
/**
* Load the new incoming records in a map, and extract the old file path.
*/
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
this.keyToNewRecords = new HashMap<>();
public HoodieMergeHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, hoodieTable);
init(fileId, recordItr);
}
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 = fileSystemView
.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
/**
* Load the new incoming records in a map, and extract the old file path.
*/
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
this.keyToNewRecords = new HashMap<>();
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), record.getPartitionPath()));
partitionMetadata.trySave(TaskContext.getPartitionId());
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 = fileSystemView
.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
+ latestValidFilePath);
String relativePath = new Path( record.getPartitionPath() + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
newFilePath = new Path(config.getBasePath(), relativePath);
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), record.getPartitionPath()));
partitionMetadata.trySave(TaskContext.getPartitionId());
// handle cases of partial failures, for update task
if (fs.exists(newFilePath)) {
fs.delete(newFilePath, false);
}
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
+ latestValidFilePath);
String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
newFilePath = new Path(config.getBasePath(), relativePath);
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().setPath(relativePath);
}
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, hoodieTable, config, schema);
// handle cases of partial failures, for update task
if (fs.exists(newFilePath)) {
fs.delete(newFilePath, false);
}
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
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().setPath(relativePath);
}
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, hoodieTable, config, schema);
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
}
}
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
if(indexedRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
recordsWritten++;
updatedRecordsWritten++;
} else {
recordsDeleted++;
}
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord,
Optional<IndexedRecord> indexedRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
if (indexedRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
recordsWritten++;
updatedRecordsWritten++;
} else {
recordsDeleted++;
}
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return true;
} catch (Exception e) {
logger.error("Error writing record "+ hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
}
return false;
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return true;
} catch (Exception e) {
logger.error("Error writing record " + hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
}
return false;
}
/**
* 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);
boolean copyOldRecord = true;
if (keyToNewRecords.containsKey(key)) {
try {
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
/**
* 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);
boolean copyOldRecord = true;
if (keyToNewRecords.containsKey(key)) {
try {
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
.combineAndGetUpdateValue(oldRecord, schema);
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
/* ONLY WHEN
* 1) we have an update for this key AND
* 2) We are able to successfully write the the combined new value
*
* We no longer need to copy the old record over.
*/
copyOldRecord = false;
}
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);
}
}
if (copyOldRecord) {
// this should work as it is, since this is an existing record
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
+ getOldFilePath() + " to new file " + newFilePath;
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 ++;
copyOldRecord = false;
}
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);
}
}
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().setNumDeletes(recordsDeleted);
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
if (copyOldRecord) {
// this should work as it is, since this is an existing record
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
+ getOldFilePath() + " to new file " + newFilePath;
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 Path getOldFilePath() {
return oldFilePath;
}
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();
public WriteStatus getWriteStatus() {
return writeStatus;
if (storageWriter != null) {
storageWriter.close();
}
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
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;
}
}

View File

@@ -18,7 +18,6 @@ package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import java.io.Serializable;
@@ -27,8 +26,8 @@ import java.util.Map;
import java.util.stream.Collectors;
/**
* Encapsulates all the needed information about a compaction
* and make a decision whether this compaction is effective or not
* Encapsulates all the needed information about a compaction and make a decision whether this
* compaction is effective or not
*
* @see CompactionStrategy
*/

View File

@@ -22,29 +22,28 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.Serializable;
import java.util.Date;
import org.apache.spark.api.java.JavaSparkContext;
/**
* A HoodieCompactor runs compaction on a hoodie table
*/
public interface HoodieCompactor extends Serializable {
/**
* Compact the delta files with the data files
* @throws Exception
*/
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTable hoodieTable) throws Exception;
/**
* Compact the delta files with the data files
*/
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTable hoodieTable) throws Exception;
// Helper methods
default String startCompactionCommit(HoodieTable hoodieTable) {
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
activeTimeline
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
return commitTime;
}
// Helper methods
default String startCompactionCommit(HoodieTable hoodieTable) {
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
activeTimeline
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
return commitTime;
}
}

View File

@@ -16,14 +16,14 @@
package com.uber.hoodie.io.compact;
import static java.util.stream.Collectors.toList;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.CompactionWriteStat;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
@@ -36,7 +36,12 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.avro.Schema;
@@ -46,18 +51,10 @@ import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import static java.util.stream.Collectors.*;
/**
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage.
* Computes all possible compactions, passes it through a CompactionFilter and executes
* all the compactions and writes a new version of base files and make a normal commit
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
* possible compactions, passes it through a CompactionFilter and executes all the compactions and
* writes a new version of base files and make a normal commit
*
* @see HoodieCompactor
*/
@@ -80,7 +77,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
String compactionCommit = startCompactionCommit(hoodieTable);
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
List<String> partitionPaths =
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
List<CompactionOperation> operations =
@@ -89,7 +87,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getRTFileSystemView()
.getLatestFileSlices(partitionPath)
.map(s -> new CompactionOperation(s.getDataFile().get(),
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
.collect(toList()).iterator()).collect();
log.info("Total of " + operations.size() + " compactions are retrieved");
@@ -150,14 +148,15 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.COMPACTION_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.COMPACTION_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(),
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
metaClient.getBasePath(),
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
if (!scanner.iterator().hasNext()) {
return Lists.newArrayList();
}

View File

@@ -28,8 +28,8 @@ import java.util.Map;
import java.util.Optional;
/**
* CompactionStrategy which looks at total IO to be done for the compaction (read + write)
* and limits the list of compactions to be under a configured limit on the IO
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and
* limits the list of compactions to be under a configured limit on the IO
*
* @see CompactionStrategy
*/
@@ -46,7 +46,7 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
Optional::isPresent).map(Optional::get).reduce(
(size1, size2) -> size1 + size2).orElse(0L);
(size1, size2) -> size1 + size2).orElse(0L);
// Total read will be the base file + all the log files
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
// Total write will be similar to the size of the base file
@@ -62,7 +62,8 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
}
@Override
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig, List<CompactionOperation> operations) {
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<CompactionOperation> operations) {
// Iterate through the operations in order and accept operations as long as we are within the IO limit
// Preserves the original ordering of compactions
List<CompactionOperation> finalOperations = Lists.newArrayList();

View File

@@ -25,12 +25,12 @@ import java.util.List;
import java.util.Map;
/**
* Strategy for compaction. Pluggable implementation of define how compaction should be done.
* The implementations of this interface can capture the relevant metrics to order and filter
* the final list of compaction operation to run in a single compaction.
* Strategy for compaction. Pluggable implementation of define how compaction should be done. The
* implementations of this interface can capture the relevant metrics to order and filter the final
* list of compaction operation to run in a single compaction.
*
* Implementation of CompactionStrategy cannot hold any state.
* Difference instantiations can be passed in every time
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
* passed in every time
*
* @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor
* @see CompactionOperation
@@ -38,8 +38,8 @@ import java.util.Map;
public interface CompactionStrategy extends Serializable {
/**
* Callback hook when a CompactionOperation is created. Individual strategies can
* capture the metrics they need to decide on the priority.
* Callback hook when a CompactionOperation is created. Individual strategies can capture the
* metrics they need to decide on the priority.
*
* @param dataFile - Base file to compact
* @param partitionPath - Partition path
@@ -50,8 +50,8 @@ public interface CompactionStrategy extends Serializable {
List<HoodieLogFile> logFiles);
/**
* Order and Filter the list of compactions. Use the metrics captured with the
* captureMetrics to order and filter out compactions
* Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to
* order and filter out compactions
*
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
* @param operations - list of compactions collected

View File

@@ -27,8 +27,8 @@ import java.util.Optional;
import java.util.stream.Collectors;
/**
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size
* and limits the compactions within a configured IO bound
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and
* limits the compactions within a configured IO bound
*
* @see BoundedIOCompactionStrategy
* @see CompactionStrategy

View File

@@ -25,9 +25,9 @@ import java.util.List;
import java.util.Map;
/**
* UnBoundedCompactionStrategy will not change ordering or filter any compaction.
* It is a pass-through and will compact all the base files which has a log file.
* This usually means no-intelligence on compaction.
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a
* pass-through and will compact all the base files which has a log file. This usually means
* no-intelligence on compaction.
*
* @see CompactionStrategy
*/

View File

@@ -17,50 +17,50 @@
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;
}
private HoodieAvroWriteSupport writeSupport;
private CompressionCodecName compressionCodecName;
private int blockSize;
private int pageSize;
private int maxFileSize;
private Configuration hadoopConf;
public HoodieAvroWriteSupport getWriteSupport() {
return writeSupport;
}
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 CompressionCodecName getCompressionCodecName() {
return compressionCodecName;
}
public HoodieAvroWriteSupport getWriteSupport() {
return writeSupport;
}
public int getBlockSize() {
return blockSize;
}
public CompressionCodecName getCompressionCodecName() {
return compressionCodecName;
}
public int getPageSize() {
return pageSize;
}
public int getBlockSize() {
return blockSize;
}
public int getMaxFileSize() {
return maxFileSize;
}
public int getPageSize() {
return pageSize;
}
public Configuration getHadoopConf() {
return hadoopConf;
}
public int getMaxFileSize() {
return maxFileSize;
}
public Configuration getHadoopConf() {
return hadoopConf;
}
}

View File

@@ -20,6 +20,8 @@ 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 java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -30,79 +32,76 @@ 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>
* 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>
*/
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 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 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;
}
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;
}
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());
}
@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;
}
public boolean canWrite() {
return fs.getBytesWritten(file) < maxFileSize;
}
@Override public void writeAvro(String key, IndexedRecord object) throws IOException {
super.write(object);
writeSupport.add(key);
}
@Override
public void writeAvro(String key, IndexedRecord object) throws IOException {
super.write(object);
writeSupport.add(key);
}
}

View File

@@ -17,13 +17,16 @@
package com.uber.hoodie.io.storage;
import com.uber.hoodie.common.model.HoodieRecord;
import java.io.IOException;
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;
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
boolean canWrite();
void close() throws IOException;
void writeAvro(String key, R oldRecord) throws IOException;
}

View File

@@ -16,42 +16,42 @@
package com.uber.hoodie.io.storage;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
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.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
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, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema)
throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema);
}
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);
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config,
Schema schema)
throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema);
}
HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
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);
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
}
HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
}
}

View File

@@ -16,44 +16,47 @@
package com.uber.hoodie.io.storage;
import org.apache.hadoop.fs.FSDataOutputStream;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.fs.FSDataOutputStream;
/**
* 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.
* 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;
}
// 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);
@Override public synchronized void write(byte[] b, int off, int len) throws IOException {
bytesWritten.addAndGet(len);
super.write(b, off, len);
}
public SizeAwareFSDataOutputStream(FSDataOutputStream out, Runnable closeCallback)
throws IOException {
super(out);
this.closeCallback = closeCallback;
}
@Override public void write(byte[] b) throws IOException {
bytesWritten.addAndGet(b.length);
super.write(b);
}
@Override
public synchronized void write(byte[] b, int off, int len) throws IOException {
bytesWritten.addAndGet(len);
super.write(b, off, len);
}
@Override public void close() throws IOException {
super.close();
closeCallback.run();
}
@Override
public void write(byte[] b) throws IOException {
bytesWritten.addAndGet(b.length);
super.write(b);
}
public long getBytesWritten() {
return bytesWritten.get();
}
@Override
public void close() throws IOException {
super.close();
closeCallback.run();
}
public long getBytesWritten() {
return bytesWritten.get();
}
}

View File

@@ -22,7 +22,6 @@ import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -30,119 +29,122 @@ import org.apache.log4j.Logger;
* Wrapper for metrics-related operations.
*/
public class HoodieMetrics {
private HoodieWriteConfig config = null;
private String tableName = null;
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
// Some timers
public String rollbackTimerName = null;
public String cleanTimerName = null;
public String commitTimerName = null;
private Timer rollbackTimer = null;
private Timer cleanTimer = null;
private Timer commitTimer = null;
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
this.config = config;
this.tableName = tableName;
if (config.isMetricsOn()) {
Metrics.init(config);
this.rollbackTimerName = getMetricsName("timer", "rollback");
this.cleanTimerName = getMetricsName("timer", "clean");
this.commitTimerName = getMetricsName("timer", "commit");
private HoodieWriteConfig config = null;
private String tableName = null;
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
// Some timers
public String rollbackTimerName = null;
public String cleanTimerName = null;
public String commitTimerName = null;
private Timer rollbackTimer = null;
private Timer cleanTimer = null;
private Timer commitTimer = null;
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
this.config = config;
this.tableName = tableName;
if (config.isMetricsOn()) {
Metrics.init(config);
this.rollbackTimerName = getMetricsName("timer", "rollback");
this.cleanTimerName = getMetricsName("timer", "clean");
this.commitTimerName = getMetricsName("timer", "commit");
}
}
private Timer createTimer(String name) {
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
}
public Timer.Context getRollbackCtx() {
if (config.isMetricsOn() && rollbackTimer == null) {
rollbackTimer = createTimer(rollbackTimerName);
}
return rollbackTimer == null ? null : rollbackTimer.time();
}
public Timer.Context getCleanCtx() {
if (config.isMetricsOn() && cleanTimer == null) {
cleanTimer = createTimer(cleanTimerName);
}
return cleanTimer == null ? null : cleanTimer.time();
}
public Timer.Context getCommitCtx() {
if (config.isMetricsOn() && commitTimer == null) {
commitTimer = createTimer(commitTimerName);
}
return commitTimer == null ? null : commitTimer.time();
}
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs,
HoodieCommitMetadata metadata) {
if (config.isMetricsOn()) {
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
long totalFilesInsert = metadata.fetchTotalFilesInsert();
long totalFilesUpdate = metadata.fetchTotalFilesUpdated();
long totalRecordsWritten = metadata.fetchTotalRecordsWritten();
long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten();
long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten();
long totalBytesWritten = metadata.fetchTotalBytesWritten();
registerGauge(getMetricsName("commit", "duration"), durationInMs);
registerGauge(getMetricsName("commit", "totalPartitionsWritten"), totalPartitionsWritten);
registerGauge(getMetricsName("commit", "totalFilesInsert"), totalFilesInsert);
registerGauge(getMetricsName("commit", "totalFilesUpdate"), totalFilesUpdate);
registerGauge(getMetricsName("commit", "totalRecordsWritten"), totalRecordsWritten);
registerGauge(getMetricsName("commit", "totalUpdateRecordsWritten"),
totalUpdateRecordsWritten);
registerGauge(getMetricsName("commit", "totalInsertRecordsWritten"),
totalInsertRecordsWritten);
registerGauge(getMetricsName("commit", "totalBytesWritten"), totalBytesWritten);
registerGauge(getMetricsName("commit", "commitTime"), commitEpochTimeInMs);
}
}
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
if (config.isMetricsOn()) {
logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)",
durationInMs, numFilesDeleted));
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
}
}
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
if (config.isMetricsOn()) {
logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)",
durationInMs, numFilesDeleted));
registerGauge(getMetricsName("clean", "duration"), durationInMs);
registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
}
}
@VisibleForTesting
String getMetricsName(String action, String metric) {
return config == null ? null :
String.format("%s.%s.%s", tableName, action, metric);
}
void registerGauge(String metricName, final long value) {
try {
MetricRegistry registry = Metrics.getInstance().getRegistry();
registry.register(metricName, new Gauge<Long>() {
@Override
public Long getValue() {
return value;
}
});
} catch (Exception e) {
// Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system
// has some issues.
logger.error("Failed to send metrics: ", e);
}
}
private Timer createTimer(String name) {
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
}
public Timer.Context getRollbackCtx() {
if (config.isMetricsOn() && rollbackTimer == null) {
rollbackTimer = createTimer(rollbackTimerName);
}
return rollbackTimer == null ? null : rollbackTimer.time();
}
public Timer.Context getCleanCtx() {
if (config.isMetricsOn() && cleanTimer == null) {
cleanTimer = createTimer(cleanTimerName);
}
return cleanTimer == null ? null : cleanTimer.time();
}
public Timer.Context getCommitCtx() {
if (config.isMetricsOn() && commitTimer == null) {
commitTimer = createTimer(commitTimerName);
}
return commitTimer == null ? null : commitTimer.time();
}
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, HoodieCommitMetadata metadata) {
if (config.isMetricsOn()) {
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
long totalFilesInsert = metadata.fetchTotalFilesInsert();
long totalFilesUpdate = metadata.fetchTotalFilesUpdated();
long totalRecordsWritten = metadata.fetchTotalRecordsWritten();
long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten();
long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten();
long totalBytesWritten = metadata.fetchTotalBytesWritten();
registerGauge(getMetricsName("commit", "duration"), durationInMs);
registerGauge(getMetricsName("commit", "totalPartitionsWritten"), totalPartitionsWritten);
registerGauge(getMetricsName("commit", "totalFilesInsert"), totalFilesInsert);
registerGauge(getMetricsName("commit", "totalFilesUpdate"), totalFilesUpdate);
registerGauge(getMetricsName("commit", "totalRecordsWritten"), totalRecordsWritten);
registerGauge(getMetricsName("commit", "totalUpdateRecordsWritten"), totalUpdateRecordsWritten);
registerGauge(getMetricsName("commit", "totalInsertRecordsWritten"), totalInsertRecordsWritten);
registerGauge(getMetricsName("commit", "totalBytesWritten"), totalBytesWritten);
registerGauge(getMetricsName("commit", "commitTime"), commitEpochTimeInMs);
}
}
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
if (config.isMetricsOn()) {
logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)",
durationInMs, numFilesDeleted));
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
}
}
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
if (config.isMetricsOn()) {
logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)",
durationInMs, numFilesDeleted));
registerGauge(getMetricsName("clean", "duration"), durationInMs);
registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
}
}
@VisibleForTesting
String getMetricsName(String action, String metric) {
return config == null ? null :
String.format("%s.%s.%s", tableName, action, metric);
}
void registerGauge(String metricName, final long value) {
try {
MetricRegistry registry = Metrics.getInstance().getRegistry();
registry.register(metricName, new Gauge<Long>() {
@Override
public Long getValue() {
return value;
}
});
} catch (Exception e) {
// Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system
// has some issues.
logger.error("Failed to send metrics: ", e);
}
}
/**
* By default, the timer context returns duration with nano seconds.
* Convert it to millisecond.
*/
public long getDurationInMs(long ctxDuration) {
return ctxDuration / 1000000;
}
/**
* By default, the timer context returns duration with nano seconds. Convert it to millisecond.
*/
public long getDurationInMs(long ctxDuration) {
return ctxDuration / 1000000;
}
}

View File

@@ -22,16 +22,17 @@ import java.io.Closeable;
* Used for testing.
*/
public class InMemoryMetricsReporter extends MetricsReporter {
@Override
public void start() {
}
@Override
public void report() {
}
@Override
public void start() {
}
@Override
public Closeable getReporter() {
return null;
}
@Override
public void report() {
}
@Override
public Closeable getReporter() {
return null;
}
}

View File

@@ -19,65 +19,64 @@ package com.uber.hoodie.metrics;
import com.codahale.metrics.MetricRegistry;
import com.google.common.io.Closeables;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.config.HoodieMetricsConfig;
import com.uber.hoodie.exception.HoodieException;
import org.apache.commons.configuration.ConfigurationException;
import java.io.Closeable;
import org.apache.commons.configuration.ConfigurationException;
/**
* This is the main class of the metrics system.
*/
public class Metrics {
private static volatile boolean initialized = false;
private static Metrics metrics = null;
private final MetricRegistry registry;
private MetricsReporter reporter = null;
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
registry = new MetricRegistry();
private static volatile boolean initialized = false;
private static Metrics metrics = null;
private final MetricRegistry registry;
private MetricsReporter reporter = null;
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
if (reporter == null) {
throw new RuntimeException("Cannot initialize Reporter.");
}
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
registry = new MetricRegistry();
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
if (reporter == null) {
throw new RuntimeException("Cannot initialize Reporter.");
}
// reporter.start();
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
try {
reporter.report();
Closeables.close(reporter.getReporter(), true);
} catch (Exception e) {
e.printStackTrace();
}
}
});
}
public static Metrics getInstance() {
assert initialized;
return metrics;
}
public static synchronized void init(HoodieWriteConfig metricConfig) {
if (initialized) {
return;
}
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
try {
metrics = new Metrics(metricConfig);
} catch (ConfigurationException e) {
throw new HoodieException(e);
reporter.report();
Closeables.close(reporter.getReporter(), true);
} catch (Exception e) {
e.printStackTrace();
}
initialized = true;
}
}
});
}
public MetricRegistry getRegistry() {
return registry;
}
public static Metrics getInstance() {
assert initialized;
return metrics;
}
public Closeable getReporter() {
return reporter.getReporter();
public static synchronized void init(HoodieWriteConfig metricConfig) {
if (initialized) {
return;
}
try {
metrics = new Metrics(metricConfig);
} catch (ConfigurationException e) {
throw new HoodieException(e);
}
initialized = true;
}
public MetricRegistry getRegistry() {
return registry;
}
public Closeable getReporter() {
return reporter.getReporter();
}
}

View File

@@ -21,75 +21,74 @@ import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.graphite.Graphite;
import com.codahale.metrics.graphite.GraphiteReporter;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.Closeable;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Implementation of Graphite reporter, which connects to the Graphite server,
* and send metrics to that server.
* Implementation of Graphite reporter, which connects to the Graphite server, and send metrics to
* that server.
*/
public class MetricsGraphiteReporter extends MetricsReporter {
private final MetricRegistry registry;
private final GraphiteReporter graphiteReporter;
private final HoodieWriteConfig config;
private String serverHost;
private int serverPort;
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
private final MetricRegistry registry;
private final GraphiteReporter graphiteReporter;
private final HoodieWriteConfig config;
private String serverHost;
private int serverPort;
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
this.registry = registry;
this.config = config;
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
// Check the serverHost and serverPort here
this.serverHost = config.getGraphiteServerHost();
this.serverPort = config.getGraphiteServerPort();
if (serverHost == null || serverPort == 0) {
throw new RuntimeException(
String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
serverHost, serverPort));
}
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
this.registry = registry;
this.config = config;
this.graphiteReporter = createGraphiteReport();
// Check the serverHost and serverPort here
this.serverHost = config.getGraphiteServerHost();
this.serverPort = config.getGraphiteServerPort();
if (serverHost == null || serverPort == 0) {
throw new RuntimeException(
String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
serverHost, serverPort));
}
@Override
public void start() {
if (graphiteReporter != null) {
graphiteReporter.start(30, TimeUnit.SECONDS);
} else {
logger.error("Cannot start as the graphiteReporter is null.");
}
}
this.graphiteReporter = createGraphiteReport();
}
@Override
public void report() {
if (graphiteReporter != null) {
graphiteReporter.report();
} else {
logger.error("Cannot report metrics as the graphiteReporter is null.");
}
@Override
public void start() {
if (graphiteReporter != null) {
graphiteReporter.start(30, TimeUnit.SECONDS);
} else {
logger.error("Cannot start as the graphiteReporter is null.");
}
}
@Override
public Closeable getReporter() {
return graphiteReporter;
@Override
public void report() {
if (graphiteReporter != null) {
graphiteReporter.report();
} else {
logger.error("Cannot report metrics as the graphiteReporter is null.");
}
}
private GraphiteReporter createGraphiteReport() {
Graphite graphite = new Graphite(
new InetSocketAddress(serverHost, serverPort));
String reporterPrefix = config.getGraphiteMetricPrefix();
return GraphiteReporter.forRegistry(registry)
.prefixedWith(reporterPrefix)
.convertRatesTo(TimeUnit.SECONDS)
.convertDurationsTo(TimeUnit.MILLISECONDS)
.filter(MetricFilter.ALL)
.build(graphite);
}
@Override
public Closeable getReporter() {
return graphiteReporter;
}
private GraphiteReporter createGraphiteReport() {
Graphite graphite = new Graphite(
new InetSocketAddress(serverHost, serverPort));
String reporterPrefix = config.getGraphiteMetricPrefix();
return GraphiteReporter.forRegistry(registry)
.prefixedWith(reporterPrefix)
.convertRatesTo(TimeUnit.SECONDS)
.convertDurationsTo(TimeUnit.MILLISECONDS)
.filter(MetricFilter.ALL)
.build(graphite);
}
}

View File

@@ -22,15 +22,16 @@ import java.io.Closeable;
* Interface for implementing a Reporter.
*/
public abstract class MetricsReporter {
/**
* Push out metrics at scheduled intervals
*/
public abstract void start();
/**
* Deterministically push out metrics
*/
public abstract void report();
/**
* Push out metrics at scheduled intervals
*/
public abstract void start();
public abstract Closeable getReporter();
/**
* Deterministically push out metrics
*/
public abstract void report();
public abstract Closeable getReporter();
}

View File

@@ -18,7 +18,6 @@ package com.uber.hoodie.metrics;
import com.codahale.metrics.MetricRegistry;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -26,23 +25,24 @@ import org.apache.log4j.Logger;
* Factory class for creating MetricsReporter.
*/
public class MetricsReporterFactory {
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
public static MetricsReporter createReporter(HoodieWriteConfig config,
MetricRegistry registry) {
MetricsReporterType type = config.getMetricsReporterType();
MetricsReporter reporter = null;
switch (type) {
case GRAPHITE:
reporter = new MetricsGraphiteReporter(config, registry);
break;
case INMEMORY:
reporter = new InMemoryMetricsReporter();
break;
default:
logger.error("Reporter type[" + type + "] is not supported.");
break;
}
return reporter;
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
public static MetricsReporter createReporter(HoodieWriteConfig config,
MetricRegistry registry) {
MetricsReporterType type = config.getMetricsReporterType();
MetricsReporter reporter = null;
switch (type) {
case GRAPHITE:
reporter = new MetricsGraphiteReporter(config, registry);
break;
case INMEMORY:
reporter = new InMemoryMetricsReporter();
break;
default:
logger.error("Reporter type[" + type + "] is not supported.");
break;
}
return reporter;
}
}

View File

@@ -17,10 +17,10 @@
package com.uber.hoodie.metrics;
/**
* Types of the reporter. Right now we only support Graphite.
* We can include JMX and CSV in the future.
* Types of the reporter. Right now we only support Graphite. We can include JMX and CSV in the
* future.
*/
public enum MetricsReporterType {
GRAPHITE,
INMEMORY
GRAPHITE,
INMEMORY
}

View File

@@ -39,13 +39,6 @@ import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Arrays;
@@ -56,179 +49,209 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/**
* Implementation of a more real-time read-optimized Hoodie Table where
*
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
* Merge with the smallest existing file, to expand it
* Merge with the smallest existing file, to expand it
*
* UPDATES - Appends the changes to a rolling log file maintained per file Id.
* Compaction merges the log file into the base file.
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the
* log file into the base file.
*
* WARNING - MOR table type does not support nested rollbacks, every rollback
* must be followed by an attempted commit action
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an
* attempted commit action
*/
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
HoodieCopyOnWriteTable<T> {
public HoodieMergeOnReadTable(HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
super(config, metaClient);
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
public HoodieMergeOnReadTable(HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
super(config, metaClient);
}
@Override
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
HoodieAppendHandle<T> appendHandle =
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
appendHandle.doAppend();
appendHandle.close();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
.iterator();
}
@Override
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc) {
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
.filterCompletedInstants().lastInstant();
String deltaCommitsSinceTs = "0";
if (lastCompaction.isPresent()) {
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
}
@Override
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
HoodieAppendHandle<T> appendHandle =
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
appendHandle.doAppend();
appendHandle.close();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
.iterator();
int deltaCommitsSinceLastCompaction = getActiveTimeline().getDeltaCommitTimeline()
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
return Optional.empty();
}
@Override
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc) {
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
.filterCompletedInstants().lastInstant();
String deltaCommitsSinceTs = "0";
if (lastCompaction.isPresent()) {
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
}
int deltaCommitsSinceLastCompaction = getActiveTimeline().getDeltaCommitTimeline()
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
return Optional.empty();
}
logger.info("Compacting merge on read table " + config.getBasePath());
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
try {
return Optional.of(compactor.compact(jsc, config, this));
} catch (IOException e) {
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
}
logger.info("Compacting merge on read table " + config.getBasePath());
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
try {
return Optional.of(compactor.compact(jsc, config, this));
} catch (IOException e) {
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
}
}
@Override
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException {
@Override
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
throws IOException {
//At the moment, MOR table type does not support nested rollbacks
if(commits.size() > 1) {
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
}
Map<String, HoodieInstant> commitsAndCompactions =
this.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
.getInstants()
.filter(i -> commits.contains(i.getTimestamp()))
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
// Atomically un-publish all non-inflight commits
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
logger.info("Unpublished " + commits);
Long startTime = System.currentTimeMillis();
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
HoodieInstant instant = commitsAndCompactions.get(commit);
List<HoodieRollbackStat> stats = null;
switch (instant.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.COMPACTION_ACTION:
try {
logger.info("Starting to rollback Commit/Compaction " + instant);
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(results).build();
}).collect();
logger.info("Finished rollback of Commit/Compaction " + instant);
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.DELTA_COMMIT_ACTION:
try {
logger.info("Starting to rollback delta commit " + instant);
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
// read commit file and (either append delete blocks or delete file)
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
filesToDeletedStatus = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
// append rollback blocks for updates
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT).forEach(wStat -> {
HoodieLogFormat.Writer writer = null;
try {
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
.withFs(FSUtils.getFs()).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
// if update belongs to an existing log file
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
numRollbackBlocks++;
if(wStat.getNumDeletes() > 0) {
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
numRollbackBlocks++;
}
filesToNumBlocksRollback.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks);
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException("Failed to rollback for commit " + commit, io);
} finally {
try {
writer.close();
} catch (IOException io) {
throw new UncheckedIOException(io);
}
}
});
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus)
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
}).collect();
logger.info("Fnished rollback of delta commit " + instant);
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
}
return stats;
}).flatMap(x -> x.stream()).collect(Collectors.toList());
commitsAndCompactions.entrySet().stream()
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp()))
.forEach(this.getActiveTimeline()::deleteInflight);
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
return allRollbackStats;
//At the moment, MOR table type does not support nested rollbacks
if (commits.size() > 1) {
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
}
Map<String, HoodieInstant> commitsAndCompactions =
this.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
.getInstants()
.filter(i -> commits.contains(i.getTimestamp()))
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
// Atomically un-publish all non-inflight commits
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
logger.info("Unpublished " + commits);
Long startTime = System.currentTimeMillis();
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
HoodieInstant instant = commitsAndCompactions.get(commit);
List<HoodieRollbackStat> stats = null;
switch (instant.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.COMPACTION_ACTION:
try {
logger.info("Starting to rollback Commit/Compaction " + instant);
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(this.getCommitTimeline().getInstantDetails(
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
.collect(Collectors.toList()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
Map<FileStatus, Boolean> results = super
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(results).build();
}).collect();
logger.info("Finished rollback of Commit/Compaction " + instant);
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.DELTA_COMMIT_ACTION:
try {
logger.info("Starting to rollback delta commit " + instant);
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(this.getCommitTimeline().getInstantDetails(
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
.collect(Collectors.toList()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
// read commit file and (either append delete blocks or delete file)
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
filesToDeletedStatus = super
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
// append rollback blocks for updates
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
.filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT)
.forEach(wStat -> {
HoodieLogFormat.Writer writer = null;
try {
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
.withFs(FSUtils.getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME,
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
// if update belongs to an existing log file
writer.appendBlock(new HoodieCommandBlock(
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
metadata));
numRollbackBlocks++;
if (wStat.getNumDeletes() > 0) {
writer.appendBlock(new HoodieCommandBlock(
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
metadata));
numRollbackBlocks++;
}
filesToNumBlocksRollback
.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()),
numRollbackBlocks);
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException(
"Failed to rollback for commit " + commit, io);
} finally {
try {
writer.close();
} catch (IOException io) {
throw new UncheckedIOException(io);
}
}
});
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus)
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
}).collect();
logger.info("Fnished rollback of delta commit " + instant);
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
}
return stats;
}).flatMap(x -> x.stream()).collect(Collectors.toList());
commitsAndCompactions.entrySet().stream()
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(),
entry.getValue().getTimestamp()))
.forEach(this.getActiveTimeline()::deleteInflight);
logger
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
return allRollbackStats;
}
}

View File

@@ -34,7 +34,6 @@ import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieSavepointException;
import java.io.IOException;
import java.io.Serializable;
@@ -43,8 +42,6 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -55,291 +52,245 @@ import org.apache.spark.api.java.JavaSparkContext;
* Abstract implementation of a HoodieTable
*/
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient;
private static Logger logger = LogManager.getLogger(HoodieTable.class);
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
this.config = config;
this.metaClient = metaClient;
protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient;
private static Logger logger = LogManager.getLogger(HoodieTable.class);
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
this.config = config;
this.metaClient = metaClient;
}
/**
* Provides a partitioner to perform the upsert operation, based on the workload profile
*/
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
/**
* Provides a partitioner to perform the insert operation, based on the workload profile
*/
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
/**
* Return whether this HoodieTable implementation can benefit from workload profiling
*/
public abstract boolean isWorkloadProfileNeeded();
public HoodieWriteConfig getConfig() {
return config;
}
public HoodieTableMetaClient getMetaClient() {
return metaClient;
}
public FileSystem getFs() {
return metaClient.getFs();
}
/**
* Get the view of the file system for this table
*/
public TableFileSystemView getFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
}
/**
* Get the read optimized view of the file system for this table
*/
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
}
/**
* Get the real time view of the file system for this table
*/
public TableFileSystemView.RealtimeView getRTFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
}
/**
* Get the completed (commit + compaction) view of the file system for this table
*/
public TableFileSystemView getCompletedFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
}
/**
* Get only the completed (no-inflights) commit timeline
*/
public HoodieTimeline getCompletedCommitTimeline() {
return getCommitTimeline().filterCompletedInstants();
}
/**
* Get only the inflights (no-completed) commit timeline
*/
public HoodieTimeline getInflightCommitTimeline() {
return getCommitTimeline().filterInflights();
}
/**
* Get only the completed (no-inflights) clean timeline
*/
public HoodieTimeline getCompletedCleanTimeline() {
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
}
/**
* Get only the completed (no-inflights) savepoint timeline
*/
public HoodieTimeline getCompletedSavepointTimeline() {
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
}
/**
* Get the list of savepoints in this table
*/
public List<String> getSavepoints() {
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
}
/**
* Get the list of data file names savepointed
*/
public Stream<String> getSavepointedDataFiles(String savepointTime) {
if (!getSavepoints().contains(savepointTime)) {
throw new HoodieSavepointException(
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
}
/**
* Provides a partitioner to perform the upsert operation, based on the
* workload profile
*
* @return
*/
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
/**
* Provides a partitioner to perform the insert operation, based on the workload profile
*
* @return
*/
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
/**
* Return whether this HoodieTable implementation can benefit from workload
* profiling
*
* @return
*/
public abstract boolean isWorkloadProfileNeeded();
public HoodieWriteConfig getConfig() {
return config;
HoodieInstant instant =
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
HoodieSavepointMetadata metadata = null;
try {
metadata = AvroUtils.deserializeHoodieSavepointMetadata(
getActiveTimeline().getInstantDetails(instant).get());
} catch (IOException e) {
throw new HoodieSavepointException(
"Could not get savepointed data files for savepoint " + savepointTime, e);
}
return metadata.getPartitionMetadata().values().stream()
.flatMap(s -> s.getSavepointDataFile().stream());
}
public HoodieTableMetaClient getMetaClient() {
return metaClient;
public HoodieActiveTimeline getActiveTimeline() {
return metaClient.getActiveTimeline();
}
/**
* Get the commit timeline visible for this table
*/
public HoodieTimeline getCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
return getActiveTimeline().getCommitsAndCompactionsTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
public FileSystem getFs() {
return metaClient.getFs();
/**
* Get only the completed (no-inflights) compaction commit timeline
*/
public HoodieTimeline getCompletedCompactionCommitTimeline() {
return getCompactionCommitTimeline().filterCompletedInstants();
}
/**
* Get the compacted commit timeline visible for this table
*/
public HoodieTimeline getCompactionCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitsAndCompactionsTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Get the view of the file system for this table
*
* @return
*/
public TableFileSystemView getFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
/**
* Gets the commit action type
*/
public String getCommitActionType() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
}
throw new HoodieCommitException(
"Could not commit on unknown storage type " + metaClient.getTableType());
}
/**
* Get the read optimized view of the file system for this table
*
* @return
*/
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
/**
* Gets the action type for a compaction commit
*/
public String getCompactedCommitActionType() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return HoodieTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieTimeline.COMPACTION_ACTION;
}
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
/**
* Get the real time view of the file system for this table
*
* @return
*/
public TableFileSystemView.RealtimeView getRTFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
/**
* Perform the ultimate IO for a given upserted (RDD) partition
*/
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
/**
* Perform the ultimate IO for a given inserted (RDD) partition
*/
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return new HoodieCopyOnWriteTable<>(config, metaClient);
case MERGE_ON_READ:
return new HoodieMergeOnReadTable<>(config, metaClient);
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Get the completed (commit + compaction) view of the file system for this table
*
* @return
*/
public TableFileSystemView getCompletedFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
}
/**
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
* access
*/
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc);
/**
* Get only the completed (no-inflights) commit timeline
* @return
*/
public HoodieTimeline getCompletedCommitTimeline() {
return getCommitTimeline().filterCompletedInstants();
}
/**
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
*/
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
/**
* Get only the inflights (no-completed) commit timeline
* @return
*/
public HoodieTimeline getInflightCommitTimeline() {
return getCommitTimeline().filterInflights();
}
/**
* Get only the completed (no-inflights) clean timeline
* @return
*/
public HoodieTimeline getCompletedCleanTimeline() {
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
}
/**
* Get only the completed (no-inflights) savepoint timeline
* @return
*/
public HoodieTimeline getCompletedSavepointTimeline() {
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
}
/**
* Get the list of savepoints in this table
* @return
*/
public List<String> getSavepoints() {
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
}
/**
* Get the list of data file names savepointed
*
* @param savepointTime
* @return
* @throws IOException
*/
public Stream<String> getSavepointedDataFiles(String savepointTime) {
if (!getSavepoints().contains(savepointTime)) {
throw new HoodieSavepointException(
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
}
HoodieInstant instant =
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
HoodieSavepointMetadata metadata = null;
try {
metadata = AvroUtils.deserializeHoodieSavepointMetadata(
getActiveTimeline().getInstantDetails(instant).get());
} catch (IOException e) {
throw new HoodieSavepointException(
"Could not get savepointed data files for savepoint " + savepointTime, e);
}
return metadata.getPartitionMetadata().values().stream()
.flatMap(s -> s.getSavepointDataFile().stream());
}
public HoodieActiveTimeline getActiveTimeline() {
return metaClient.getActiveTimeline();
}
/**
* Get the commit timeline visible for this table
*
* @return
*/
public HoodieTimeline getCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
return getActiveTimeline().getCommitsAndCompactionsTimeline();
default:
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
}
}
/**
* Get only the completed (no-inflights) compaction commit timeline
* @return
*/
public HoodieTimeline getCompletedCompactionCommitTimeline() {
return getCompactionCommitTimeline().filterCompletedInstants();
}
/**
* Get the compacted commit timeline visible for this table
*
* @return
*/
public HoodieTimeline getCompactionCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitsAndCompactionsTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
default:
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
}
}
/**
* Gets the commit action type
* @return
*/
public String getCommitActionType() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
}
throw new HoodieCommitException(
"Could not commit on unknown storage type " + metaClient.getTableType());
}
/**
* Gets the action type for a compaction commit
* @return
*/
public String getCompactedCommitActionType() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return HoodieTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieTimeline.COMPACTION_ACTION;
}
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
}
/**
* Perform the ultimate IO for a given upserted (RDD) partition
*
* @param partition
* @param recordIterator
* @param partitioner
*/
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
/**
* Perform the ultimate IO for a given inserted (RDD) partition
*
* @param partition
* @param recordIterator
* @param partitioner
*/
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return new HoodieCopyOnWriteTable<>(config, metaClient);
case MERGE_ON_READ:
return new HoodieMergeOnReadTable<>(config, metaClient);
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Run Compaction on the table.
* Compaction arranges the data so that it is optimized for data access
*/
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc);
/**
* Clean partition paths according to cleaning policy and returns the number
* of files cleaned.
*/
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
/**
* Rollback the (inflight/committed) record changes with the given commit time.
* Four steps:
* (1) Atomically unpublish this commit
* (2) clean indexing data
* (3) clean new generated parquet files / log blocks
* (4) Finally, delete .<action>.commit or .<action>.inflight file
* @param commits
* @return
* @throws HoodieRollbackException
*/
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException;
/**
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1)
* Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files
* / log blocks (4) Finally, delete .<action>.commit or .<action>.inflight file
*/
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
throws IOException;
}

View File

@@ -20,13 +20,13 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
import org.apache.spark.api.java.JavaRDD;
/**
* Repartition input records into at least expected number of output spark partitions. It should give
* below guarantees
* - Output spark partition will have records from only one hoodie partition.
* - Average records per output spark partitions should be almost equal to (#inputRecords / #outputSparkPartitions)
* to avoid possible skews.
* Repartition input records into at least expected number of output spark partitions. It should
* give below guarantees - Output spark partition will have records from only one hoodie partition.
* - Average records per output spark partitions should be almost equal to (#inputRecords /
* #outputSparkPartitions) to avoid possible skews.
*/
public interface UserDefinedBulkInsertPartitioner<T extends HoodieRecordPayload> {
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions);
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
int outputSparkPartitions);
}

View File

@@ -20,15 +20,11 @@ package com.uber.hoodie.table;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.PairFunction;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.spark.api.java.JavaRDD;
import scala.Option;
import scala.Tuple2;
@@ -40,73 +36,76 @@ import scala.Tuple2;
*/
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
/**
* Input workload
*/
private final JavaRDD<HoodieRecord<T>> taggedRecords;
/**
* Input workload
*/
private final JavaRDD<HoodieRecord<T>> taggedRecords;
/**
* Computed workload profile
*/
private final HashMap<String, WorkloadStat> partitionPathStatMap;
/**
* Computed workload profile
*/
private final HashMap<String, WorkloadStat> partitionPathStatMap;
private final WorkloadStat globalStat;
private final WorkloadStat globalStat;
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
this.taggedRecords = taggedRecords;
this.partitionPathStatMap = new HashMap<>();
this.globalStat = new WorkloadStat();
buildProfile();
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
this.taggedRecords = taggedRecords;
this.partitionPathStatMap = new HashMap<>();
this.globalStat = new WorkloadStat();
buildProfile();
}
private void buildProfile() {
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
.mapToPair(record ->
new Tuple2<>(
new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())),
record))
.countByKey();
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
.entrySet()) {
String partitionPath = e.getKey()._1();
Long count = e.getValue();
Option<HoodieRecordLocation> locOption = e.getKey()._2();
if (!partitionPathStatMap.containsKey(partitionPath)) {
partitionPathStatMap.put(partitionPath, new WorkloadStat());
}
if (locOption.isDefined()) {
// update
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
globalStat.addUpdates(locOption.get(), count);
} else {
// insert
partitionPathStatMap.get(partitionPath).addInserts(count);
globalStat.addInserts(count);
}
}
}
private void buildProfile() {
public WorkloadStat getGlobalStat() {
return globalStat;
}
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
.mapToPair(record ->
new Tuple2<>(new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), record))
.countByKey();
public Set<String> getPartitionPaths() {
return partitionPathStatMap.keySet();
}
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e: partitionLocationCounts.entrySet()) {
String partitionPath = e.getKey()._1();
Long count = e.getValue();
Option<HoodieRecordLocation> locOption = e.getKey()._2();
public WorkloadStat getWorkloadStat(String partitionPath) {
return partitionPathStatMap.get(partitionPath);
}
if (!partitionPathStatMap.containsKey(partitionPath)){
partitionPathStatMap.put(partitionPath, new WorkloadStat());
}
if (locOption.isDefined()) {
// update
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
globalStat.addUpdates(locOption.get(), count);
} else {
// insert
partitionPathStatMap.get(partitionPath).addInserts(count);
globalStat.addInserts(count);
}
}
}
public WorkloadStat getGlobalStat() {
return globalStat;
}
public Set<String> getPartitionPaths() {
return partitionPathStatMap.keySet();
}
public WorkloadStat getWorkloadStat(String partitionPath){
return partitionPathStatMap.get(partitionPath);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
sb.append("globalStat=").append(globalStat).append(", ");
sb.append("partitionStat=").append(partitionPathStatMap);
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
sb.append("globalStat=").append(globalStat).append(", ");
sb.append("partitionStat=").append(partitionPathStatMap);
sb.append('}');
return sb.toString();
}
}

View File

@@ -17,7 +17,6 @@
package com.uber.hoodie.table;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import java.io.Serializable;
import java.util.HashMap;
@@ -25,43 +24,44 @@ import java.util.HashMap;
* Wraps stats about a single partition path.
*/
public class WorkloadStat implements Serializable {
private long numInserts = 0L;
private long numUpdates = 0L;
private long numInserts = 0L;
private HashMap<String, Long> updateLocationToCount;
private long numUpdates = 0L;
public WorkloadStat() {
updateLocationToCount = new HashMap<>();
}
private HashMap<String, Long> updateLocationToCount;
long addInserts(long numInserts) {
return this.numInserts += numInserts;
}
public WorkloadStat() {
updateLocationToCount = new HashMap<>();
}
long addUpdates(HoodieRecordLocation location, long numUpdates) {
updateLocationToCount.put(location.getFileId(), numUpdates);
return this.numUpdates += numUpdates;
}
long addInserts(long numInserts) {
return this.numInserts += numInserts;
}
public long getNumUpdates() {
return numUpdates;
}
long addUpdates(HoodieRecordLocation location, long numUpdates) {
updateLocationToCount.put(location.getFileId(), numUpdates);
return this.numUpdates += numUpdates;
}
public long getNumInserts() {
return numInserts;
}
public long getNumUpdates() {
return numUpdates;
}
public HashMap<String, Long> getUpdateLocationToCount() {
return updateLocationToCount;
}
public long getNumInserts() {
return numInserts;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadStat {");
sb.append("numInserts=").append(numInserts).append(", ");
sb.append("numUpdates=").append(numUpdates);
sb.append('}');
return sb.toString();
}
public HashMap<String, Long> getUpdateLocationToCount() {
return updateLocationToCount;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadStat {");
sb.append("numInserts=").append(numInserts).append(", ");
sb.append("numUpdates=").append(numUpdates);
sb.append('}');
return sb.toString();
}
}

View File

@@ -13,7 +13,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
# Set root logger level to DEBUG and its only appender to A1.
log4j.rootLogger=INFO, A1
# A1 is set to be a ConsoleAppender.

View File

@@ -22,13 +22,12 @@ import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -36,7 +35,6 @@ import org.apache.log4j.Logger;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.util.List;
/**
* Driver program that uses the Hoodie client with synthetic workload, and performs basic
@@ -44,75 +42,77 @@ import java.util.List;
*/
public class HoodieClientExample {
@Parameter(names={"--table-path", "-p"}, description = "path for Hoodie sample table")
private String tablePath = "file:///tmp/hoodie/sample-table";
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
private String tablePath = "file:///tmp/hoodie/sample-table";
@Parameter(names={"--table-name", "-n"}, description = "table name for Hoodie sample table")
private String tableName = "hoodie_rt";
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
private String tableName = "hoodie_rt";
@Parameter(names={"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
private static Logger logger = LogManager.getLogger(HoodieClientExample.class);
private static Logger logger = LogManager.getLogger(HoodieClientExample.class);
public static void main(String[] args) throws Exception {
HoodieClientExample cli = new HoodieClientExample();
JCommander cmd = new JCommander(cli, args);
public static void main(String[] args) throws Exception {
HoodieClientExample cli = new HoodieClientExample();
JCommander cmd = new JCommander(cli, args);
if (cli.help) {
cmd.usage();
System.exit(1);
}
cli.run();
if (cli.help) {
cmd.usage();
System.exit(1);
}
cli.run();
}
public void run() throws Exception {
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
sparkConf.setMaster("local[1]");
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
// Generator of some records to be loaded in.
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// initialize the table, if not done already
Path path = new Path(tablePath);
FileSystem fs = FSUtils.getFs();
if (!fs.exists(path)) {
HoodieTableMetaClient
.initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName,
HoodieAvroPayload.class.getName());
}
// Create the write client to write some records in
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable(tableName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
public void run() throws Exception {
/**
* Write 1 (only inserts)
*/
String newCommitTime = client.startCommit();
logger.info("Starting commit " + newCommitTime);
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
sparkConf.setMaster("local[1]");
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
client.upsert(writeRecords, newCommitTime);
// Generator of some records to be loaded in.
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// initialize the table, if not done already
Path path = new Path(tablePath);
FileSystem fs = FSUtils.getFs();
if (!fs.exists(path)) {
HoodieTableMetaClient.initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName, HoodieAvroPayload.class.getName());
}
// Create the write client to write some records in
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable(tableName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
/**
* Write 1 (only inserts)
*/
String newCommitTime = client.startCommit();
logger.info("Starting commit " + newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
client.upsert(writeRecords, newCommitTime);
/**
* Write 2 (updates)
*/
newCommitTime = client.startCommit();
logger.info("Starting commit " + newCommitTime);
records.addAll(dataGen.generateUpdates(newCommitTime, 100));
writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
client.upsert(writeRecords, newCommitTime);
}
/**
* Write 2 (updates)
*/
newCommitTime = client.startCommit();
logger.info("Starting commit " + newCommitTime);
records.addAll(dataGen.generateUpdates(newCommitTime, 100));
writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
client.upsert(writeRecords, newCommitTime);
}
}

View File

@@ -29,15 +29,6 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.SparkConf;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
@@ -49,6 +40,12 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
/**
* Utility methods to aid testing inside the HoodieClient module.
@@ -56,133 +53,142 @@ import java.util.stream.Collectors;
public class HoodieClientTestUtils {
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
List<WriteStatus> statuses = new ArrayList<>();
while (statusListItr.hasNext()) {
statuses.addAll(statusListItr.next());
}
return statuses;
}
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
Set<String> keys = new HashSet<>();
for (HoodieRecord rec: hoodieRecords) {
keys.add(rec.getRecordKey());
}
return keys;
}
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
String parentPath = basePath + "/"+ HoodieTableMetaClient.METAFOLDER_NAME;
new File(parentPath).mkdirs();
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) throws Exception {
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length) throws Exception {
String parentPath = String.format("%s/%s", basePath, partitionPath);
new File(parentPath).mkdirs();
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
public static SparkConf getSparkConfForTest(String appName) {
SparkConf sparkConf = new SparkConf()
.setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setMaster("local[1]");
return HoodieReadClient.addHoodieSupport(sparkConf);
}
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath,
HoodieTimeline commitTimeline,
List<HoodieInstant> commitsToReturn) throws IOException {
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return fileIdToFullPath;
}
public static Dataset<Row> readCommit(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String commitTime) {
HoodieInstant commitInstant =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
return sqlContext.read()
.parquet(paths.values().toArray(new String[paths.size()]))
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
} catch (Exception e) {
throw new HoodieException("Error reading commit " + commitTime, e);
}
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String lastCommitTime) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
return sqlContext.read()
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
} catch (IOException e) {
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
}
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public static Dataset<Row> read(String basePath,
SQLContext sqlContext,
FileSystem fs,
String... paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTable hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
}
return sqlContext.read()
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
} catch (Exception e) {
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
List<WriteStatus> statuses = new ArrayList<>();
while (statusListItr.hasNext()) {
statuses.addAll(statusListItr.next());
}
return statuses;
}
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
Set<String> keys = new HashSet<>();
for (HoodieRecord rec : hoodieRecords) {
keys.add(rec.getRecordKey());
}
return keys;
}
private static void fakeMetaFile(String basePath, String commitTime, String suffix)
throws IOException {
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
new File(parentPath).mkdirs();
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime,
String fileId) throws Exception {
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime,
String fileId, long length) throws Exception {
String parentPath = String.format("%s/%s", basePath, partitionPath);
new File(parentPath).mkdirs();
String path = String
.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
public static SparkConf getSparkConfForTest(String appName) {
SparkConf sparkConf = new SparkConf()
.setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setMaster("local[1]");
return HoodieReadClient.addHoodieSupport(sparkConf);
}
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath,
HoodieTimeline commitTimeline,
List<HoodieInstant> commitsToReturn) throws IOException {
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return fileIdToFullPath;
}
public static Dataset<Row> readCommit(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String commitTime) {
HoodieInstant commitInstant =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline,
Arrays.asList(commitInstant));
return sqlContext.read()
.parquet(paths.values().toArray(new String[paths.size()]))
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
} catch (Exception e) {
throw new HoodieException("Error reading commit " + commitTime, e);
}
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String lastCommitTime) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath,
commitTimeline, commitsToReturn);
return sqlContext.read()
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(
String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
} catch (IOException e) {
throw new HoodieException(
"Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
}
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public static Dataset<Row> read(String basePath,
SQLContext sqlContext,
FileSystem fs,
String... paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTable hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
hoodieTable.getMetaClient(),
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
}
return sqlContext.read()
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
} catch (Exception e) {
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
}
}
}

View File

@@ -16,9 +16,16 @@
package com.uber.hoodie.common;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
@@ -30,69 +37,64 @@ import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
/**
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR)
*/
public class HoodieMergeOnReadTestUtils {
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths) throws IOException {
JobConf jobConf = new JobConf();
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
setPropsForInputFormat(inputFormat, jobConf, schema);
return inputPaths.stream().map(path -> {
setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>();
try {
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
Void key = (Void) recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().forEach(field -> {
newRecord.set(field, values[2]);
});
records.add(newRecord.build());
}
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}).reduce((a, b) -> {
a.addAll(b);
return a;
}).get();
}
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths)
throws IOException {
JobConf jobConf = new JobConf();
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
setPropsForInputFormat(inputFormat, jobConf, schema);
return inputPaths.stream().map(path -> {
setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>();
try {
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
Void key = (Void) recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().forEach(field -> {
newRecord.set(field, values[2]);
});
records.add(newRecord.build());
}
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}).reduce((a, b) -> {
a.addAll(b);
return a;
}).get();
}
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema) {
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
Configuration conf = FSUtils.getFs().getConf();
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
conf.set("partition_columns", "datestr");
inputFormat.setConf(conf);
jobConf.addResource(conf);
}
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf,
Schema schema) {
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
Configuration conf = FSUtils.getFs().getConf();
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
conf.set("partition_columns", "datestr");
inputFormat.setConf(conf);
jobConf.addResource(conf);
}
private static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
private static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
}

View File

@@ -16,17 +16,21 @@
package com.uber.hoodie.common;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.Random;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
@@ -34,15 +38,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.Random;
import java.util.UUID;
/**
* Class to be used in tests to keep generating test inserts and updates against a corpus.
*
@@ -51,153 +46,164 @@ import java.util.UUID;
public class HoodieTestDataGenerator {
static class KeyPartition {
HoodieKey key;
String partitionPath;
HoodieKey key;
String partitionPath;
}
public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\","
+ "\"name\": \"triprec\","
+ "\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"rider\", \"type\": \"string\"},"
+ "{\"name\": \"driver\", \"type\": \"string\"},"
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},"
+ "{\"name\": \"begin_lon\", \"type\": \"double\"},"
+ "{\"name\": \"end_lat\", \"type\": \"double\"},"
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
// based on examination of sample file, the schema produces the following per record size
public static final int SIZE_PER_RECORD = 50 * 1024;
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths,
String basePath) {
for (String partitionPath : partitionPaths) {
new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath))
.trySave(0);
}
}
public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\","
+ "\"name\": \"triprec\","
+ "\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"rider\", \"type\": \"string\"},"
+ "{\"name\": \"driver\", \"type\": \"string\"},"
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},"
+ "{\"name\": \"begin_lon\", \"type\": \"double\"},"
+ "{\"name\": \"end_lat\", \"type\": \"double\"},"
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
private List<KeyPartition> existingKeysList = new ArrayList<>();
public static Schema avroSchema = HoodieAvroUtils
.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
private static Random rand = new Random(46474747);
private String[] partitionPaths = DEFAULT_PARTITION_PATHS;
// based on examination of sample file, the schema produces the following per record size
public static final int SIZE_PER_RECORD = 50 * 1024;
public HoodieTestDataGenerator(String[] partitionPaths) {
this.partitionPaths = partitionPaths;
}
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
public HoodieTestDataGenerator() {
this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"});
}
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
for (String partitionPath: partitionPaths) {
new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0);
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of
* existing keys.
*/
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = new ArrayList<>();
for (int i = 0; i < n; i++) {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
inserts.add(record);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeysList.add(kp);
}
return inserts;
}
private List<KeyPartition> existingKeysList = new ArrayList<>();
public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
private static Random rand = new Random(46474747);
private String[] partitionPaths = DEFAULT_PARTITION_PATHS;
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts);
}
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords)
throws IOException {
List<HoodieRecord> deletes = new ArrayList<>();
for (HoodieRecord existingRecord : existingRecords) {
HoodieRecord record = generateDeleteRecord(existingRecord);
deletes.add(record);
public HoodieTestDataGenerator(String[] partitionPaths) {
this.partitionPaths = partitionPaths;
}
return deletes;
}
public HoodieTestDataGenerator() {
this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"});
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
HoodieKey key = existingRecord.getKey();
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(),
key.getPartitionPath(), null, true);
return new HoodieRecord(key, payload);
}
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords)
throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (HoodieRecord baseRecord : baseRecords) {
HoodieRecord record = new HoodieRecord(baseRecord.getKey(),
generateRandomValue(baseRecord.getKey(), commitTime));
updates.add(record);
}
return updates;
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list
* of existing keys.
*/
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = new ArrayList<>();
for (int i = 0; i < n; i++) {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
inserts.add(record);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeysList.add(kp);
}
return inserts;
/**
* Generates new updates, randomly distributed across the keys above.
*/
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
updates.add(record);
}
return updates;
}
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts);
}
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords) throws IOException {
List<HoodieRecord> deletes = new ArrayList<>();
for (HoodieRecord existingRecord: existingRecords) {
HoodieRecord record = generateDeleteRecord(existingRecord);
deletes.add(record);
}
return deletes;
}
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
HoodieKey key = existingRecord.getKey();
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), null, true);
return new HoodieRecord(key, payload);
}
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (HoodieRecord baseRecord: baseRecords) {
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime));
updates.add(record);
}
return updates;
}
/**
* Generates new updates, randomly distributed across the keys above.
*/
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
updates.add(record);
}
return updates;
}
/**
* Generates a new avro record of the above schema format, retaining the key if optionally
* provided.
*/
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime,
"driver-" + commitTime, 0.0);
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
}
public static GenericRecord generateGenericRecord(String rowKey, String riderName,
String driverName, double timestamp) {
GenericRecord rec = new GenericData.Record(avroSchema);
rec.put("_row_key", rowKey);
rec.put("timestamp", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("begin_lat", rand.nextDouble());
rec.put("begin_lon", rand.nextDouble());
rec.put("end_lat", rand.nextDouble());
rec.put("end_lon", rand.nextDouble());
rec.put("fare", rand.nextDouble() * 100);
return rec;
}
public static void createCommitFile(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
try {
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(
StandardCharsets.UTF_8)));
} finally {
os.close();
}
/**
* Generates a new avro record of the above schema format, retaining the key if optionally
* provided.
*/
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime)
throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime,
"driver-" + commitTime, 0.0);
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(),
TRIP_EXAMPLE_SCHEMA);
}
public static GenericRecord generateGenericRecord(String rowKey, String riderName,
String driverName, double timestamp) {
GenericRecord rec = new GenericData.Record(avroSchema);
rec.put("_row_key", rowKey);
rec.put("timestamp", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("begin_lat", rand.nextDouble());
rec.put("begin_lon", rand.nextDouble());
rec.put("end_lat", rand.nextDouble());
rec.put("end_lon", rand.nextDouble());
rec.put("fare", rand.nextDouble() * 100);
return rec;
}
public static void createCommitFile(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
try {
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(
StandardCharsets.UTF_8)));
} finally {
os.close();
}
}
public static void createSavepointFile(String basePath, String commitTime) throws IOException {
Path commitFile =
@@ -215,7 +221,7 @@ public class HoodieTestDataGenerator {
}
}
public String[] getPartitionPaths() {
return partitionPaths;
}
public String[] getPartitionPaths() {
return partitionPaths;
}
}

View File

@@ -17,174 +17,182 @@
package com.uber.hoodie.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.util.HashMap;
import java.util.List;
import java.util.Map.Entry;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
import java.io.*;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
/**
* Example row change event based on some example data used by testcases. The data avro schema is
* src/test/resources/schema1.
*/
public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayload> {
private transient static final ObjectMapper mapper = new ObjectMapper();
private String partitionPath;
private String rowKey;
private byte[] jsonDataCompressed;
private int dataSize;
private boolean isDeleted;
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
String schemaStr, Boolean isDeleted) throws IOException {
if(jsonData.isPresent()) {
this.jsonDataCompressed = compressData(jsonData.get());
this.dataSize = jsonData.get().length();
}
this.rowKey = rowKey;
this.partitionPath = partitionPath;
this.isDeleted = isDeleted;
private transient static final ObjectMapper mapper = new ObjectMapper();
private String partitionPath;
private String rowKey;
private byte[] jsonDataCompressed;
private int dataSize;
private boolean isDeleted;
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
String schemaStr, Boolean isDeleted) throws IOException {
if (jsonData.isPresent()) {
this.jsonDataCompressed = compressData(jsonData.get());
this.dataSize = jsonData.get().length();
}
this.rowKey = rowKey;
this.partitionPath = partitionPath;
this.isDeleted = isDeleted;
}
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
String schemaStr)throws IOException {
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
String schemaStr) throws IOException {
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
}
public TestRawTripPayload(String jsonData) throws IOException {
this.jsonDataCompressed = compressData(jsonData);
this.dataSize = jsonData.length();
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
this.rowKey = jsonRecordMap.get("_row_key").toString();
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
this.isDeleted = false;
}
public String getPartitionPath() {
return partitionPath;
}
@Override
public TestRawTripPayload preCombine(TestRawTripPayload another) {
return another;
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
throws IOException {
return this.getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (isDeleted) {
return Optional.empty();
} else {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
}
public TestRawTripPayload(String jsonData) throws IOException {
this.jsonDataCompressed = compressData(jsonData);
this.dataSize = jsonData.length();
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
this.rowKey = jsonRecordMap.get("_row_key").toString();
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
this.isDeleted = false;
}
public String getPartitionPath() {
return partitionPath;
@Override
public Optional<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
// that are processed. Let the time-bucket for this row change event be 1506582000.
Map<String, String> metadataMap = new HashMap<>();
metadataMap.put("InputRecordCount_1506582000", "2");
return Optional.of(metadataMap);
}
public String getRowKey() {
return rowKey;
}
public String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
}
private byte[] compressData(String jsonData) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DeflaterOutputStream dos =
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
try {
dos.write(jsonData.getBytes());
} finally {
dos.flush();
dos.close();
}
return baos.toByteArray();
}
@Override public TestRawTripPayload preCombine(TestRawTripPayload another) {
return another;
}
private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
StringWriter sw = new StringWriter(dataSize);
IOUtils.copy(iis, sw);
return sw.toString();
}
@Override public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
return this.getInsertValue(schema);
}
/**
* A custom {@link WriteStatus} that merges passed metadata key value map to {@code
* WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
*/
public static class MetadataMergeWriteStatus extends WriteStatus {
@Override public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if(isDeleted){
return Optional.empty();
} else {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
private Map<String, String> mergedMetadataMap = new HashMap<>();
@Override
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
super.markSuccess(record, recordMetadata);
if (recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
@Override
public Optional<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
// that are processed. Let the time-bucket for this row change event be 1506582000.
Map<String, String> metadataMap = new HashMap<>();
metadataMap.put("InputRecordCount_1506582000", "2");
return Optional.of(metadataMap);
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> recordMetadata) {
super.markFailure(record, t, recordMetadata);
if (recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
public String getRowKey() {
return rowKey;
public static Map<String, String> mergeMetadataForWriteStatuses(
List<WriteStatus> writeStatuses) {
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
for (WriteStatus writeStatus : writeStatuses) {
MetadataMergeWriteStatus.mergeMetadataMaps(
((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(),
allWriteStatusMergedMetadataMap);
}
return allWriteStatusMergedMetadataMap;
}
public String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
private static void mergeMetadataMaps(Map<String, String> mergeFromMap,
Map<String, String> mergeToMap) {
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
String key = entry.getKey();
if (!mergeToMap.containsKey(key)) {
mergeToMap.put(key, "0");
}
mergeToMap
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
}
}
private byte[] compressData(String jsonData) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DeflaterOutputStream dos =
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
try {
dos.write(jsonData.getBytes());
} finally {
dos.flush();
dos.close();
}
return baos.toByteArray();
private Map<String, String> getMergedMetadataMap() {
return mergedMetadataMap;
}
private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
StringWriter sw = new StringWriter(dataSize);
IOUtils.copy(iis, sw);
return sw.toString();
}
/**
* A custom {@link WriteStatus} that merges passed metadata key value map
* to {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
*/
public static class MetadataMergeWriteStatus extends WriteStatus {
private Map<String, String> mergedMetadataMap = new HashMap<>();
@Override
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
super.markSuccess(record, recordMetadata);
if(recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
@Override
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> recordMetadata) {
super.markFailure(record, t, recordMetadata);
if(recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
public static Map<String, String> mergeMetadataForWriteStatuses(List<WriteStatus> writeStatuses) {
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
for (WriteStatus writeStatus : writeStatuses) {
MetadataMergeWriteStatus.mergeMetadataMaps(
((MetadataMergeWriteStatus)writeStatus).getMergedMetadataMap(),
allWriteStatusMergedMetadataMap);
}
return allWriteStatusMergedMetadataMap;
}
private static void mergeMetadataMaps(Map<String, String> mergeFromMap, Map<String, String> mergeToMap) {
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
String key = entry.getKey();
if(!mergeToMap.containsKey(key)) {
mergeToMap.put(key, "0");
}
mergeToMap
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
}
}
private Map<String, String> getMergedMetadataMap() {
return mergedMetadataMap;
}
private static String addStrsAsInt(String a, String b) {
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
}
private static String addStrsAsInt(String a, String b) {
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
}
}
}

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.config;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import com.google.common.collect.Maps;
import com.uber.hoodie.config.HoodieWriteConfig.Builder;
@@ -29,6 +29,7 @@ import java.util.Properties;
import org.junit.Test;
public class HoodieWriteConfigTest {
@Test
public void testPropertyLoading() throws IOException {
Builder builder = HoodieWriteConfig.newBuilder().withPath("/tmp");
@@ -46,13 +47,14 @@ public class HoodieWriteConfigTest {
HoodieWriteConfig config = builder.build();
assertEquals(config.getMaxCommitsToKeep(), 5);
assertEquals(config.getMinCommitsToKeep(), 2);
}
}
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params) throws IOException {
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params)
throws IOException {
Properties properties = new Properties();
properties.putAll(params);
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
properties.store(outStream, "Saved on " + new Date(System.currentTimeMillis()));
return outStream;
}
}
}

View File

@@ -16,105 +16,103 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import static org.junit.Assert.fail;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.Path;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import static org.junit.Assert.fail;
public class TestUpdateMapFunction {
private String basePath = null;
@Before
public void init() throws Exception {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
private String basePath = null;
@Before
public void init() throws Exception {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
}
@Test
public void testSchemaEvolutionOnUpdate() throws Exception {
// Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
String recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 =
"{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 =
"{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
FSUtils.getFs().create(commitFile);
// Now try an update with an evolved schema
// Evolved schema does not have guarantee on preserving the original field ordering
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId);
table = new HoodieCopyOnWriteTable(config, metadata);
// New content with values for the newly added field
recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
records = new ArrayList<>();
rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 =
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1);
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
records.add(record1);
try {
table.handleUpdate("101", fileId, records.iterator());
} catch (ClassCastException e) {
fail(
"UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt");
}
}
@Test
public void testSchemaEvolutionOnUpdate() throws Exception {
// Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
String recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 =
"{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 =
"{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
FSUtils.getFs().create(commitFile);
// Now try an update with an evolved schema
// Evolved schema does not have guarantee on preserving the original field ordering
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId);
table = new HoodieCopyOnWriteTable(config, metadata);
// New content with values for the newly added field
recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
records = new ArrayList<>();
rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 =
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1);
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
records.add(record1);
try {
table.handleUpdate("101", fileId, records.iterator());
} catch (ClassCastException e) {
fail(
"UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt");
}
}
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
// Prepare the AvroParquetIO
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
}
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
// Prepare the AvroParquetIO
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
}
}

View File

@@ -16,32 +16,31 @@
package com.uber.hoodie.index;
import com.uber.hoodie.config.HoodieWriteConfig;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.hbase.HBaseIndex;
import org.junit.Test;
import static org.junit.Assert.*;
public class TestHoodieIndex {
@Test
public void testCreateIndex() throws Exception {
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
// Different types
HoodieWriteConfig config = clientConfigBuilder.withPath("")
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build())
.build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex);
config = clientConfigBuilder.withPath("").withIndexConfig(
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex);
config = clientConfigBuilder.withPath("")
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex);
}
@Test
public void testCreateIndex() throws Exception {
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
// Different types
HoodieWriteConfig config = clientConfigBuilder.withPath("")
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build())
.build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex);
config = clientConfigBuilder.withPath("").withIndexConfig(
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex);
config = clientConfigBuilder.withPath("")
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex);
}
}

View File

@@ -18,28 +18,39 @@
package com.uber.hoodie.index.bloom;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.index.bloom.BloomIndexFileInfo;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.bloom.HoodieBloomIndexCheckFunction;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.storage.HoodieParquetConfig;
import com.uber.hoodie.io.storage.HoodieParquetWriter;
import com.uber.hoodie.table.HoodieTable;
import java.io.File;
import java.io.IOException;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
@@ -47,11 +58,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -59,464 +67,489 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;
import scala.Tuple2;
import java.io.File;
import java.io.IOException;
import java.text.SimpleDateFormat;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.*;
public class TestHoodieBloomIndex {
private JavaSparkContext jsc = null;
private String basePath = null;
private transient final FileSystem fs;
private String schemaStr;
private Schema schema;
public TestHoodieBloomIndex() throws Exception {
fs = FSUtils.getFs();
}
private JavaSparkContext jsc = null;
private String basePath = null;
private transient final FileSystem fs;
private String schemaStr;
private Schema schema;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieBloomIndex"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
// We have some records to be tagged (two different partitions)
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
}
public TestHoodieBloomIndex() throws Exception {
fs = FSUtils.getFs();
}
@Test
public void testLoadUUIDsInMemory() throws IOException {
// Create one RDD of hoodie record
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieBloomIndex"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
// We have some records to be tagged (two different partitions)
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
}
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
@Test
public void testLoadUUIDsInMemory() throws IOException {
// Create one RDD of hoodie record
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
// Load to memory
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
JavaRDD<HoodieRecord> recordRDD = jsc
.parallelize(Arrays.asList(record1, record2, record3, record4));
Map<String, Iterable<String>> map = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
.groupByKey().collectAsMap();
assertEquals(map.size(), 2);
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
assertEquals(list1.size(), 3);
assertEquals(list2.size(), 1);
}
// Load to memory
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
Map<String, Iterable<String>> map = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
.groupByKey().collectAsMap();
assertEquals(map.size(), 2);
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
assertEquals(list1.size(), 3);
assertEquals(list2.size(), 1);
}
// Create some partitions, and put some files
// "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
new File(basePath + "/2016/01/21").mkdirs();
new File(basePath + "/2016/04/01").mkdirs();
new File(basePath + "/2015/03/12").mkdirs();
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
TestRawTripPayload rowChange1 = new TestRawTripPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
// Create some partitions, and put some files
// "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
new File(basePath + "/2016/01/21").mkdirs();
new File(basePath + "/2016/04/01").mkdirs();
new File(basePath + "/2015/03/12").mkdirs();
TestRawTripPayload rowChange1 = new TestRawTripPayload(
"{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record1 = new HoodieRecord(
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(
"{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record2 = new HoodieRecord(
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(
"{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record3 = new HoodieRecord(
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(
"{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record4 = new HoodieRecord(
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
writeParquetFile("2016/04/01","2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, false);
writeParquetFile("2015/03/12","1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, false);
writeParquetFile("2015/03/12","3_0_20150312101010.parquet", Arrays.asList(record1), schema, null, false);
writeParquetFile("2015/03/12","4_0_20150312101010.parquet", Arrays.asList(record2, record3, record4), schema, null, false);
writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null,
false);
writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null,
false);
writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema,
null, false);
writeParquetFile("2015/03/12", "4_0_20150312101010.parquet",
Arrays.asList(record2, record3, record4), schema, null, false);
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
// Still 0, as no valid commit
assertEquals(filesList.size(), 0);
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
// Still 0, as no valid commit
assertEquals(filesList.size(), 0);
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
filesList = index.loadInvolvedFiles(partitions, table);
assertEquals(filesList.size(), 4);
// these files will not have the key ranges
assertNull(filesList.get(0)._2().getMaxRecordKey());
assertNull(filesList.get(0)._2().getMinRecordKey());
assertFalse(filesList.get(1)._2().hasKeyRanges());
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
assertNotNull(filesList.get(2)._2().getMinRecordKey());
assertTrue(filesList.get(3)._2().hasKeyRanges());
filesList = index.loadInvolvedFiles(partitions, table);
assertEquals(filesList.size(), 4);
// these files will not have the key ranges
assertNull(filesList.get(0)._2().getMaxRecordKey());
assertNull(filesList.get(0)._2().getMinRecordKey());
assertFalse(filesList.get(1)._2().hasKeyRanges());
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
assertNotNull(filesList.get(2)._2().getMinRecordKey());
assertTrue(filesList.get(3)._2().hasKeyRanges());
// no longer sorted, but should have same files.
// no longer sorted, but should have same files.
List<Tuple2<String, BloomIndexFileInfo>> expected = Arrays.asList(
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")),
new Tuple2<>("2015/03/12",new BloomIndexFileInfo("1_0_20150312101010.parquet")),
new Tuple2<>("2015/03/12",new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")),
new Tuple2<>("2015/03/12",new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"))
);
assertEquals(expected, filesList);
}
List<Tuple2<String, BloomIndexFileInfo>> expected = Arrays.asList(
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")),
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1_0_20150312101010.parquet")),
new Tuple2<>("2015/03/12",
new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")),
new Tuple2<>("2015/03/12",
new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"))
);
assertEquals(expected, filesList);
}
@Test
public void testRangePruning() {
@Test
public void testRangePruning() {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(
new BloomIndexFileInfo("f1"),
new BloomIndexFileInfo("f2", "000", "000"),
new BloomIndexFileInfo("f3", "001", "003"),
new BloomIndexFileInfo("f4", "002", "007"),
new BloomIndexFileInfo("f5", "009", "010")
));
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(
new BloomIndexFileInfo("f1"),
new BloomIndexFileInfo("f2", "000", "000"),
new BloomIndexFileInfo("f3", "001", "003"),
new BloomIndexFileInfo("f4", "002", "007"),
new BloomIndexFileInfo("f5", "009", "010")
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc
.parallelize(Arrays.asList(
new Tuple2<>("2017/10/22", "003"),
new Tuple2<>("2017/10/22", "002"),
new Tuple2<>("2017/10/22", "005"),
new Tuple2<>("2017/10/22", "004")
))
.mapToPair(t -> t);
List<Tuple2<String, Tuple2<String, HoodieKey>>> comparisonKeyList = index
.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
.collect();
assertEquals(10, comparisonKeyList.size());
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
.collect(Collectors.groupingBy(
t -> t._2()._2().getRecordKey(),
Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()
)
));
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc
.parallelize(Arrays.asList(
new Tuple2<>("2017/10/22","003"),
new Tuple2<>("2017/10/22","002"),
new Tuple2<>("2017/10/22","005"),
new Tuple2<>("2017/10/22","004")
))
.mapToPair(t -> t);
assertEquals(4, recordKeyToFileComps.size());
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002"));
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("003"));
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("004"));
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("005"));
}
@Test
public void testCheckUUIDsAgainstOneFile()
throws IOException, InterruptedException, ClassNotFoundException {
// Create some records to use
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
// We write record1, record2 to a parquet file, but the bloom filter contains (record1, record2, record3).
BloomFilter filter = new BloomFilter(10000, 0.0000001);
filter.add(record3.getRecordKey());
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema,
filter, true);
// The bloom filter contains 3 records
assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.getRecordKey()));
assertTrue(filter.mightContain(record3.getRecordKey()));
assertFalse(filter.mightContain(record4.getRecordKey()));
// Compare with file
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
record3.getRecordKey(), record4.getRecordKey());
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids,
new Path(basePath + "/2016/01/31/" + filename));
assertEquals(results.size(), 2);
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")
|| results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0"));
// TODO(vc): Need more coverage on actual filenames
//assertTrue(results.get(0)._2().equals(filename));
//assertTrue(results.get(1)._2().equals(filename));
}
@Test
public void testTagLocationWithEmptyRDD() throws Exception {
// We have some records to be tagged (two different partitions)
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
try {
bloomIndex.tagLocation(recordRDD, table);
} catch (IllegalArgumentException e) {
fail(
"EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
}
}
List<Tuple2<String, Tuple2<String, HoodieKey>>> comparisonKeyList = index
.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
.collect();
@Test
public void testTagLocation() throws Exception {
// We have some records to be tagged (two different partitions)
assertEquals(10, comparisonKeyList.size());
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
.collect(Collectors.groupingBy(
t -> t._2()._2().getRecordKey(),
Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()
)
));
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
JavaRDD<HoodieRecord> recordRDD = jsc
.parallelize(Arrays.asList(record1, record2, record3, record4));
assertEquals(4, recordKeyToFileComps.size());
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002"));
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("003"));
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("004"));
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("005"));
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// Should not find any files
for (HoodieRecord record : taggedRecordRDD.collect()) {
assertTrue(!record.isCurrentLocationKnown());
}
@Test
public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException, ClassNotFoundException {
// We create three parquet file, each having one record. (two different partitions)
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// Create some records to use
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
// We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// We write record1, record2 to a parquet file, but the bloom filter contains (record1, record2, record3).
BloomFilter filter = new BloomFilter(10000, 0.0000001);
filter.add(record3.getRecordKey());
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, filter, true);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
} else if (record.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(!record.isCurrentLocationKnown());
} else if (record.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
}
}
}
// The bloom filter contains 3 records
assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.getRecordKey()));
assertTrue(filter.mightContain(record3.getRecordKey()));
assertFalse(filter.mightContain(record4.getRecordKey()));
@Test
public void testCheckExists() throws Exception {
// We have some records to be tagged (two different partitions)
// Compare with file
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
record3.getRecordKey(), record4.getRecordKey());
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
HoodieRecord record1 = new HoodieRecord(key1, rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
HoodieRecord record3 = new HoodieRecord(key3, rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids,
new Path(basePath + "/2016/01/31/" + filename));
assertEquals(results.size(), 2);
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")
|| results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0"));
// TODO(vc): Need more coverage on actual filenames
//assertTrue(results.get(0)._2().equals(filename));
//assertTrue(results.get(1)._2().equals(filename));
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex
.fetchRecordLocation(keysRDD, table);
// Should not find any files
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
assertTrue(!record._2.isPresent());
}
@Test
public void testTagLocationWithEmptyRDD() throws Exception {
// We have some records to be tagged (two different partitions)
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// We create three parquet file, each having one record. (two different partitions)
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
// We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
try {
bloomIndex.tagLocation(recordRDD, table);
} catch (IllegalArgumentException e) {
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
}
// Check results
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path1 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename1), FSUtils.getFileId(path1.getName()));
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path2 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(!record._2.isPresent());
} else if (record._1.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path3 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path3.getName()));
}
}
}
@Test
public void testTagLocation() throws Exception {
// We have some records to be tagged (two different partitions)
@Test
public void testBloomFilterFalseError() throws IOException, InterruptedException {
// We have two hoodie records
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
// We write record1 to a parquet file, using a bloom filter having both records
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
BloomFilter filter = new BloomFilter(10000, 0.0000001);
filter.add(record2.getRecordKey());
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, filter, true);
assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.getRecordKey()));
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// We do the tag
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Should not find any files
for (HoodieRecord record : taggedRecordRDD.collect()) {
assertTrue(!record.isCurrentLocationKnown());
}
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// We create three parquet file, each having one record. (two different partitions)
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
} else if (record.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(!record.isCurrentLocationKnown());
} else if (record.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
}
}
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename)));
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertFalse(record.isCurrentLocationKnown());
}
}
}
@Test
public void testCheckExists() throws Exception {
// We have some records to be tagged (two different partitions)
private String writeParquetFile(String partitionPath, List<HoodieRecord> records, Schema schema,
BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
Thread.sleep(1000);
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
String fileId = UUID.randomUUID().toString();
String filename = FSUtils.makeDataFileName(commitTime, 1, fileId);
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
HoodieRecord record1 = new HoodieRecord(key1, rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
HoodieRecord record3 = new HoodieRecord(key3, rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime);
}
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
private String writeParquetFile(String partitionPath, String filename, List<HoodieRecord> records,
Schema schema,
BloomFilter filter, boolean createCommitTime) throws IOException {
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
// Should not find any files
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
assertTrue(!record._2.isPresent());
}
// We create three parquet file, each having one record. (two different partitions)
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
// Check results
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path1 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename1), FSUtils.getFileId(path1.getName()));
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path2 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(!record._2.isPresent());
} else if (record._1.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path3 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path3.getName()));
}
}
if (filter == null) {
filter = new BloomFilter(10000, 0.0000001);
}
@Test
public void testBloomFilterFalseError() throws IOException, InterruptedException {
// We have two hoodie records
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
// We write record1 to a parquet file, using a bloom filter having both records
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
BloomFilter filter = new BloomFilter(10000, 0.0000001);
filter.add(record2.getRecordKey());
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, filter, true);
assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.getRecordKey()));
// We do the tag
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename)));
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertFalse(record.isCurrentLocationKnown());
}
}
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
new AvroSchemaConverter().convert(schema), schema, filter);
String commitTime = FSUtils.getCommitTime(filename);
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
new Configuration());
HoodieParquetWriter writer = new HoodieParquetWriter(
commitTime,
new Path(basePath + "/" + partitionPath + "/" + filename),
config,
schema);
int seqId = 1;
for (HoodieRecord record : records) {
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
HoodieAvroUtils
.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(),
filename);
writer.writeAvro(record.getRecordKey(), avroRecord);
filter.add(record.getRecordKey());
}
writer.close();
private String writeParquetFile(String partitionPath, List<HoodieRecord> records, Schema schema,
BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
Thread.sleep(1000);
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
String fileId = UUID.randomUUID().toString();
String filename = FSUtils.makeDataFileName(commitTime, 1, fileId);
return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime);
if (createCommitTime) {
// Also make sure the commit is valid
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
new File(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit")
.createNewFile();
}
return filename;
}
private String writeParquetFile(String partitionPath, String filename, List<HoodieRecord> records, Schema schema,
BloomFilter filter, boolean createCommitTime) throws IOException {
if (filter == null) {
filter = new BloomFilter(10000, 0.0000001);
}
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
String commitTime = FSUtils.getCommitTime(filename);
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration());
HoodieParquetWriter writer = new HoodieParquetWriter(
commitTime,
new Path(basePath + "/" + partitionPath + "/" + filename),
config,
schema);
int seqId = 1;
for (HoodieRecord record : records) {
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
writer.writeAvro(record.getRecordKey(), avroRecord);
filter.add(record.getRecordKey());
}
writer.close();
if (createCommitTime) {
// Also make sure the commit is valid
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
}
return filename;
@After
public void clean() {
if (jsc != null) {
jsc.stop();
}
@After
public void clean() {
if (jsc != null) {
jsc.stop();
}
if (basePath != null) {
new File(basePath).delete();
}
if (basePath != null) {
new File(basePath).delete();
}
}
}

View File

@@ -16,9 +16,11 @@
package com.uber.hoodie.io;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -29,6 +31,11 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
@@ -37,197 +44,196 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestHoodieCommitArchiveLog {
private String basePath;
private FileSystem fs;
@Before
public void init() throws Exception {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
fs = FSUtils.getFs();
private String basePath;
private FileSystem fs;
@Before
public void init() throws Exception {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
fs = FSUtils.getFs();
}
@Test
public void testArchiveEmptyDataset() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
}
@Test
public void testArchiveDatasetWithArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
.forTable("test-trip-table").build();
HoodieTestUtils.init(basePath);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
HoodieTestUtils.createCleanFiles(basePath, "100");
HoodieTestUtils.createCleanFiles(basePath, "101");
HoodieTestUtils.createCleanFiles(basePath, "102");
HoodieTestUtils.createCleanFiles(basePath, "103");
HoodieTestUtils.createCleanFiles(basePath, "104");
HoodieTestUtils.createCleanFiles(basePath, "105");
//reload the timeline and get all the commmits before archive
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline()
.filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
assertTrue(archiveLog.archiveIfRequired());
//reload the timeline and remove the remaining commits
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline()
.filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
HoodieArchivedMetaEntry.getClassSchema(), false);
int archivedRecordsCount = 0;
List<IndexedRecord> readRecords = new ArrayList<>();
//read the avro blocks and validate the number of records written in each avro block
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
List<IndexedRecord> records = blk.getRecords();
readRecords.addAll(records);
assertEquals("Archived and read records for each block are same", 8, records.size());
archivedRecordsCount += records.size();
}
assertEquals("Total archived records and total read records are the same count", 8,
archivedRecordsCount);
@Test
public void testArchiveEmptyDataset() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
}
//make sure the archived commits are the same as the (originalcommits - commitsleft)
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
return r.get("commitTime").toString();
}).collect(Collectors.toList());
Collections.sort(readCommits);
@Test
public void testArchiveDatasetWithArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
.forTable("test-trip-table").build();
HoodieTestUtils.init(basePath);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
assertEquals(
"Read commits map should match the originalCommits - commitsLoadedFromArchival",
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
readCommits);
}
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
@Test
public void testArchiveDatasetWithNoArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
HoodieTestUtils.createCleanFiles(basePath, "100");
HoodieTestUtils.createCleanFiles(basePath, "101");
HoodieTestUtils.createCleanFiles(basePath, "102");
HoodieTestUtils.createCleanFiles(basePath, "103");
HoodieTestUtils.createCleanFiles(basePath, "104");
HoodieTestUtils.createCleanFiles(basePath, "105");
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
.filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
timeline.countInstants());
}
//reload the timeline and get all the commmits before archive
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
@Test
public void testArchiveCommitSafety() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
.filterCompletedInstants();
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("100"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("101"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("102"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("103"));
}
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
@Test
public void testArchiveCommitSavepointNoHole() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createSavepointFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
assertTrue(archiveLog.archiveIfRequired());
//reload the timeline and remove the remaining commits
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema(), false);
int archivedRecordsCount = 0;
List<IndexedRecord> readRecords = new ArrayList<>();
//read the avro blocks and validate the number of records written in each avro block
while(reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
List<IndexedRecord> records = blk.getRecords();
readRecords.addAll(records);
assertEquals("Archived and read records for each block are same", 8, records.size());
archivedRecordsCount += records.size();
}
assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount);
//make sure the archived commits are the same as the (originalcommits - commitsleft)
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord)r).map(r -> {
return r.get("commitTime").toString();
}).collect(Collectors.toList());
Collections.sort(readCommits);
assertEquals(
"Read commits map should match the originalCommits - commitsLoadedFromArchival",
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
readCommits);
}
@Test
public void testArchiveDatasetWithNoArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
timeline.countInstants());
}
@Test
public void testArchiveCommitSafety() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("100"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("101"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("102"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("103"));
}
@Test
public void testArchiveCommitSavepointNoHole() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createSavepointFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals(
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)",
5, timeline.countInstants());
assertTrue("Archived commits should always be safe",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")));
assertTrue("Archived commits should always be safe",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
assertTrue("Archived commits should always be safe",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
}
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
.filterCompletedInstants();
assertEquals(
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)",
5, timeline.countInstants());
assertTrue("Archived commits should always be safe",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")));
assertTrue("Archived commits should always be safe",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
assertTrue("Archived commits should always be safe",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
}
}

View File

@@ -16,7 +16,9 @@
package com.uber.hoodie.io;
import com.uber.hoodie.HoodieReadClient;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils;
@@ -34,13 +36,16 @@ import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
@@ -48,161 +53,154 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestHoodieCompactor {
private transient JavaSparkContext jsc = null;
private String basePath = null;
private HoodieCompactor compactor;
private transient HoodieTestDataGenerator dataGen = null;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
private transient JavaSparkContext jsc = null;
private String basePath = null;
private HoodieCompactor compactor;
private transient HoodieTestDataGenerator dataGen = null;
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
dataGen = new HoodieTestDataGenerator();
compactor = new HoodieRealtimeTableCompactor();
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
dataGen = new HoodieTestDataGenerator();
compactor = new HoodieRealtimeTableCompactor();
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
if (jsc != null) {
jsc.stop();
}
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder().build();
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder().build();
}
private HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
.withInlineCompaction(false).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
}
private HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
.withInlineCompaction(false).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
}
@Test(expected = IllegalArgumentException.class)
public void testCompactionOnCopyOnWriteFail() throws Exception {
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
@Test(expected = IllegalArgumentException.class)
public void testCompactionOnCopyOnWriteFail() throws Exception {
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
compactor.compact(jsc, getConfig(), table);
}
@Test
public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
writeClient.insert(recordsRDD, newCommitTime).collect();
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table);
String basePath = table.getMetaClient().getBasePath();
assertTrue("If there is nothing to compact, result will be empty",
result.getFileIdAndFullPaths(basePath).isEmpty());
}
@Test
public void testLogFileCountsAfterCompaction() throws Exception {
FileSystem fs = FSUtils.getFs();
// insert 100 records
HoodieWriteConfig config = getConfig();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
.writeRecordsToLogFiles(metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema,
updatedRecords);
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles =
table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice fileSlice : groupedLogFiles) {
assertEquals("There should be 1 log file written for every data file", 1,
fileSlice.getLogFiles().count());
}
}
@Test
public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
// Do a compaction
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
writeClient.insert(recordsRDD, newCommitTime).collect();
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table);
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table);
String basePath = table.getMetaClient().getBasePath();
assertTrue("If there is nothing to compact, result will be empty",
result.getFileIdAndFullPaths(basePath).isEmpty());
// Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue("Compaction commit should be > than last insert",
HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime,
HoodieTimeline.GREATER));
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView()
.getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice slice : groupedLogFiles) {
assertTrue(
"After compaction there should be no log files visiable on a Realtime view",
slice.getLogFiles().collect(Collectors.toList()).isEmpty());
}
assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath));
}
}
@Test
public void testLogFileCountsAfterCompaction() throws Exception {
FileSystem fs = FSUtils.getFs();
// insert 100 records
HoodieWriteConfig config = getConfig();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
.writeRecordsToLogFiles(metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema,
updatedRecords);
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles =
table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice fileSlice : groupedLogFiles) {
assertEquals("There should be 1 log file written for every data file", 1,
fileSlice.getLogFiles().count());
}
}
// Do a compaction
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table);
// Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue("Compaction commit should be > than last insert",
HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime,
HoodieTimeline.GREATER));
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView()
.getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice slice: groupedLogFiles) {
assertTrue(
"After compaction there should be no log files visiable on a Realtime view",
slice.getLogFiles().collect(Collectors.toList()).isEmpty());
}
assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath));
}
}
// TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make sure the data read is the updated data (compaction correctness)
// TODO - add more test cases for compactions after a failed commit/compaction
// TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make sure the data read is the updated data (compaction correctness)
// TODO - add more test cases for compactions after a failed commit/compaction
}

View File

@@ -17,12 +17,10 @@
package com.uber.hoodie.io.strategy;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import com.beust.jcommander.internal.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;

View File

@@ -17,9 +17,7 @@
package com.uber.hoodie.io.strategy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.util.FSUtils;
import java.util.UUID;
import org.apache.hadoop.fs.FileStatus;
public class TestHoodieDataFile extends HoodieDataFile {

View File

@@ -18,7 +18,6 @@ package com.uber.hoodie.io.strategy;
import com.uber.hoodie.common.model.HoodieLogFile;
import java.util.Optional;
import org.apache.hadoop.fs.Path;
public class TestHoodieLogFile extends HoodieLogFile {

View File

@@ -16,30 +16,31 @@
package com.uber.hoodie.metrics;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.commons.configuration.ConfigurationException;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.commons.configuration.ConfigurationException;
import org.junit.Before;
import org.junit.Test;
public class TestHoodieMetrics {
private HoodieMetrics metrics = null;
@Before
public void start() throws ConfigurationException {
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
when(config.isMetricsOn()).thenReturn(true);
when(config.getMetricsReporterType()).thenReturn(MetricsReporterType.INMEMORY);
metrics = new HoodieMetrics(config, "raw_table");
}
private HoodieMetrics metrics = null;
@Test
public void testRegisterGauge() {
metrics.registerGauge("metric1", 123L);
assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123"));
}
@Before
public void start() throws ConfigurationException {
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
when(config.isMetricsOn()).thenReturn(true);
when(config.getMetricsReporterType()).thenReturn(MetricsReporterType.INMEMORY);
metrics = new HoodieMetrics(config, "raw_table");
}
@Test
public void testRegisterGauge() {
metrics.registerGauge("metric1", 123L);
assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString()
.equals("123"));
}
}

View File

@@ -16,26 +16,37 @@
package com.uber.hoodie.table;
import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus;
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.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.HoodieCreateHandle;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
@@ -47,424 +58,452 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.UUID;
import scala.Option;
import scala.Tuple2;
import static org.junit.Assert.*;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestCopyOnWriteTable {
private String basePath = null;
private transient JavaSparkContext jsc = null;
@Before
public void init() throws Exception {
private String basePath = null;
private transient JavaSparkContext jsc = null;
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestCopyOnWriteTable"));
@Before
public void init() throws Exception {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestCopyOnWriteTable"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
}
@Test
public void testMakeNewPath() throws Exception {
String fileName = UUID.randomUUID().toString();
String partitionPath = "2016/05/04";
int unitNumber = (int) (Math.random() * 10);
HoodieRecord record = mock(HoodieRecord.class);
when(record.getPartitionPath()).thenReturn(partitionPath);
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, unitNumber, fileName)));
}
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
return makeHoodieClientConfigBuilder().build();
}
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
// Prepare the AvroParquetIO
String schemaStr = IOUtils
.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
}
// TODO (weiy): Add testcases for crossing file writing.
@Test
public void testUpdateRecords() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
// We should have a parquet file generated (TODO: better control # files after we revise AvroParquetIO)
File parquetFile = null;
for (File file : new File(this.basePath + partitionPath).listFiles()) {
if (file.getName().endsWith(".parquet")) {
parquetFile = file;
break;
}
}
assertTrue(parquetFile != null);
// Read out the bloom filter and make sure filter can answer record exist or not
Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(parquetFilePath);
for (HoodieRecord record : records) {
assertTrue(filter.mightContain(record.getRecordKey()));
}
// Create a commit file
new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
// Read the parquet file, check the record content
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(parquetFilePath);
GenericRecord newRecord;
int index = 0;
for (GenericRecord record : fileRecords) {
assertTrue(record.get("_row_key").toString().equals(records.get(index).getRecordKey()));
index++;
}
@Test
public void testMakeNewPath() throws Exception {
String fileName = UUID.randomUUID().toString();
String partitionPath = "2016/05/04";
int unitNumber = (int) (Math.random() * 10);
HoodieRecord record = mock(HoodieRecord.class);
when(record.getPartitionPath()).thenReturn(partitionPath);
// We update the 1st record & add a new record
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1);
HoodieRecord updatedRecord1 = new HoodieRecord(
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()),
updateRowChanges1);
updatedRecord1.setCurrentLocation(
new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName())));
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord insertedRecord1 = new HoodieRecord(
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, unitNumber, fileName)));
}
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
return makeHoodieClientConfigBuilder().build();
}
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
table = new HoodieCopyOnWriteTable(config, metadata);
Iterator<List<WriteStatus>> iter = table
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
updatedRecords.iterator());
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
// Prepare the AvroParquetIO
String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
}
// TODO (weiy): Add testcases for crossing file writing.
@Test
public void testUpdateRecords() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
// We should have a parquet file generated (TODO: better control # files after we revise AvroParquetIO)
File parquetFile = null;
for (File file : new File(this.basePath + partitionPath).listFiles()) {
if (file.getName().endsWith(".parquet")) {
parquetFile = file;
break;
}
// Check the updated file
File updatedParquetFile = null;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet")) {
if (FSUtils.getFileId(file.getName())
.equals(FSUtils.getFileId(parquetFile.getName())) &&
HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
updatedParquetFile = file;
break;
}
assertTrue(parquetFile != null);
// Read out the bloom filter and make sure filter can answer record exist or not
Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(parquetFilePath);
for (HoodieRecord record : records) {
assertTrue(filter.mightContain(record.getRecordKey()));
}
// Create a commit file
new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
// Read the parquet file, check the record content
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(parquetFilePath);
GenericRecord newRecord;
int index = 0;
for (GenericRecord record: fileRecords) {
assertTrue(record.get("_row_key").toString().equals(records.get(index).getRecordKey()));
index++;
}
// We update the 1st record & add a new record
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1);
HoodieRecord updatedRecord1 = new HoodieRecord(new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
updatedRecord1.setCurrentLocation(new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName())));
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord insertedRecord1 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
table = new HoodieCopyOnWriteTable(config, metadata);
Iterator<List<WriteStatus>> iter = table.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
// Check the updated file
File updatedParquetFile = null;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet")) {
if (FSUtils.getFileId(file.getName())
.equals(FSUtils.getFileId(parquetFile.getName())) &&
HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
updatedParquetFile = file;
break;
}
}
}
assertTrue(updatedParquetFile != null);
// Check whether the record has been updated
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
BloomFilter updatedFilter = ParquetUtils.readBloomFilterFromParquetMetadata(updatedParquetFilePath);
for (HoodieRecord record : records) {
// No change to the _row_key
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
}
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
records.add(insertedRecord1);// add this so it can further check below
ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
index = 0;
while ((newRecord = (GenericRecord) updatedReader.read()) != null) {
assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey()));
if (index == 0) {
assertTrue(newRecord.get("number").toString().equals("15"));
}
index++;
}
updatedReader.close();
// Also check the numRecordsWritten
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(iter);
WriteStatus writeStatus = statuses.get(0);
assertTrue("Should be only one file generated", statuses.size() == 1);
assertEquals(4, writeStatus.getStat().getNumWrites());//3 rewritten records + 1 new record
}
}
assertTrue(updatedParquetFile != null);
// Check whether the record has been updated
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
BloomFilter updatedFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(updatedParquetFilePath);
for (HoodieRecord record : records) {
// No change to the _row_key
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
}
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
records.add(insertedRecord1);// add this so it can further check below
private List<HoodieRecord> newHoodieRecords(int n, String time) throws Exception {
List<HoodieRecord> records = new ArrayList<>();
for (int i = 0; i < n; i++) {
String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}",
UUID.randomUUID().toString(),
time,
i);
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records.add(new HoodieRecord(
new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
rowChange));
}
return records;
ParquetReader updatedReader = ParquetReader
.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
index = 0;
while ((newRecord = (GenericRecord) updatedReader.read()) != null) {
assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey()));
if (index == 0) {
assertTrue(newRecord.get("number").toString().equals("15"));
}
index++;
}
updatedReader.close();
// Also check the numRecordsWritten
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(iter);
WriteStatus writeStatus = statuses.get(0);
assertTrue("Should be only one file generated", statuses.size() == 1);
assertEquals(4, writeStatus.getStat().getNumWrites());//3 rewritten records + 1 new record
}
private List<HoodieRecord> newHoodieRecords(int n, String time) throws Exception {
List<HoodieRecord> records = new ArrayList<>();
for (int i = 0; i < n; i++) {
String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}",
UUID.randomUUID().toString(),
time,
i);
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records.add(new HoodieRecord(
new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
rowChange));
}
return records;
}
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withWriteStatusClass(MetadataMergeWriteStatus.class).build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
// Insert new records
List<WriteStatus> writeStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus
.mergeMetadataForWriteStatuses(writeStatuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * 3
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
@Test
public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Write a few records, and get atleast one file
// 10 records for partition 1, 1 record for partition 2.
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Simulate crash after first file
List<WriteStatus> statuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
WriteStatus status = statuses.get(0);
Path partialFile = new Path(String.format("%s/%s/%s",
basePath,
status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))
);
assertTrue(fs.exists(partialFile));
// When we retry
records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
statuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
status = statuses.get(0);
Path retriedFIle = new Path(String.format("%s/%s/%s",
basePath,
status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))
);
assertTrue(fs.exists(retriedFIle));
assertFalse(fs.exists(partialFile));
}
@Test
public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Case 1:
// 10 records for partition 1, 1 record for partition 2.
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Insert new records
List<WriteStatus> returnedStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
// TODO: check the actual files and make sure 11 records, total were written.
assertEquals(2, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
assertEquals(10, returnedStatuses.get(0).getWrittenRecords().size());
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
assertEquals(1, returnedStatuses.get(1).getWrittenRecords().size());
// Case 2:
// 1 record for partition 1, 5 record for partition 2, 1 records for partition 3.
records = newHoodieRecords(1, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(5, "2016-02-01T03:16:41.415Z"));
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
// Insert new records
returnedStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
assertEquals(3, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
assertEquals(1, returnedStatuses.get(0).getWrittenRecords().size());
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
assertEquals(5, returnedStatuses.get(1).getWrittenRecords().size());
assertEquals("2016/02/02", returnedStatuses.get(2).getPartitionPath());
assertEquals(1, returnedStatuses.get(2).getWrittenRecords().size());
}
@Test
public void testFileSizeUpsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
.parquetPageSize(64 * 1024).build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB
for (int i = 0; i < 2000; i++) {
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString()
+ "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records
.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
rowChange));
}
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class).build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
List<WriteStatus> writeStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus
.mergeMetadataForWriteStatuses(writeStatuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * 3
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
// Check the updated file
int counts = 0;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName())
.equals(commitTime)) {
System.out.println(file.getName() + "-" + file.length());
counts++;
}
}
assertEquals(
"If the number of records are more than 1150, then there should be a new file", 3,
counts);
}
@Test
public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Write a few records, and get atleast one file
// 10 records for partition 1, 1 record for partition 2.
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
private List<HoodieCopyOnWriteTable.InsertBucket> testUpsertPartitioner(int smallFileSize,
int numInserts,
int numUpdates,
int fileSize,
boolean autoSplitInserts) throws Exception {
final String TEST_PARTITION_PATH = "2016/09/26";
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.compactionSmallFileSize(smallFileSize).insertSplitSize(100)
.autoTuneInsertSplits(autoSplitInserts).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build())
.build();
// Simulate crash after first file
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
WriteStatus status = statuses.get(0);
Path partialFile = new Path(String.format("%s/%s/%s",
basePath,
status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))
);
assertTrue(fs.exists(partialFile));
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
// When we retry
records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
status = statuses.get(0);
Path retriedFIle = new Path(String.format("%s/%s/%s",
basePath,
status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))
);
assertTrue(fs.exists(retriedFIle));
assertFalse(fs.exists(partialFile));
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(
new String[]{TEST_PARTITION_PATH});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec : updateRecords) {
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
}
List<HoodieRecord> records = new ArrayList<>();
records.addAll(insertRecords);
records.addAll(updateRecords);
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner)
table.getUpsertPartitioner(profile);
assertEquals("Should have 3 partitions", 3, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Update record should have gone to the 1 update partiton", 0,
partitioner.getPartition(new Tuple2<>(updateRecords.get(0).getKey(),
Option.apply(updateRecords.get(0).getCurrentLocation()))));
return partitioner.getInsertBuckets(TEST_PARTITION_PATH);
}
@Test public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Case 1:
// 10 records for partition 1, 1 record for partition 2.
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Insert new records
List<WriteStatus> returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
@Test
public void testUpsertPartitioner() throws Exception {
// Inserts + Updates... Check all updates go together & inserts subsplit
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(0, 200, 100,
1024, false);
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}
// TODO: check the actual files and make sure 11 records, total were written.
assertEquals(2, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
assertEquals(10, returnedStatuses.get(0).getWrittenRecords().size());
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
assertEquals(1, returnedStatuses.get(1).getWrittenRecords().size());
@Test
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding smallest file
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(1000 * 1024,
400, 100, 800 * 1024, false);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0,
insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight,
0.01);
// Case 2:
// 1 record for partition 1, 5 record for partition 2, 1 records for partition 3.
records = newHoodieRecords(1, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(5, "2016-02-01T03:16:41.415Z"));
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
// Insert new records
returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
assertEquals(3, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
assertEquals(1, returnedStatuses.get(0).getWrittenRecords().size());
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
assertEquals(5, returnedStatuses.get(1).getWrittenRecords().size());
assertEquals("2016/02/02", returnedStatuses.get(2).getPartitionPath());
assertEquals(1, returnedStatuses.get(2).getWrittenRecords().size());
// Now with insert split size auto tuned
insertBuckets = testUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, true);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0,
insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400,
insertBuckets.get(0).weight, 0.01);
}
@After
public void cleanup() {
if (basePath != null) {
new File(basePath).delete();
}
@Test public void testFileSizeUpsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
.parquetPageSize(64 * 1024).build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB
for (int i = 0; i < 2000; i++) {
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
rowChange));
}
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
// Check the updated file
int counts = 0;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(commitTime)) {
System.out.println(file.getName() + "-" + file.length());
counts++;
}
}
assertEquals(
"If the number of records are more than 1150, then there should be a new file", 3,
counts);
}
private List<HoodieCopyOnWriteTable.InsertBucket> testUpsertPartitioner(int smallFileSize,
int numInserts,
int numUpdates,
int fileSize,
boolean autoSplitInserts) throws Exception {
final String TEST_PARTITION_PATH = "2016/09/26";
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.compactionSmallFileSize(smallFileSize).insertSplitSize(100).autoTuneInsertSplits(autoSplitInserts).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec: updateRecords) {
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
}
List<HoodieRecord> records = new ArrayList<>();
records.addAll(insertRecords);
records.addAll(updateRecords);
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner)
table.getUpsertPartitioner(profile);
assertEquals("Should have 3 partitions", 3, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Update record should have gone to the 1 update partiton", 0,
partitioner.getPartition(new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation()))));
return partitioner.getInsertBuckets(TEST_PARTITION_PATH);
}
@Test
public void testUpsertPartitioner() throws Exception {
// Inserts + Updates... Check all updates go together & inserts subsplit
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(0, 200, 100, 1024, false);
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}
@Test
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding smallest file
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, false);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, 0.01);
// Now with insert split size auto tuned
insertBuckets = testUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, true);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 200.0/2400, insertBuckets.get(0).weight, 0.01);
}
@After
public void cleanup() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
if (jsc != null) {
jsc.stop();
}
}
}

View File

@@ -20,7 +20,6 @@ log4j.category.com.uber.hoodie.io=WARN
log4j.category.com.uber.hoodie.common=WARN
log4j.category.com.uber.hoodie.table.log=WARN
log4j.category.org.apache.parquet.hadoop=WARN
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.