[HUDI-308] Avoid Renames for tracking state transitions of all actions on dataset
This commit is contained in:
committed by
Balaji Varadarajan
parent
8963a68e6a
commit
9a1f698eef
@@ -18,7 +18,9 @@
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
@@ -39,6 +41,7 @@ import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import com.esotericsoftware.kryo.Kryo;
|
||||
import com.esotericsoftware.kryo.io.Input;
|
||||
@@ -68,6 +71,7 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -123,6 +127,12 @@ public class HoodieTestUtils {
|
||||
|
||||
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeRequestedCommitFileName(commitTime)).createNewFile();
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime))
|
||||
.createNewFile();
|
||||
@@ -142,26 +152,43 @@ public class HoodieTestUtils {
|
||||
}
|
||||
|
||||
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
+ HoodieTimeline.makeRequestedCommitFileName(commitTime)).createNewFile();
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName(
|
||||
commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCleanFiles(String basePath, Configuration configuration, String... commitTimes)
|
||||
throws IOException {
|
||||
public static final void createPendingCleanFiles(HoodieTableMetaClient metaClient, Configuration configuration,
|
||||
String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
Path commitFile = new Path((basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeInflightCleanerFileName(commitTime)));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
Arrays.asList(HoodieTimeline.makeRequestedCleanerFileName(commitTime),
|
||||
HoodieTimeline.makeInflightCleanerFileName(commitTime)).forEach(f -> {
|
||||
FSDataOutputStream os = null;
|
||||
try {
|
||||
Path commitFile = new Path(
|
||||
metaClient.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
|
||||
os = metaClient.getFs().create(commitFile, true);
|
||||
// Write empty clean metadata
|
||||
os.write(AvroUtils.serializeCleanerPlan(
|
||||
new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(), 1)).get());
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} finally {
|
||||
if (null != os) {
|
||||
try {
|
||||
os.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException {
|
||||
createInflightCleanFiles(basePath, HoodieTestUtils.getDefaultHadoopConf(), commitTimes);
|
||||
}
|
||||
|
||||
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
@@ -276,6 +303,7 @@ public class HoodieTestUtils {
|
||||
public static void createCleanFiles(HoodieTableMetaClient metaClient, String basePath,
|
||||
String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
createPendingCleanFiles(metaClient, configuration, commitTime);
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
|
||||
@@ -68,7 +68,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness {
|
||||
assertNotNull(deseralizedMetaClient);
|
||||
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
|
||||
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
commitTimeline.createInflight(instant);
|
||||
commitTimeline.createNewInstant(instant);
|
||||
commitTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
|
||||
commitTimeline = commitTimeline.reload();
|
||||
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
|
||||
@@ -84,7 +84,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness {
|
||||
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
|
||||
|
||||
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
activeTimeline.createInflight(instant);
|
||||
activeTimeline.createNewInstant(instant);
|
||||
activeTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
|
||||
|
||||
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
|
||||
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you 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 org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class TestTimelineLayout {
|
||||
|
||||
@Test
|
||||
public void testTimelineLayoutFilter() {
|
||||
List<HoodieInstant> rawInstants = Arrays.asList(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "001"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "001"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "003"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "004"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "005"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "007"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007"));
|
||||
|
||||
List<HoodieInstant> layout0Instants = TimelineLayout.getLayout(new TimelineLayoutVersion(0))
|
||||
.filterHoodieInstants(rawInstants.stream()).collect(Collectors.toList());
|
||||
Assert.assertEquals(rawInstants, layout0Instants);
|
||||
List<HoodieInstant> layout1Instants = TimelineLayout.getLayout(TimelineLayoutVersion.CURR_LAYOUT_VERSION)
|
||||
.filterHoodieInstants(rawInstants.stream()).collect(Collectors.toList());
|
||||
Assert.assertEquals(7, layout1Instants.size());
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001")));
|
||||
}
|
||||
}
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.junit.Before;
|
||||
@@ -53,10 +54,10 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
|
||||
@Test
|
||||
public void testLoadingInstantsFromFiles() throws IOException {
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant4 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8");
|
||||
HoodieInstant instant1 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant4 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "8");
|
||||
HoodieInstant instant1Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
@@ -65,11 +66,23 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
|
||||
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.saveAsComplete(instant1, Option.empty());
|
||||
timeline.saveAsComplete(instant2, Option.empty());
|
||||
timeline.saveAsComplete(instant3, Option.empty());
|
||||
timeline.saveAsComplete(instant4, Option.empty());
|
||||
timeline.createInflight(instant5);
|
||||
timeline.createNewInstant(instant1);
|
||||
timeline.transitionRequestedToInflight(instant1, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant1.getAction(), instant1.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant2);
|
||||
timeline.transitionRequestedToInflight(instant2, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant2.getAction(), instant2.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant3);
|
||||
timeline.transitionRequestedToInflight(instant3, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant3.getAction(), instant3.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant4);
|
||||
timeline.transitionRequestedToInflight(instant4, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant4.getAction(), instant4.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant5);
|
||||
timeline = timeline.reload();
|
||||
|
||||
assertEquals("Total instants should be 5", 5, timeline.countInstants());
|
||||
@@ -83,7 +96,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete),
|
||||
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
|
||||
timeline.getCommitTimeline().filterInflightsExcludingCompaction().getInstants());
|
||||
timeline.getCommitTimeline().filterPendingExcludingCompaction().getInstants());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -107,7 +120,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"), timeline.getCommitTimeline().filterCompletedInstants()
|
||||
.findInstantsAfter("07", 2).getInstants().map(HoodieInstant::getTimestamp));
|
||||
assertFalse(timeline.empty());
|
||||
assertFalse(timeline.getCommitTimeline().filterInflightsExcludingCompaction().empty());
|
||||
assertFalse(timeline.getCommitTimeline().filterPendingExcludingCompaction().empty());
|
||||
assertEquals("", 12, timeline.countInstants());
|
||||
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals("", 10, activeCommitTimeline.countInstants());
|
||||
|
||||
@@ -129,9 +129,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
|
||||
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
|
||||
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
|
||||
|
||||
refreshFsView();
|
||||
|
||||
@@ -262,9 +262,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
|
||||
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
|
||||
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
|
||||
|
||||
refreshFsView();
|
||||
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
@@ -309,8 +309,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile();
|
||||
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant4, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant5, Option.empty());
|
||||
refreshFsView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
|
||||
@@ -415,9 +415,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
inflightDeltaInstantTime, 0, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + inflightLogFileName).createNewFile();
|
||||
// Mark instant as inflight
|
||||
commitTimeline.saveToInflight(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, inflightDeltaInstantTime),
|
||||
Option.empty());
|
||||
commitTimeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
inflightDeltaInstantTime));
|
||||
commitTimeline.transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
inflightDeltaInstantTime), Option.empty());
|
||||
refreshFsView();
|
||||
|
||||
List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
@@ -480,18 +481,16 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
}
|
||||
|
||||
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
|
||||
// Now simulate Compaction completing - Check the view
|
||||
if (!isCompactionInFlight) {
|
||||
// For inflight compaction, we already create a data-file to test concurrent inflight case.
|
||||
// If we skipped creating data file corresponding to compaction commit, create it now
|
||||
new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile();
|
||||
commitTimeline.createNewInstant(compactionInstant);
|
||||
}
|
||||
if (isCompactionInFlight) {
|
||||
commitTimeline.deleteInflight(compactionInstant);
|
||||
} else {
|
||||
commitTimeline.deleteCompactionRequested(compactionInstant);
|
||||
}
|
||||
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
|
||||
|
||||
commitTimeline.saveAsComplete(compactionInstant, Option.empty());
|
||||
refreshFsView();
|
||||
// populate the cache
|
||||
@@ -566,7 +565,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
// Make this commit safe
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1, roView.getLatestDataFiles(partitionPath)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
|
||||
@@ -581,7 +580,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
|
||||
// Make it safe
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
|
||||
commitTimeline.saveAsComplete(instant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant2, Option.empty());
|
||||
refreshFsView();
|
||||
assertEquals("", fileName2, roView.getLatestDataFiles(partitionPath)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
|
||||
@@ -1043,9 +1042,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
|
||||
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
|
||||
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
|
||||
|
||||
// Now we list all partitions
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(
|
||||
@@ -1090,7 +1089,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(requested);
|
||||
|
||||
// Fake delta-ingestion after compaction-requested
|
||||
String deltaInstantTime4 = "3";
|
||||
String deltaInstantTime4 = "4";
|
||||
String deltaInstantTime5 = "6";
|
||||
List<String> allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2,
|
||||
compactionRequestedTime, deltaInstantTime4, deltaInstantTime5);
|
||||
@@ -1107,8 +1106,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
|
||||
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant4, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant5, Option.empty());
|
||||
refreshFsView();
|
||||
|
||||
// Test Data Files
|
||||
@@ -1168,6 +1167,17 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
Assert.assertTrue(fileIdsInCompaction.contains(fileId));
|
||||
}
|
||||
|
||||
private static void saveAsComplete(HoodieActiveTimeline timeline, HoodieInstant inflight, Option<byte[]> data) {
|
||||
if (inflight.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
timeline.transitionCompactionInflightToComplete(inflight, data);
|
||||
} else {
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp());
|
||||
timeline.createNewInstant(requested);
|
||||
timeline.transitionRequestedToInflight(requested, Option.empty());
|
||||
timeline.saveAsComplete(inflight, data);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
|
||||
@@ -158,6 +158,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
// Add an empty ingestion
|
||||
String firstEmptyInstantTs = "11";
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
@@ -421,9 +423,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
new ArrayList<>(), Integer.toString(Integer.parseInt(instant) + 1));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieInstant cleanInflightInstant = new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant);
|
||||
metaClient.getActiveTimeline().createNewInstant(cleanInflightInstant);
|
||||
HoodieCleanMetadata cleanMetadata = CleanerUtils
|
||||
.convertCleanMetadata(metaClient, cleanInstant, Option.empty(), cleanStats);
|
||||
metaClient.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant),
|
||||
metaClient.getActiveTimeline().saveAsComplete(cleanInflightInstant,
|
||||
AvroUtils.serializeCleanMetadata(cleanMetadata));
|
||||
}
|
||||
|
||||
@@ -458,10 +462,12 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
metadata.setInstantsToRollback(rollbackInstants);
|
||||
metadata.setStartRestoreTime(rollbackInstant);
|
||||
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRestoreMetadata(metadata));
|
||||
HoodieInstant restoreInstant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant);
|
||||
metaClient.getActiveTimeline().createNewInstant(restoreInstant);
|
||||
metaClient.getActiveTimeline().saveAsComplete(restoreInstant, AvroUtils.serializeRestoreMetadata(metadata));
|
||||
} else {
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
@@ -741,13 +747,16 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
|
||||
metaClient.getActiveTimeline()
|
||||
.saveAsComplete(new HoodieInstant(true,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, instant),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
HoodieInstant inflightInstant = new HoodieInstant(true,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, instant);
|
||||
metaClient.getActiveTimeline().createNewInstant(inflightInstant);
|
||||
metaClient.getActiveTimeline().saveAsComplete(inflightInstant,
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
/**
|
||||
// Delete pending compaction if present
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant).getFileName()));
|
||||
*/
|
||||
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
||||
@@ -129,8 +129,11 @@ public class CompactionTestUtils {
|
||||
}
|
||||
|
||||
public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException {
|
||||
metaClient.getActiveTimeline().saveAsComplete(new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime),
|
||||
Option.empty());
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, DELTA_COMMIT_ACTION, instantTime);
|
||||
metaClient.getActiveTimeline().createNewInstant(requested);
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(requested, Option.empty());
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Option.empty());
|
||||
}
|
||||
|
||||
public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime,
|
||||
|
||||
@@ -100,7 +100,7 @@ public class SchemaTestUtil {
|
||||
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
List<IndexedRecord> records = generateTestRecords(from, limit);
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
return records.stream().map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)).map(p -> {
|
||||
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
|
||||
|
||||
@@ -118,7 +118,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
|
||||
|
||||
// generate updates from inserts
|
||||
List<IndexedRecord> updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys,
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime());
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewInstantTime());
|
||||
String newCommitTime =
|
||||
((GenericRecord) updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
|
||||
|
||||
@@ -101,7 +101,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
List<IndexedRecord> updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys,
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime());
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewInstantTime());
|
||||
|
||||
// update records already inserted
|
||||
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
@@ -214,7 +214,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
List<IndexedRecord> recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
|
||||
|
||||
String newCommitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<String> keysToBeUpdated = new ArrayList<>();
|
||||
keysToBeUpdated.add(key);
|
||||
// Update the commitTime for this record
|
||||
@@ -232,7 +232,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
|
||||
|
||||
newCommitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
keysToBeUpdated = new ArrayList<>();
|
||||
keysToBeUpdated.add(key);
|
||||
// Update the commitTime for this record
|
||||
|
||||
Reference in New Issue
Block a user