Adding ability for inserts to be written to log files
This commit is contained in:
committed by
vinoth chandar
parent
34827d50e1
commit
3da063f83b
@@ -28,7 +28,6 @@ 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.exception.HoodieException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
@@ -43,6 +42,7 @@ 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.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
@@ -152,14 +152,17 @@ public class HoodieClientTestUtils {
|
||||
/**
|
||||
* 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) {
|
||||
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext
|
||||
sqlContext,
|
||||
FileSystem
|
||||
fs, String...
|
||||
paths) {
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
try {
|
||||
HoodieTable hoodieTable = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
|
||||
for (String path : paths) {
|
||||
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
|
||||
hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
|
||||
metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path)));
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList());
|
||||
for (HoodieDataFile file : latestFiles) {
|
||||
filteredPaths.add(file.getPath());
|
||||
|
||||
Reference in New Issue
Block a user