1
0

Add new APIs in HoodieReadClient and HoodieWriteClient

This commit is contained in:
Jian Xu
2018-02-22 11:20:54 -08:00
committed by vinoth chandar
parent 6fec9655a8
commit 5d5c306e64
3 changed files with 173 additions and 65 deletions

View File

@@ -19,11 +19,14 @@ package com.uber.hoodie;
import com.google.common.base.Optional;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
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.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.Serializable;
import java.util.HashSet;
@@ -45,7 +48,7 @@ import scala.Tuple2;
/**
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
*/
public class HoodieReadClient implements Serializable {
public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializable {
private static Logger logger = LogManager.getLogger(HoodieReadClient.class);
@@ -57,7 +60,7 @@ public class HoodieReadClient implements Serializable {
* just with a simple basepath pointing to the dataset. Until, then just always assume a
* BloomIndex
*/
private transient final HoodieBloomIndex index;
private transient final HoodieIndex<T> index;
private final HoodieTimeline commitTimeline;
private HoodieTable hoodieTable;
private transient Optional<SQLContext> sqlContextOpt;
@@ -66,15 +69,12 @@ public class HoodieReadClient implements Serializable {
* @param basePath path to Hoodie dataset
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc;
this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), null);
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
this(jsc, HoodieWriteConfig.newBuilder()
.withPath(basePath)
// by default we use HoodieBloomIndex
.withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build());
}
/**
@@ -88,6 +88,22 @@ public class HoodieReadClient implements Serializable {
this.sqlContextOpt = Optional.of(sqlContext);
}
/**
* @param clientConfig instance of HoodieWriteConfig
*/
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
final String basePath = clientConfig.getBasePath();
this.jsc = jsc;
this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
clientConfig);
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
this.index = HoodieIndex.createIndex(clientConfig, jsc);
this.sqlContextOpt = Optional.absent();
}
/**
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
*
@@ -158,8 +174,20 @@ public class HoodieReadClient implements Serializable {
* @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);
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
JavaRDD<HoodieRecord<T>> recordsWithLocation = tagLocation(hoodieRecords);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present). Input RDD should contain no duplicates if needed.
*
* @param hoodieRecords Input RDD of Hoodie records
* @return Tagged RDD of Hoodie records
*/
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords)
throws HoodieIndexException {
return index.tagLocation(hoodieRecords, hoodieTable);
}
}