1
0

Take 2: Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

- Refactored timelines to be a single timeline for all active events and one for archived events. CommitTimeline and other timelines can be inferred by applying a filter on the activeTimelime
- Introduced HoodieInstant to abstract different types of action, commit time and if isInFlight
- Implemented other review comments
This commit is contained in:
Prasanna Rajaperumal
2017-01-18 01:00:36 -08:00
parent 8ee777a9bb
commit ccd8cb2407
46 changed files with 1194 additions and 1106 deletions

View File

@@ -19,8 +19,10 @@ package com.uber.hoodie.common.model;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileSystem;
@@ -30,6 +32,8 @@ import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.Date;
@@ -66,13 +70,13 @@ public class HoodieTestUtils {
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTableMetaClient.makeCommitFileName(commitTime)).createNewFile();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeCommitFileName(commitTime)).createNewFile();
}
}
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTableMetaClient.makeInflightCommitFileName(commitTime)).createNewFile();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
}
}
@@ -97,15 +101,15 @@ public class HoodieTestUtils {
}
public static final boolean doesCommitExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetaClient.COMMIT_EXTENSION).exists();
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION).exists();
}
public static final boolean doesInflightExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX).exists();
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION).exists();
}
public static String makeInflightTestFileName(String instant) {
return instant + TEST_EXTENSION + HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX;
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
}
public static String makeTestFileName(String instant) {
@@ -123,9 +127,12 @@ public class HoodieTestUtils {
assert !iter1.hasNext() && !iter2.hasNext();
}
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz) {
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz)
throws IOException, ClassNotFoundException {
// Using Kyro as the default serializer in Spark Jobs
Kryo kryo = new Kryo();
kryo.register(HoodieTableMetaClient.class, new JavaSerializer());
ByteArrayOutputStream baos = new ByteArrayOutputStream();
Output output = new Output(baos);
kryo.writeObject(output, object);

View File

@@ -16,14 +16,12 @@
package com.uber.hoodie.common.table;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.ArrayFile;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
@@ -31,13 +29,15 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Collectors;
import static org.junit.Assert.*;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
public class HoodieTableMetaClientTest {
private HoodieTableMetaClient metaClient;
@@ -61,48 +61,53 @@ public class HoodieTableMetaClientTest {
}
@Test
public void checkSerDe() throws IOException {
// check if this object is serialized and se-serialized, we are able to read from the file system
public void checkSerDe() throws IOException, ClassNotFoundException {
// check if this object is serialized and de-serialized, we are able to read from the file system
HoodieTableMetaClient deseralizedMetaClient =
HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
commitTimeline.saveInstantAsInflight("1");
commitTimeline.saveInstantAsComplete("1", Optional.of("test-detail".getBytes()));
assertNotNull(deseralizedMetaClient);
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
commitTimeline.createInflight(instant);
commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
commitTimeline = commitTimeline.reload();
assertEquals("Commit should be 1", "1", commitTimeline.getInstants().findFirst().get());
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
assertEquals("Commit should be 1 and completed", completedInstant,
commitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
commitTimeline.readInstantDetails("1").get());
commitTimeline.getInstantDetails(completedInstant).get());
}
@Test
public void checkCommitTimeline() throws IOException {
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
assertFalse("Should be empty commit timeline",
commitTimeline.getInstants().findFirst().isPresent());
assertFalse("Should be empty commit timeline",
commitTimeline.getInflightInstants().findFirst().isPresent());
commitTimeline.saveInstantAsInflight("1");
commitTimeline.saveInstantAsComplete("1", Optional.of("test-detail".getBytes()));
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
activeTimeline.createInflight(instant);
activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
commitTimeline = metaClient.getActiveCommitTimeline();
assertFalse("Should be empty commit timeline",
commitTimeline.getInstants().findFirst().isPresent());
assertFalse("Should be empty commit timeline",
commitTimeline.getInflightInstants().findFirst().isPresent());
activeTimeline = metaClient.getActiveTimeline();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
commitTimeline = commitTimeline.reload();
assertTrue("Should be the 1 commit we made",
commitTimeline.getInstants().findFirst().isPresent());
assertEquals("Commit should be 1", "1", commitTimeline.getInstants().findFirst().get());
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
activeTimeline = activeTimeline.reload();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty());
assertEquals("Commit should be 1", completedInstant,
activeCommitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
commitTimeline.readInstantDetails("1").get());
activeCommitTimeline.getInstantDetails(completedInstant).get());
}
@Test
public void checkArchiveCommitTimeline() throws IOException {
Path archiveLogPath =
HoodieArchivedCommitTimeline.getArchiveLogPath(metaClient.getMetaPath());
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath());
SequenceFile.Writer writer = SequenceFile
.createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath),
SequenceFile.Writer.keyClass(Text.class),
@@ -114,13 +119,24 @@ public class HoodieTableMetaClientTest {
IOUtils.closeStream(writer);
HoodieTimeline archivedTimeline = metaClient.getArchivedCommitTimeline();
assertEquals(Lists.newArrayList("1", "2", "3"),
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
HoodieInstant instant3 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
archivedTimeline.getInstants().collect(Collectors.toList()));
System.out.println(new String( archivedTimeline.readInstantDetails("1").get()));
assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.readInstantDetails("1").get());
assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.readInstantDetails("2").get());
assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.readInstantDetails("3").get());
assertArrayEquals(new Text("data1").getBytes(),
archivedTimeline.getInstantDetails(instant1).get());
assertArrayEquals(new Text("data2").getBytes(),
archivedTimeline.getInstantDetails(instant2).get());
assertArrayEquals(new Text("data3").getBytes(),
archivedTimeline.getInstantDetails(instant3).get());
}

