Adding hoodie-spark to support Spark Datasource for Hoodie
- Write with COW/MOR paths work fully - Read with RO view works on both storages* - Incremental view supported on COW - Refactored out HoodieReadClient methods, to just contain key based access - HoodieDataSourceHelpers class can be now used to construct inputs to datasource - Tests in hoodie-client using new helpers and mechanisms - Basic tests around save modes & insert/upserts (more to follow) - Bumped up scala to 2.11, since 2.10 is deprecated & complains with scalatest - Updated documentation to describe usage - New sample app written using the DataSource API
This commit is contained in:
committed by
vinoth chandar
parent
c98ee057fc
commit
64e0573aca
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
@@ -33,6 +34,7 @@ import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
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.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
@@ -43,11 +45,11 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.scheduler.SparkListener;
|
||||
@@ -58,7 +60,6 @@ import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import scala.collection.Iterator;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
@@ -76,12 +77,14 @@ import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.collection.Iterator;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestHoodieClient implements Serializable {
|
||||
public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
private transient JavaSparkContext jsc = null;
|
||||
private transient SQLContext sqlContext;
|
||||
private String basePath = null;
|
||||
@@ -142,6 +145,8 @@ public class TestHoodieClient implements Serializable {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testFilterExist() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
@@ -219,12 +224,15 @@ public class TestHoodieClient implements Serializable {
|
||||
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting a single commit.", readClient.listCommitsSince("000").size(), 1);
|
||||
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
|
||||
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), records.size());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
assertEquals("Must contain 200 records",
|
||||
records.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
|
||||
@@ -248,9 +256,9 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 2 commits
|
||||
readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting two commits.", readClient.listCommitsSince("000").size(), 2);
|
||||
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
|
||||
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
|
||||
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
@@ -264,16 +272,18 @@ public class TestHoodieClient implements Serializable {
|
||||
for (int i=0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals("Must contain 200 records", readClient.read(fullPartitionPaths).count(), 200);
|
||||
assertEquals("Must contain 200 records",
|
||||
200,
|
||||
HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count());
|
||||
|
||||
|
||||
// Check that the incremental consumption from time 000
|
||||
assertEquals("Incremental consumption from time 002, should give all records in commit 004",
|
||||
readClient.readCommit(newCommitTime).count(),
|
||||
readClient.readSince("002").count());
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "002").count());
|
||||
assertEquals("Incremental consumption from time 001, should give all records in commit 004",
|
||||
readClient.readCommit(newCommitTime).count(),
|
||||
readClient.readSince("001").count());
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext,timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext,timeline, "001").count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -301,12 +311,13 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting a single commit.", readClient.listCommitsSince("000").size(), 1);
|
||||
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
|
||||
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), fewRecordsForInsert.size());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp());
|
||||
assertEquals("Must contain 200 records", fewRecordsForInsert.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect();
|
||||
@@ -327,31 +338,29 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 2 commits
|
||||
readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting two commits.", readClient.listCommitsSince("000").size(), 2);
|
||||
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
|
||||
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime);
|
||||
|
||||
// Check the entire dataset has 150 records(200-50) still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i=0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals("Must contain 150 records", readClient.read(fullPartitionPaths).count(), 150);
|
||||
assertEquals("Must contain 150 records", 150,
|
||||
HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count());
|
||||
|
||||
|
||||
// Check that the incremental consumption from time 000
|
||||
assertEquals("Incremental consumption from latest commit, should give 50 updated records",
|
||||
readClient.readCommit(newCommitTime).count(),
|
||||
50);
|
||||
50,
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
assertEquals("Incremental consumption from time 001, should give 50 updated records",
|
||||
50,
|
||||
readClient.readSince("001").count());
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count());
|
||||
assertEquals("Incremental consumption from time 000, should give 150",
|
||||
150,
|
||||
readClient.readSince("000").count());
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count());
|
||||
}
|
||||
|
||||
|
||||
@@ -566,10 +575,10 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
assertFalse(table.getCompletedCommitTimeline().empty());
|
||||
String commitTime =
|
||||
@@ -594,7 +603,7 @@ public class TestHoodieClient implements Serializable {
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, getConfig());
|
||||
HoodieTimeline timeline = table.getCommitTimeline();
|
||||
timeline = table.getCommitTimeline();
|
||||
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
// Need to ensure the following
|
||||
@@ -660,9 +669,10 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
assertFalse(table.getCompletedCommitTimeline().empty());
|
||||
@@ -18,10 +18,23 @@ package com.uber.hoodie.common;
|
||||
|
||||
import com.uber.hoodie.HoodieReadClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
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.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;
|
||||
|
||||
@@ -29,10 +42,13 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
@@ -90,4 +106,83 @@ public class HoodieClientTestUtils {
|
||||
.setMaster("local[1]");
|
||||
return HoodieReadClient.addHoodieSupport(sparkConf);
|
||||
}
|
||||
|
||||
public static HashMap<String, String> getLatestFileIsToFullPath(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 = getLatestFileIsToFullPath(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 = getLatestFileIsToFullPath(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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -109,11 +109,10 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
return rowKey;
|
||||
}
|
||||
|
||||
private String getJsonData() throws IOException {
|
||||
public String getJsonData() throws IOException {
|
||||
return unCompressData(jsonDataCompressed);
|
||||
}
|
||||
|
||||
|
||||
private byte[] compressData(String jsonData) throws IOException {
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DeflaterOutputStream dos =
|
||||
|
||||
@@ -1,21 +1,26 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie;
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieMergeOnReadTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
@@ -39,7 +44,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.compact.HoodieCompactor;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
@@ -217,12 +221,15 @@ public class TestMergeOnReadTable {
|
||||
assertTrue(dataFilesToRead.findAny().isPresent());
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
|
||||
assertEquals("Expecting a single commit.", 1, readClient.listCommitsSince("000").size());
|
||||
String latestCompactionCommitTime = readClient.latestCommit();
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig());
|
||||
HoodieTimeline timeline = table.getCompletedCompactionCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
assertTrue(HoodieTimeline
|
||||
.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER));
|
||||
assertEquals("Must contain 200 records", 200, readClient.readSince("000").count());
|
||||
|
||||
assertEquals("Must contain 200 records", 200,
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count());
|
||||
}
|
||||
|
||||
// Check if record level metadata is aggregated properly at the end of write.
|
||||
Reference in New Issue
Block a user