View File

@@ -0,0 +1,126 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.string;
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.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import static org.junit.Assert.*;
public class HoodieActiveTimelineTest {
private HoodieActiveTimeline timeline;
private HoodieTableMetaClient metaClient;
@Rule
public final ExpectedException exception = ExpectedException.none();
@Before
public void setUp() throws Exception {
this.metaClient = HoodieTestUtils.initOnTemp();
}
@After
public void tearDown() throws Exception {
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
}
@Test
public void testLoadingInstantsFromFiles() throws IOException {
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
HoodieInstant instant3 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
HoodieInstant instant4 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8");
HoodieInstant instant5 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
timeline.saveAsComplete(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty());
timeline.saveAsComplete(instant4, Optional.empty());
timeline.createInflight(instant5);
timeline = timeline.reload();
assertEquals("Total instants should be 5", 5, timeline.countInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1, instant2, instant3, instant4, instant5), timeline.getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1, instant2, instant3, instant4, instant5),
timeline.getCommitTimeline().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1, instant2, instant3, instant4),
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
timeline.getCommitTimeline().filterInflights().getInstants());
}
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01")));
}
@Test
public void testTimelineOperations() throws Exception {
timeline = new MockHoodieTimeline(
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
Stream.of("21", "23"));
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11")
.getInstants().map(HoodieInstant::getTimestamp));
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2)
.getInstants().map(HoodieInstant::getTimestamp));
assertFalse(timeline.empty());
assertFalse(timeline.getCommitTimeline().filterInflights().empty());
assertEquals("", 12, timeline.countInstants());
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
assertEquals("", 10, activeCommitTimeline.countInstants());
assertEquals("", "01", activeCommitTimeline.firstInstant().get().getTimestamp());
assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp());
assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp());
assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp());
assertTrue("", activeCommitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09")));
assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
}
}

View File

@@ -1,104 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.string;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import static org.junit.Assert.*;
public class HoodieDefaultTimelineTest {
private HoodieTimeline timeline;
private HoodieTableMetaClient metaClient;
@Rule
public final ExpectedException exception = ExpectedException.none();
@Before
public void setUp() throws Exception {
this.metaClient = HoodieTestUtils.initOnTemp();
}
@After
public void tearDown() throws Exception {
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
}
@Test
public void testLoadingInstantsFromFiles() throws IOException {
timeline =
new MockHoodieTimeline(HoodieTestUtils.fs, metaClient.getMetaPath(), ".test");
timeline.saveInstantAsComplete("1", Optional.empty());
timeline.saveInstantAsComplete("3", Optional.empty());
timeline.saveInstantAsComplete("5", Optional.empty());
timeline.saveInstantAsComplete("8", Optional.empty());
timeline.saveInstantAsInflight("9");
timeline = timeline.reload();
assertEquals("Total instants should be 4", 4, timeline.getTotalInstants());
HoodieTestUtils
.assertStreamEquals("Check the instants stream", Stream.of("1", "3", "5", "8"),
timeline.getInstants());
assertTrue("Inflights should be present in the timeline", timeline.hasInflightInstants());
HoodieTestUtils.assertStreamEquals("Check the inflights stream", Stream.of("9"),
timeline.getInflightInstants());
}
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new MockHoodieTimeline(Stream.empty(), Stream.empty());
assertFalse(timeline.hasInstants());
assertFalse(timeline.hasInflightInstants());
assertEquals("", 0, timeline.getTotalInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant("01"));
}
@Test
public void testTimelineOperations() throws Exception {
timeline = new MockHoodieTimeline(
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
Stream.of("21", "23"));
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
timeline.findInstantsInRange("04", "11"));
HoodieTestUtils
.assertStreamEquals("", Stream.of("09", "11"), timeline.findInstantsAfter("07", 2));
assertTrue(timeline.hasInstants());
assertTrue(timeline.hasInflightInstants());
assertEquals("", 10, timeline.getTotalInstants());
assertEquals("", "01", timeline.firstInstant().get());
assertEquals("", "11", timeline.nthInstant(5).get());
assertEquals("", "19", timeline.lastInstant().get());
assertEquals("", "09", timeline.nthFromLastInstant(5).get());
assertTrue("", timeline.containsInstant("09"));
assertFalse("", timeline.isInstantBeforeTimelineStarts("02"));
assertTrue("", timeline.isInstantBeforeTimelineStarts("00"));
}
}

View File

@@ -16,51 +16,29 @@
package com.uber.hoodie.common.table.string;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hadoop.fs.FileSystem;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.IOException;
import java.util.Optional;
import java.util.Comparator;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class MockHoodieTimeline extends HoodieDefaultTimeline {
private String fileExt;
public MockHoodieTimeline(FileSystem fs, String metaPath, String fileExtension)
public class MockHoodieTimeline extends HoodieActiveTimeline {
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights)
throws IOException {
super(fs, metaPath, fileExtension);
this.fileExt = fileExtension;
}
public MockHoodieTimeline(Stream<String> instants, Stream<String> inflights)
throws IOException {
super(instants, inflights);
}
@Override
public HoodieTimeline reload() throws IOException {
return new MockHoodieTimeline(fs, metaPath, fileExt);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
return Optional.empty();
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTestUtils.makeInflightTestFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTestUtils.makeTestFileName(instant);
}
@Override
protected String getTimelineName() {
return "mock-test";
super();
this.instants = Stream.concat(completed
.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
inflights.map(
s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
@Override
public String apply(HoodieInstant hoodieInstant) {
return hoodieInstant.getFileName();
}
})).collect(Collectors.toList());
}
}

View File

@@ -23,6 +23,8 @@ 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.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 org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
@@ -40,6 +42,7 @@ import java.util.stream.Collectors;
import static org.junit.Assert.*;
@SuppressWarnings("ResultOfMethodCallIgnored")
public class ReadOptimizedTableViewTest {
private HoodieTableMetaClient metaClient;
private String basePath;
@@ -77,8 +80,10 @@ public class ReadOptimizedTableViewTest {
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().isPresent());
// Make this commit safe
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
commitTimeline.saveInstantAsComplete(commitTime1, Optional.empty());
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime1);
commitTimeline.saveAsComplete(instant1, Optional.empty());
refreshFsView();
assertEquals("", fileName1,
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
@@ -94,7 +99,9 @@ public class ReadOptimizedTableViewTest {
.getFileName());
// Make it safe
commitTimeline.saveInstantAsComplete(commitTime2, Optional.empty());
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime2);
commitTimeline.saveAsComplete(instant2, Optional.empty());
refreshFsView();
assertEquals("", fileName2,
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
@@ -140,7 +147,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersionInPartition("2016/05/01", commitTime4)
fsView.getLatestVersionInPartition("2016/05/01", commitTime4)
.collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
Set<String> filenames = Sets.newHashSet();
@@ -153,7 +160,7 @@ public class ReadOptimizedTableViewTest {
// Reset the max commit time
List<HoodieDataFile> statuses2 =
fsView.streamLatestVersionInPartition("2016/05/01", commitTime3)
fsView.getLatestVersionInPartition("2016/05/01", commitTime3)
.collect(Collectors.toList());
assertEquals(statuses2.size(), 3);
filenames = Sets.newHashSet();
@@ -204,7 +211,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<List<HoodieDataFile>> statuses1 =
fsView.streamEveryVersionInPartition("2016/05/01").collect(Collectors.toList());
fsView.getEveryVersionInPartition("2016/05/01").collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
for (List<HoodieDataFile> status : statuses1) {
@@ -269,9 +276,9 @@ public class ReadOptimizedTableViewTest {
assertEquals(statuses.length, 7);
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersionInRange(statuses, Lists.newArrayList(commitTime2, commitTime3))
.collect(Collectors.toList());
List<HoodieDataFile> statuses1 = fsView
.getLatestVersionInRange(statuses, Lists.newArrayList(commitTime2, commitTime3))
.collect(Collectors.toList());
assertEquals(statuses1.size(), 2);
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
@@ -320,7 +327,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersionsBeforeOrOn(statuses, commitTime2)
fsView.getLatestVersionsBeforeOrOn(statuses, commitTime2)
.collect(Collectors.toList());
assertEquals(statuses1.size(), 2);
Set<String> filenames = Sets.newHashSet();
@@ -371,8 +378,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersions(statuses)
.collect(Collectors.toList());
fsView.getLatestVersions(statuses).collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {