[HUDI-2767] Enabling timeline-server-based marker as default (#4112)
- Changes the default config of marker type (HoodieWriteConfig.MARKERS_TYPE or hoodie.write.markers.type) from DIRECT to TIMELINE_SERVER_BASED for Spark Engine. - Adds engine-specific marker type configs: Spark -> TIMELINE_SERVER_BASED, Flink -> DIRECT, Java -> DIRECT. - Uses DIRECT markers as well for Spark structured streaming due to timeline server only available for the first mini-batch. - Fixes the marker creation method for non-partitioned table in TimelineServerBasedWriteMarkers. - Adds the fallback to direct markers even when TIMELINE_SERVER_BASED is configured, in WriteMarkersFactory: when HDFS is used, or embedded timeline server is disabled, the fallback to direct markers happens. - Fixes the closing of timeline service. - Fixes tests that depend on markers, mainly by starting the timeline service for each test.
This commit is contained in:
@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
|||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieSparkTable;
|
import org.apache.hudi.table.HoodieSparkTable;
|
||||||
@@ -72,6 +73,8 @@ public class TestArchivedCommitsCommand extends CLIFunctionalTestHarness {
|
|||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||||
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.forTable("test-trip-table").build();
|
.forTable("test-trip-table").build();
|
||||||
|
|
||||||
|
|||||||
@@ -32,6 +32,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||||
import org.apache.hudi.common.util.NumericUtils;
|
import org.apache.hudi.common.util.NumericUtils;
|
||||||
@@ -209,6 +210,8 @@ public class TestCommitsCommand extends CLIFunctionalTestHarness {
|
|||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1)
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1)
|
||||||
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.forTable("test-trip-table").build();
|
.forTable("test-trip-table").build();
|
||||||
|
|
||||||
|
|||||||
@@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.CompactionTestUtils;
|
import org.apache.hudi.common.testutils.CompactionTestUtils;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||||
@@ -159,6 +160,8 @@ public class TestCompactionCommand extends CLIFunctionalTestHarness {
|
|||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||||
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.forTable("test-trip-table").build();
|
.forTable("test-trip-table").build();
|
||||||
// archive
|
// archive
|
||||||
|
|||||||
@@ -28,8 +28,10 @@ import org.apache.hudi.common.table.HoodieTableConfig;
|
|||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.HoodieTableVersion;
|
import org.apache.hudi.common.table.HoodieTableVersion;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
|
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
@@ -62,6 +64,8 @@ public class TestUpgradeDowngradeCommand extends CLIFunctionalTestHarness {
|
|||||||
new TableCommand().createTable(
|
new TableCommand().createTable(
|
||||||
tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
|
tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
|
||||||
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
|
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
|
||||||
|
timelineService = HoodieClientTestUtils.initTimelineService(
|
||||||
|
context, basePath(), FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue());
|
||||||
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
|
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
|
||||||
//Create some commits files and base files
|
//Create some commits files and base files
|
||||||
HoodieTestTable.of(metaClient)
|
HoodieTestTable.of(metaClient)
|
||||||
|
|||||||
@@ -22,7 +22,10 @@ package org.apache.hudi.cli.functional;
|
|||||||
import org.apache.hudi.client.HoodieReadClient;
|
import org.apache.hudi.client.HoodieReadClient;
|
||||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
|
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||||
import org.apache.hudi.testutils.providers.SparkProvider;
|
import org.apache.hudi.testutils.providers.SparkProvider;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
@@ -39,10 +42,13 @@ import java.nio.file.Paths;
|
|||||||
|
|
||||||
public class CLIFunctionalTestHarness implements SparkProvider {
|
public class CLIFunctionalTestHarness implements SparkProvider {
|
||||||
|
|
||||||
|
protected static int timelineServicePort =
|
||||||
|
FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue();
|
||||||
|
protected static transient TimelineService timelineService;
|
||||||
|
protected static transient HoodieSparkEngineContext context;
|
||||||
private static transient SparkSession spark;
|
private static transient SparkSession spark;
|
||||||
private static transient SQLContext sqlContext;
|
private static transient SQLContext sqlContext;
|
||||||
private static transient JavaSparkContext jsc;
|
private static transient JavaSparkContext jsc;
|
||||||
private static transient HoodieSparkEngineContext context;
|
|
||||||
private static transient JLineShellComponent shell;
|
private static transient JLineShellComponent shell;
|
||||||
/**
|
/**
|
||||||
* An indicator of the initialization status.
|
* An indicator of the initialization status.
|
||||||
@@ -107,6 +113,9 @@ public class CLIFunctionalTestHarness implements SparkProvider {
|
|||||||
jsc = new JavaSparkContext(spark.sparkContext());
|
jsc = new JavaSparkContext(spark.sparkContext());
|
||||||
context = new HoodieSparkEngineContext(jsc);
|
context = new HoodieSparkEngineContext(jsc);
|
||||||
shell = new Bootstrap().getJLineShellComponent();
|
shell = new Bootstrap().getJLineShellComponent();
|
||||||
|
timelineService = HoodieClientTestUtils.initTimelineService(
|
||||||
|
context, basePath(), incrementTimelineServicePortToUse());
|
||||||
|
timelineServicePort = timelineService.getServerPort();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -120,14 +129,25 @@ public class CLIFunctionalTestHarness implements SparkProvider {
|
|||||||
shell.stop();
|
shell.stop();
|
||||||
shell = null;
|
shell = null;
|
||||||
}
|
}
|
||||||
|
if (timelineService != null) {
|
||||||
|
timelineService.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper to prepare string for matching.
|
* Helper to prepare string for matching.
|
||||||
|
*
|
||||||
* @param str Input string.
|
* @param str Input string.
|
||||||
* @return pruned string with non word characters removed.
|
* @return pruned string with non word characters removed.
|
||||||
*/
|
*/
|
||||||
protected static String removeNonWordAndStripSpace(String str) {
|
protected static String removeNonWordAndStripSpace(String str) {
|
||||||
return str.replaceAll("[\\s]+", ",").replaceAll("[\\W]+", ",");
|
return str.replaceAll("[\\s]+", ",").replaceAll("[\\W]+", ",");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected int incrementTimelineServicePortToUse() {
|
||||||
|
// Increment the timeline service port for each individual test
|
||||||
|
// to avoid port reuse causing failures
|
||||||
|
timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024;
|
||||||
|
return timelineServicePort;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -45,6 +45,7 @@ import org.apache.hudi.config.metrics.HoodieMetricsDatadogConfig;
|
|||||||
import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig;
|
import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig;
|
||||||
import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig;
|
import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig;
|
||||||
import org.apache.hudi.config.metrics.HoodieMetricsPrometheusConfig;
|
import org.apache.hudi.config.metrics.HoodieMetricsPrometheusConfig;
|
||||||
|
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||||
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
|
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
|
||||||
import org.apache.hudi.index.HoodieIndex;
|
import org.apache.hudi.index.HoodieIndex;
|
||||||
import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
|
import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
|
||||||
@@ -248,14 +249,17 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
|
|
||||||
public static final ConfigProperty<String> MARKERS_TYPE = ConfigProperty
|
public static final ConfigProperty<String> MARKERS_TYPE = ConfigProperty
|
||||||
.key("hoodie.write.markers.type")
|
.key("hoodie.write.markers.type")
|
||||||
.defaultValue(MarkerType.DIRECT.toString())
|
.defaultValue(MarkerType.TIMELINE_SERVER_BASED.toString())
|
||||||
.sinceVersion("0.9.0")
|
.sinceVersion("0.9.0")
|
||||||
.withDocumentation("Marker type to use. Two modes are supported: "
|
.withDocumentation("Marker type to use. Two modes are supported: "
|
||||||
+ "- DIRECT: individual marker file corresponding to each data file is directly "
|
+ "- DIRECT: individual marker file corresponding to each data file is directly "
|
||||||
+ "created by the writer. "
|
+ "created by the writer. "
|
||||||
+ "- TIMELINE_SERVER_BASED: marker operations are all handled at the timeline service "
|
+ "- TIMELINE_SERVER_BASED: marker operations are all handled at the timeline service "
|
||||||
+ "which serves as a proxy. New marker entries are batch processed and stored "
|
+ "which serves as a proxy. New marker entries are batch processed and stored "
|
||||||
+ "in a limited number of underlying files for efficiency.");
|
+ "in a limited number of underlying files for efficiency. If HDFS is used or "
|
||||||
|
+ "timeline server is disabled, DIRECT markers are used as fallback even if this "
|
||||||
|
+ "is configure. For Spark structured streaming, this configuration does not "
|
||||||
|
+ "take effect, i.e., DIRECT markers are always used for Spark structured streaming.");
|
||||||
|
|
||||||
public static final ConfigProperty<Integer> MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS = ConfigProperty
|
public static final ConfigProperty<Integer> MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS = ConfigProperty
|
||||||
.key("hoodie.markers.timeline_server_based.batch.num_threads")
|
.key("hoodie.markers.timeline_server_based.batch.num_threads")
|
||||||
@@ -2175,6 +2179,7 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void setDefaults() {
|
protected void setDefaults() {
|
||||||
|
writeConfig.setDefaultValue(MARKERS_TYPE, getDefaultMarkersType(engineType));
|
||||||
// Check for mandatory properties
|
// Check for mandatory properties
|
||||||
writeConfig.setDefaults(HoodieWriteConfig.class.getName());
|
writeConfig.setDefaults(HoodieWriteConfig.class.getName());
|
||||||
// Make sure the props is propagated
|
// Make sure the props is propagated
|
||||||
@@ -2229,5 +2234,18 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
// Build WriteConfig at the end
|
// Build WriteConfig at the end
|
||||||
return new HoodieWriteConfig(engineType, writeConfig.getProps());
|
return new HoodieWriteConfig(engineType, writeConfig.getProps());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private String getDefaultMarkersType(EngineType engineType) {
|
||||||
|
switch (engineType) {
|
||||||
|
case SPARK:
|
||||||
|
return MarkerType.TIMELINE_SERVER_BASED.toString();
|
||||||
|
case FLINK:
|
||||||
|
case JAVA:
|
||||||
|
// Timeline-server-based marker is not supported for Flink and Java engines
|
||||||
|
return MarkerType.DIRECT.toString();
|
||||||
|
default:
|
||||||
|
throw new HoodieNotSupportedException("Unsupported engine " + engineType);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,9 +19,11 @@
|
|||||||
package org.apache.hudi.table.marker;
|
package org.apache.hudi.table.marker;
|
||||||
|
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.IOType;
|
import org.apache.hudi.common.model.IOType;
|
||||||
import org.apache.hudi.common.util.HoodieTimer;
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.StringUtils;
|
||||||
import org.apache.hudi.exception.HoodieRemoteException;
|
import org.apache.hudi.exception.HoodieRemoteException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
@@ -132,18 +134,24 @@ public class TimelineServerBasedWriteMarkers extends WriteMarkers {
|
|||||||
|
|
||||||
Map<String, String> paramsMap = new HashMap<>();
|
Map<String, String> paramsMap = new HashMap<>();
|
||||||
paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
||||||
paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName);
|
if (StringUtils.isNullOrEmpty(partitionPath)) {
|
||||||
|
paramsMap.put(MARKER_NAME_PARAM, markerFileName);
|
||||||
|
} else {
|
||||||
|
paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName);
|
||||||
|
}
|
||||||
|
|
||||||
boolean success;
|
boolean success;
|
||||||
try {
|
try {
|
||||||
success = executeRequestToTimelineServer(
|
success = executeRequestToTimelineServer(
|
||||||
CREATE_MARKER_URL, paramsMap, new TypeReference<Boolean>() {}, RequestMethod.POST);
|
CREATE_MARKER_URL, paramsMap, new TypeReference<Boolean>() {
|
||||||
|
}, RequestMethod.POST);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieRemoteException("Failed to create marker file " + partitionPath + "/" + markerFileName, e);
|
throw new HoodieRemoteException("Failed to create marker file " + partitionPath + "/" + markerFileName, e);
|
||||||
}
|
}
|
||||||
LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName
|
LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName
|
||||||
+ " in " + timer.endTimer() + " ms");
|
+ " in " + timer.endTimer() + " ms");
|
||||||
if (success) {
|
if (success) {
|
||||||
return Option.of(new Path(new Path(markerDirPath, partitionPath), markerFileName));
|
return Option.of(new Path(FSUtils.getPartitionPath(markerDirPath, partitionPath), markerFileName));
|
||||||
} else {
|
} else {
|
||||||
return Option.empty();
|
return Option.empty();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,10 +18,13 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table.marker;
|
package org.apache.hudi.table.marker;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.fs.StorageSchemes;
|
||||||
import org.apache.hudi.common.table.marker.MarkerType;
|
import org.apache.hudi.common.table.marker.MarkerType;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
|
import com.esotericsoftware.minlog.Log;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -43,6 +46,18 @@ public class WriteMarkersFactory {
|
|||||||
case DIRECT:
|
case DIRECT:
|
||||||
return new DirectWriteMarkers(table, instantTime);
|
return new DirectWriteMarkers(table, instantTime);
|
||||||
case TIMELINE_SERVER_BASED:
|
case TIMELINE_SERVER_BASED:
|
||||||
|
if (!table.getConfig().isEmbeddedTimelineServerEnabled()) {
|
||||||
|
Log.warn("Timeline-server-based markers are configured as the marker type "
|
||||||
|
+ "but embedded timeline server is not enabled. Falling back to direct markers.");
|
||||||
|
return new DirectWriteMarkers(table, instantTime);
|
||||||
|
}
|
||||||
|
String basePath = table.getMetaClient().getBasePath();
|
||||||
|
if (StorageSchemes.HDFS.getScheme().equals(
|
||||||
|
FSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) {
|
||||||
|
Log.warn("Timeline-server-based markers are not supported for HDFS: "
|
||||||
|
+ "base path " + basePath + ". Falling back to direct markers.");
|
||||||
|
return new DirectWriteMarkers(table, instantTime);
|
||||||
|
}
|
||||||
return new TimelineServerBasedWriteMarkers(table, instantTime);
|
return new TimelineServerBasedWriteMarkers(table, instantTime);
|
||||||
default:
|
default:
|
||||||
throw new HoodieException("The marker type \"" + markerType.name() + "\" is not supported.");
|
throw new HoodieException("The marker type \"" + markerType.name() + "\" is not supported.");
|
||||||
|
|||||||
@@ -19,8 +19,10 @@
|
|||||||
package org.apache.hudi.config;
|
package org.apache.hudi.config;
|
||||||
|
|
||||||
import org.apache.hudi.common.engine.EngineType;
|
import org.apache.hudi.common.engine.EngineType;
|
||||||
|
import org.apache.hudi.common.table.marker.MarkerType;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig.Builder;
|
import org.apache.hudi.config.HoodieWriteConfig.Builder;
|
||||||
import org.apache.hudi.index.HoodieIndex;
|
import org.apache.hudi.index.HoodieIndex;
|
||||||
|
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
import org.junit.jupiter.params.provider.ValueSource;
|
import org.junit.jupiter.params.provider.ValueSource;
|
||||||
@@ -32,6 +34,7 @@ import java.util.Date;
|
|||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
@@ -67,63 +70,38 @@ public class TestHoodieWriteConfig {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDefaultIndexAccordingToEngineType() {
|
public void testDefaultIndexAccordingToEngineType() {
|
||||||
// default bloom
|
testEngineSpecificConfig(HoodieWriteConfig::getIndexType,
|
||||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build();
|
constructConfigMap(
|
||||||
assertEquals(HoodieIndex.IndexType.BLOOM, writeConfig.getIndexType());
|
EngineType.SPARK, HoodieIndex.IndexType.BLOOM,
|
||||||
|
EngineType.FLINK, HoodieIndex.IndexType.INMEMORY,
|
||||||
// spark default bloom
|
EngineType.JAVA, HoodieIndex.IndexType.INMEMORY));
|
||||||
writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.SPARK).withPath("/tmp").build();
|
|
||||||
assertEquals(HoodieIndex.IndexType.BLOOM, writeConfig.getIndexType());
|
|
||||||
|
|
||||||
// flink default in-memory
|
|
||||||
writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.FLINK).withPath("/tmp").build();
|
|
||||||
assertEquals(HoodieIndex.IndexType.INMEMORY, writeConfig.getIndexType());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDefaultClusteringPlanStrategyClassAccordingToEngineType() {
|
public void testDefaultClusteringPlanStrategyClassAccordingToEngineType() {
|
||||||
// Default (as Spark)
|
testEngineSpecificConfig(HoodieWriteConfig::getClusteringPlanStrategyClass,
|
||||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build();
|
constructConfigMap(
|
||||||
assertEquals(
|
EngineType.SPARK, HoodieClusteringConfig.SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY,
|
||||||
HoodieClusteringConfig.SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY,
|
EngineType.FLINK, HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY,
|
||||||
writeConfig.getClusteringPlanStrategyClass());
|
EngineType.JAVA, HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY));
|
||||||
|
|
||||||
// Spark
|
|
||||||
writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.SPARK).withPath("/tmp").build();
|
|
||||||
assertEquals(
|
|
||||||
HoodieClusteringConfig.SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY,
|
|
||||||
writeConfig.getClusteringPlanStrategyClass());
|
|
||||||
|
|
||||||
// Flink and Java
|
|
||||||
for (EngineType engineType : new EngineType[] {EngineType.FLINK, EngineType.JAVA}) {
|
|
||||||
writeConfig = HoodieWriteConfig.newBuilder().withEngineType(engineType).withPath("/tmp").build();
|
|
||||||
assertEquals(
|
|
||||||
HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY,
|
|
||||||
writeConfig.getClusteringPlanStrategyClass());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDefaultClusteringExecutionStrategyClassAccordingToEngineType() {
|
public void testDefaultClusteringExecutionStrategyClassAccordingToEngineType() {
|
||||||
// Default (as Spark)
|
testEngineSpecificConfig(HoodieWriteConfig::getClusteringExecutionStrategyClass,
|
||||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build();
|
constructConfigMap(
|
||||||
assertEquals(
|
EngineType.SPARK, HoodieClusteringConfig.SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY,
|
||||||
HoodieClusteringConfig.SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY,
|
EngineType.FLINK, HoodieClusteringConfig.JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY,
|
||||||
writeConfig.getClusteringExecutionStrategyClass());
|
EngineType.JAVA, HoodieClusteringConfig.JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY));
|
||||||
|
}
|
||||||
|
|
||||||
// Spark
|
@Test
|
||||||
writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.SPARK).withPath("/tmp").build();
|
public void testDefaultMarkersTypeAccordingToEngineType() {
|
||||||
assertEquals(
|
testEngineSpecificConfig(HoodieWriteConfig::getMarkersType,
|
||||||
HoodieClusteringConfig.SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY,
|
constructConfigMap(
|
||||||
writeConfig.getClusteringExecutionStrategyClass());
|
EngineType.SPARK, MarkerType.TIMELINE_SERVER_BASED,
|
||||||
|
EngineType.FLINK, MarkerType.DIRECT,
|
||||||
// Flink and Java
|
EngineType.JAVA, MarkerType.DIRECT));
|
||||||
for (EngineType engineType : new EngineType[] {EngineType.FLINK, EngineType.JAVA}) {
|
|
||||||
writeConfig = HoodieWriteConfig.newBuilder().withEngineType(engineType).withPath("/tmp").build();
|
|
||||||
assertEquals(
|
|
||||||
HoodieClusteringConfig.JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY,
|
|
||||||
writeConfig.getClusteringExecutionStrategyClass());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params) throws IOException {
|
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params) throws IOException {
|
||||||
@@ -133,4 +111,44 @@ public class TestHoodieWriteConfig {
|
|||||||
properties.store(outStream, "Saved on " + new Date(System.currentTimeMillis()));
|
properties.store(outStream, "Saved on " + new Date(System.currentTimeMillis()));
|
||||||
return outStream;
|
return outStream;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests the engine-specific configuration values for one configuration key .
|
||||||
|
*
|
||||||
|
* @param getConfigFunc Function to get the config value.
|
||||||
|
* @param expectedConfigMap Expected config map, with key as the engine type
|
||||||
|
* and value as the corresponding config value for the engine.
|
||||||
|
*/
|
||||||
|
private void testEngineSpecificConfig(Function<HoodieWriteConfig, Object> getConfigFunc,
|
||||||
|
Map<EngineType, Object> expectedConfigMap) {
|
||||||
|
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build();
|
||||||
|
assertEquals(expectedConfigMap.get(EngineType.SPARK), getConfigFunc.apply(writeConfig));
|
||||||
|
|
||||||
|
for (EngineType engineType : expectedConfigMap.keySet()) {
|
||||||
|
writeConfig = HoodieWriteConfig.newBuilder()
|
||||||
|
.withEngineType(engineType).withPath("/tmp").build();
|
||||||
|
assertEquals(expectedConfigMap.get(engineType), getConfigFunc.apply(writeConfig));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs the map.
|
||||||
|
*
|
||||||
|
* @param k1 First engine type.
|
||||||
|
* @param v1 Config value for the first engine type.
|
||||||
|
* @param k2 Second engine type.
|
||||||
|
* @param v2 Config value for the second engine type.
|
||||||
|
* @param k3 Third engine type.
|
||||||
|
* @param v3 Config value for the third engine type.
|
||||||
|
* @return {@link Map<EngineType, Object>} instance, with key as the engine type
|
||||||
|
* and value as the corresponding config value for the engine.
|
||||||
|
*/
|
||||||
|
private Map<EngineType, Object> constructConfigMap(
|
||||||
|
EngineType k1, Object v1, EngineType k2, Object v2, EngineType k3, Object v3) {
|
||||||
|
Map<EngineType, Object> mapping = new HashMap<>();
|
||||||
|
mapping.put(k1, v1);
|
||||||
|
mapping.put(k2, v2);
|
||||||
|
mapping.put(k3, v3);
|
||||||
|
return mapping;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -0,0 +1,116 @@
|
|||||||
|
/*
|
||||||
|
* 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.table.marker;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||||
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
|
import org.apache.hudi.common.table.marker.MarkerType;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
|
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||||
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
|
import org.junit.jupiter.params.provider.Arguments;
|
||||||
|
import org.junit.jupiter.params.provider.MethodSource;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
|
|
||||||
|
public class TestWriteMarkersFactory extends HoodieCommonTestHarness {
|
||||||
|
private static final String NON_HDFS_BASE_PATH = "/tmp/dir";
|
||||||
|
private static final String HDFS_BASE_PATH = "hdfs://localhost/dir";
|
||||||
|
private final HoodieWriteConfig writeConfig = Mockito.mock(HoodieWriteConfig.class);
|
||||||
|
private final HoodieTableMetaClient metaClient = Mockito.mock(HoodieTableMetaClient.class);
|
||||||
|
private final HoodieWrapperFileSystem fileSystem = Mockito.mock(HoodieWrapperFileSystem.class);
|
||||||
|
private final HoodieEngineContext context = Mockito.mock(HoodieEngineContext.class);
|
||||||
|
private final HoodieTable table = Mockito.mock(HoodieTable.class);
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void init() throws IOException {
|
||||||
|
initMetaClient();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Stream<Arguments> configParams() {
|
||||||
|
Object[][] data = new Object[][] {
|
||||||
|
{NON_HDFS_BASE_PATH, true}, {HDFS_BASE_PATH, false},
|
||||||
|
{NON_HDFS_BASE_PATH, true}, {HDFS_BASE_PATH, false},
|
||||||
|
};
|
||||||
|
return Stream.of(data).map(Arguments::of);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("configParams")
|
||||||
|
public void testDirectMarkers(String basePath, boolean isTimelineServerEnabled) {
|
||||||
|
testWriteMarkersFactory(
|
||||||
|
MarkerType.DIRECT, basePath, isTimelineServerEnabled, DirectWriteMarkers.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimelineServerBasedMarkersWithTimelineServerEnabled() {
|
||||||
|
testWriteMarkersFactory(
|
||||||
|
MarkerType.TIMELINE_SERVER_BASED, NON_HDFS_BASE_PATH, true,
|
||||||
|
TimelineServerBasedWriteMarkers.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimelineServerBasedMarkersWithTimelineServerDisabled() {
|
||||||
|
// Fallback to direct markers should happen
|
||||||
|
testWriteMarkersFactory(
|
||||||
|
MarkerType.TIMELINE_SERVER_BASED, NON_HDFS_BASE_PATH, false,
|
||||||
|
DirectWriteMarkers.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimelineServerBasedMarkersWithHDFS() {
|
||||||
|
// Fallback to direct markers should happen
|
||||||
|
testWriteMarkersFactory(
|
||||||
|
MarkerType.TIMELINE_SERVER_BASED, HDFS_BASE_PATH, true,
|
||||||
|
DirectWriteMarkers.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testWriteMarkersFactory(
|
||||||
|
MarkerType markerTypeConfig, String basePath, boolean isTimelineServerEnabled,
|
||||||
|
Class<?> expectedWriteMarkersClass) {
|
||||||
|
String instantTime = "001";
|
||||||
|
Mockito.when(table.getConfig()).thenReturn(writeConfig);
|
||||||
|
Mockito.when(writeConfig.isEmbeddedTimelineServerEnabled())
|
||||||
|
.thenReturn(isTimelineServerEnabled);
|
||||||
|
Mockito.when(table.getMetaClient()).thenReturn(metaClient);
|
||||||
|
Mockito.when(metaClient.getFs()).thenReturn(fileSystem);
|
||||||
|
Mockito.when(metaClient.getBasePath()).thenReturn(basePath);
|
||||||
|
Mockito.when(metaClient.getMarkerFolderPath(any())).thenReturn(basePath + ".hoodie/.temp");
|
||||||
|
Mockito.when(table.getContext()).thenReturn(context);
|
||||||
|
Mockito.when(context.getHadoopConf()).thenReturn(new SerializableConfiguration(new Configuration()));
|
||||||
|
Mockito.when(writeConfig.getViewStorageConfig())
|
||||||
|
.thenReturn(FileSystemViewStorageConfig.newBuilder().build());
|
||||||
|
assertEquals(expectedWriteMarkersClass,
|
||||||
|
WriteMarkersFactory.get(markerTypeConfig, table, instantTime).getClass());
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
|
|||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
import org.apache.hudi.common.model.TableServiceType;
|
import org.apache.hudi.common.model.TableServiceType;
|
||||||
import org.apache.hudi.common.model.WriteConcurrencyMode;
|
import org.apache.hudi.common.model.WriteConcurrencyMode;
|
||||||
|
import org.apache.hudi.common.table.marker.MarkerType;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||||
@@ -91,6 +92,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
|||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||||
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
|
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
|
||||||
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
|
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
|
||||||
|
// Timeline-server-based markers are not used for multi-writer tests
|
||||||
|
.withMarkersType(MarkerType.DIRECT.name())
|
||||||
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
||||||
.build()).withAutoCommit(false).withProperties(properties).build();
|
.build()).withAutoCommit(false).withProperties(properties).build();
|
||||||
// Create the first commit
|
// Create the first commit
|
||||||
@@ -162,6 +165,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
|||||||
.withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
.withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
||||||
.build())
|
.build())
|
||||||
.withAutoCommit(false)
|
.withAutoCommit(false)
|
||||||
|
// Timeline-server-based markers are not used for multi-writer tests
|
||||||
|
.withMarkersType(MarkerType.DIRECT.name())
|
||||||
.withProperties(properties)
|
.withProperties(properties)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
@@ -208,11 +213,13 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
|||||||
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000");
|
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000");
|
||||||
// Disabling embedded timeline server, it doesn't work with multiwriter
|
// Disabling embedded timeline server, it doesn't work with multiwriter
|
||||||
HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder()
|
HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder()
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false)
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false)
|
||||||
.withInlineCompaction(false).withAsyncClean(true)
|
.withInlineCompaction(false).withAsyncClean(true)
|
||||||
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
|
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
|
||||||
.withMaxNumDeltaCommitsBeforeCompaction(2).build())
|
.withMaxNumDeltaCommitsBeforeCompaction(2).build())
|
||||||
.withEmbeddedTimelineServerEnabled(false)
|
.withEmbeddedTimelineServerEnabled(false)
|
||||||
|
// Timeline-server-based markers are not used for multi-writer tests
|
||||||
|
.withMarkersType(MarkerType.DIRECT.name())
|
||||||
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(
|
||||||
FileSystemViewStorageType.MEMORY).build())
|
FileSystemViewStorageType.MEMORY).build())
|
||||||
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
|
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
|
||||||
@@ -326,6 +333,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
|||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
|
||||||
.withAutoClean(false).build())
|
.withAutoClean(false).build())
|
||||||
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
|
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
|
||||||
|
// Timeline-server-based markers are not used for multi-writer tests
|
||||||
|
.withMarkersType(MarkerType.DIRECT.name())
|
||||||
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
||||||
.build()).withAutoCommit(false).withProperties(properties);
|
.build()).withAutoCommit(false).withProperties(properties);
|
||||||
HoodieWriteConfig cfg = writeConfigBuilder.build();
|
HoodieWriteConfig cfg = writeConfigBuilder.build();
|
||||||
|
|||||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
|||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||||
import org.apache.hudi.common.util.BaseFileUtils;
|
import org.apache.hudi.common.util.BaseFileUtils;
|
||||||
@@ -64,6 +65,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
|||||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
||||||
initSparkContexts("TestUpdateSchemaEvolution");
|
initSparkContexts("TestUpdateSchemaEvolution");
|
||||||
initFileSystem();
|
initFileSystem();
|
||||||
|
initTimelineService();
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterEach
|
@AfterEach
|
||||||
@@ -228,6 +230,9 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
|||||||
|
|
||||||
private HoodieWriteConfig makeHoodieClientConfig(String name) {
|
private HoodieWriteConfig makeHoodieClientConfig(String name) {
|
||||||
Schema schema = getSchemaFromResource(getClass(), name);
|
Schema schema = getSchemaFromResource(getClass(), name);
|
||||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()).build();
|
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
|
.withSchema(schema.toString()).build();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -1316,6 +1316,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
|||||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
|
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
|
||||||
|
.withRemoteServerPort(timelineServicePort)
|
||||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -47,11 +47,13 @@ import org.apache.hudi.common.model.HoodieWriteStat;
|
|||||||
import org.apache.hudi.common.model.IOType;
|
import org.apache.hudi.common.model.IOType;
|
||||||
import org.apache.hudi.common.model.WriteOperationType;
|
import org.apache.hudi.common.model.WriteOperationType;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
|
import org.apache.hudi.common.table.marker.MarkerType;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
@@ -61,6 +63,7 @@ import org.apache.hudi.common.util.BaseFileUtils;
|
|||||||
import org.apache.hudi.common.util.ClusteringUtils;
|
import org.apache.hudi.common.util.ClusteringUtils;
|
||||||
import org.apache.hudi.common.util.CollectionUtils;
|
import org.apache.hudi.common.util.CollectionUtils;
|
||||||
import org.apache.hudi.common.util.FileIOUtils;
|
import org.apache.hudi.common.util.FileIOUtils;
|
||||||
|
import org.apache.hudi.common.util.MarkerUtils;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.util.StringUtils;
|
import org.apache.hudi.common.util.StringUtils;
|
||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
@@ -2336,13 +2339,22 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
|||||||
|
|
||||||
// Create a dummy marker file to simulate the case that a marker file was created without data file.
|
// Create a dummy marker file to simulate the case that a marker file was created without data file.
|
||||||
// This should fail the commit
|
// This should fail the commit
|
||||||
String partitionPath = Arrays
|
String partitionPath;
|
||||||
.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", metaClient.getMarkerFolderPath(instantTime))),
|
String markerFolderPath = metaClient.getMarkerFolderPath(instantTime);
|
||||||
path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN)))
|
if (cfg.getMarkersType() == MarkerType.TIMELINE_SERVER_BASED) {
|
||||||
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
|
String markerName = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(
|
||||||
|
markerFolderPath, fs, context, 1).values().stream()
|
||||||
|
.flatMap(Collection::stream).findFirst().get();
|
||||||
|
partitionPath = new Path(markerFolderPath, markerName).getParent().toString();
|
||||||
|
} else {
|
||||||
|
partitionPath = Arrays
|
||||||
|
.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", markerFolderPath)),
|
||||||
|
path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN)))
|
||||||
|
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
|
||||||
|
}
|
||||||
|
|
||||||
Option<Path> markerFilePath = WriteMarkersFactory.get(
|
Option<Path> markerFilePath = WriteMarkersFactory.get(
|
||||||
cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime)
|
cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime)
|
||||||
.create(partitionPath,
|
.create(partitionPath,
|
||||||
FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()),
|
FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()),
|
||||||
IOType.MERGE);
|
IOType.MERGE);
|
||||||
@@ -2489,6 +2501,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
|||||||
.withAutoClean(false).build())
|
.withAutoClean(false).build())
|
||||||
.withTimelineLayoutVersion(1)
|
.withTimelineLayoutVersion(1)
|
||||||
.withHeartbeatIntervalInMs(3 * 1000)
|
.withHeartbeatIntervalInMs(3 * 1000)
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withAutoCommit(false)
|
.withAutoCommit(false)
|
||||||
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build();
|
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -85,6 +85,7 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
|||||||
initSparkContexts("TestHoodieMetadata");
|
initSparkContexts("TestHoodieMetadata");
|
||||||
initFileSystem();
|
initFileSystem();
|
||||||
fs.mkdirs(new Path(basePath));
|
fs.mkdirs(new Path(basePath));
|
||||||
|
initTimelineService();
|
||||||
initMetaClient(tableType);
|
initMetaClient(tableType);
|
||||||
initTestDataGenerator();
|
initTestDataGenerator();
|
||||||
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
||||||
|
|||||||
@@ -21,6 +21,7 @@ package org.apache.hudi.client.functional;
|
|||||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
@@ -241,6 +242,8 @@ public class TestHoodieMetadataBootstrap extends TestHoodieMetadataBase {
|
|||||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.forTable("test-trip-table").build();
|
.forTable("test-trip-table").build();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -36,6 +36,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler;
|
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
@@ -94,6 +95,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
|||||||
public void init(HoodieTableType tableType) throws Exception {
|
public void init(HoodieTableType tableType) throws Exception {
|
||||||
initPath();
|
initPath();
|
||||||
initSparkContexts();
|
initSparkContexts();
|
||||||
|
initTimelineService();
|
||||||
initMetaClient();
|
initMetaClient();
|
||||||
hadoopConf = context.getHadoopConf().get();
|
hadoopConf = context.getHadoopConf().get();
|
||||||
metaClient.getFs().mkdirs(new Path(basePath));
|
metaClient.getFs().mkdirs(new Path(basePath));
|
||||||
@@ -126,6 +128,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
|||||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath)
|
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata)
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata)
|
||||||
.withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsMetadataTable).build())
|
.withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsMetadataTable).build())
|
||||||
.forTable("test-trip-table").build();
|
.forTable("test-trip-table").build();
|
||||||
@@ -210,6 +214,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
|||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
@@ -328,6 +334,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
|||||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||||
.withParallelism(2, 2).forTable("test-trip-table")
|
.withParallelism(2, 2).forTable("test-trip-table")
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.build();
|
.build();
|
||||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||||
@@ -484,6 +492,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
|||||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||||
.withParallelism(2, 2).forTable("test-trip-table")
|
.withParallelism(2, 2).forTable("test-trip-table")
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.build();
|
.build();
|
||||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||||
@@ -519,6 +529,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
|||||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||||
.withParallelism(2, 2).forTable("test-trip-table")
|
.withParallelism(2, 2).forTable("test-trip-table")
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.build();
|
.build();
|
||||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||||
|
|||||||
@@ -18,8 +18,6 @@
|
|||||||
|
|
||||||
package org.apache.hudi.io.storage.row;
|
package org.apache.hudi.io.storage.row;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hudi.common.bloom.BloomFilter;
|
import org.apache.hudi.common.bloom.BloomFilter;
|
||||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
@@ -27,6 +25,9 @@ import org.apache.hudi.config.HoodieStorageConfig;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||||
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
@@ -66,7 +67,8 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness
|
|||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@ValueSource(booleans = {true, false})
|
||||||
public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws Exception {
|
public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws Exception {
|
||||||
HoodieWriteConfig.Builder writeConfigBuilder = SparkDatasetTestUtils.getConfigBuilder(basePath);
|
HoodieWriteConfig.Builder writeConfigBuilder =
|
||||||
|
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort);
|
||||||
for (int i = 0; i < 5; i++) {
|
for (int i = 0; i < 5; i++) {
|
||||||
// init write support and parquet config
|
// init write support and parquet config
|
||||||
HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
|
HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
|
||||||
|
|||||||
@@ -28,8 +28,8 @@ import org.apache.hudi.exception.HoodieInsertException;
|
|||||||
import org.apache.hudi.table.HoodieSparkTable;
|
import org.apache.hudi.table.HoodieSparkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||||
|
|
||||||
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
||||||
|
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
import org.apache.spark.sql.catalyst.InternalRow;
|
import org.apache.spark.sql.catalyst.InternalRow;
|
||||||
@@ -65,6 +65,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
|||||||
initFileSystem();
|
initFileSystem();
|
||||||
initTestDataGenerator();
|
initTestDataGenerator();
|
||||||
initMetaClient();
|
initMetaClient();
|
||||||
|
initTimelineService();
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterEach
|
@AfterEach
|
||||||
@@ -75,7 +76,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
|||||||
@Test
|
@Test
|
||||||
public void testRowCreateHandle() throws Exception {
|
public void testRowCreateHandle() throws Exception {
|
||||||
// init config and table
|
// init config and table
|
||||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
|
HoodieWriteConfig cfg =
|
||||||
|
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build();
|
||||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||||
List<String> fileNames = new ArrayList<>();
|
List<String> fileNames = new ArrayList<>();
|
||||||
List<String> fileAbsPaths = new ArrayList<>();
|
List<String> fileAbsPaths = new ArrayList<>();
|
||||||
@@ -116,7 +118,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
|||||||
@Test
|
@Test
|
||||||
public void testGlobalFailure() throws Exception {
|
public void testGlobalFailure() throws Exception {
|
||||||
// init config and table
|
// init config and table
|
||||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
|
HoodieWriteConfig cfg =
|
||||||
|
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build();
|
||||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||||
|
|
||||||
@@ -124,7 +127,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
|||||||
String fileId = UUID.randomUUID().toString();
|
String fileId = UUID.randomUUID().toString();
|
||||||
String instantTime = "000";
|
String instantTime = "000";
|
||||||
|
|
||||||
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
HoodieRowCreateHandle handle =
|
||||||
|
new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||||
int size = 10 + RANDOM.nextInt(1000);
|
int size = 10 + RANDOM.nextInt(1000);
|
||||||
int totalFailures = 5;
|
int totalFailures = 5;
|
||||||
// Generate first batch of valid rows
|
// Generate first batch of valid rows
|
||||||
@@ -169,7 +173,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
|||||||
@Test
|
@Test
|
||||||
public void testInstantiationFailure() throws IOException {
|
public void testInstantiationFailure() throws IOException {
|
||||||
// init config and table
|
// init config and table
|
||||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort)
|
||||||
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||||
.withPath("/dummypath/abc/").build();
|
.withPath("/dummypath/abc/").build();
|
||||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||||
|
|
||||||
|
|||||||
@@ -18,8 +18,6 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table;
|
package org.apache.hudi.table;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||||
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
||||||
@@ -62,6 +60,7 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
|||||||
import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator;
|
import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanMigrator;
|
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanMigrator;
|
||||||
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV1MigrationHandler;
|
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV1MigrationHandler;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.table.view.TableFileSystemView;
|
import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||||
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
@@ -81,6 +80,9 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
|||||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||||
import org.apache.hudi.table.action.clean.CleanPlanner;
|
import org.apache.hudi.table.action.clean.CleanPlanner;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
@@ -89,7 +91,6 @@ import org.junit.jupiter.params.ParameterizedTest;
|
|||||||
import org.junit.jupiter.params.provider.Arguments;
|
import org.junit.jupiter.params.provider.Arguments;
|
||||||
import org.junit.jupiter.params.provider.MethodSource;
|
import org.junit.jupiter.params.provider.MethodSource;
|
||||||
import org.junit.jupiter.params.provider.ValueSource;
|
import org.junit.jupiter.params.provider.ValueSource;
|
||||||
import scala.Tuple3;
|
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@@ -111,6 +112,8 @@ import java.util.function.Predicate;
|
|||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import scala.Tuple3;
|
||||||
|
|
||||||
import static org.apache.hudi.common.testutils.HoodieTestTable.makeIncrementalCommitTimes;
|
import static org.apache.hudi.common.testutils.HoodieTestTable.makeIncrementalCommitTimes;
|
||||||
import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime;
|
import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime;
|
||||||
import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
||||||
@@ -1300,7 +1303,10 @@ public class TestCleaner extends HoodieClientTestBase {
|
|||||||
final int numTempFilesBefore = testTable.listAllFilesInTempFolder().length;
|
final int numTempFilesBefore = testTable.listAllFilesInTempFolder().length;
|
||||||
assertEquals(10, numTempFilesBefore, "Some marker files are created.");
|
assertEquals(10, numTempFilesBefore, "Some marker files are created.");
|
||||||
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
|
.withPath(basePath).build();
|
||||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||||
table.getActiveTimeline().transitionRequestedToInflight(
|
table.getActiveTimeline().transitionRequestedToInflight(
|
||||||
|
|||||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
|||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||||
import org.apache.hudi.common.testutils.Transformations;
|
import org.apache.hudi.common.testutils.Transformations;
|
||||||
@@ -112,7 +113,9 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
|||||||
|
|
||||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
||||||
// Prepare the AvroParquetIO
|
// Prepare the AvroParquetIO
|
||||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());
|
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build());
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO (weiy): Add testcases for crossing file writing.
|
// TODO (weiy): Add testcases for crossing file writing.
|
||||||
@@ -405,8 +408,10 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
|||||||
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
|
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
|
||||||
Schema schema = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/testDataGeneratorSchema.txt");
|
Schema schema = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/testDataGeneratorSchema.txt");
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString())
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString())
|
||||||
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
.parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
|
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||||
|
.parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
|
||||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||||
String instantTime = "000";
|
String instantTime = "000";
|
||||||
|
|||||||
@@ -30,8 +30,10 @@ import org.apache.hudi.common.model.HoodieRecord;
|
|||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
|
import org.apache.hudi.common.table.marker.MarkerType;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||||
import org.apache.hudi.common.table.view.TableFileSystemView;
|
import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||||
@@ -297,6 +299,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
|||||||
void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
|
void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
|
||||||
boolean populateMetaFields = true;
|
boolean populateMetaFields = true;
|
||||||
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false)
|
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false)
|
||||||
|
// Timeline-server-based markers are not used for multi-rollback tests
|
||||||
|
.withMarkersType(MarkerType.DIRECT.name())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build());
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build());
|
||||||
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
|
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
|
||||||
HoodieWriteConfig cfg = cfgBuilder.build();
|
HoodieWriteConfig cfg = cfgBuilder.build();
|
||||||
@@ -347,6 +351,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
|||||||
newCommitTime = "002";
|
newCommitTime = "002";
|
||||||
// WriteClient with custom config (disable small file handling)
|
// WriteClient with custom config (disable small file handling)
|
||||||
HoodieWriteConfig smallFileWriteConfig = getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields)
|
HoodieWriteConfig smallFileWriteConfig = getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields)
|
||||||
|
// Timeline-server-based markers are not used for multi-rollback tests
|
||||||
|
.withMarkersType(MarkerType.DIRECT.name())
|
||||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build();
|
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build();
|
||||||
try (SparkRDDWriteClient nClient = getHoodieWriteClient(smallFileWriteConfig)) {
|
try (SparkRDDWriteClient nClient = getHoodieWriteClient(smallFileWriteConfig)) {
|
||||||
nClient.startCommitWithTime(newCommitTime);
|
nClient.startCommitWithTime(newCommitTime);
|
||||||
@@ -468,6 +474,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
|||||||
.withAutoCommit(false)
|
.withAutoCommit(false)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
|
||||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.withEmbeddedTimelineServerEnabled(true)
|
.withEmbeddedTimelineServerEnabled(true)
|
||||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table");
|
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table");
|
||||||
|
|
||||||
@@ -562,7 +570,7 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (rollbackUsingMarkers) {
|
if (rollbackUsingMarkers) {
|
||||||
metaClient.getFs().copyFromLocalFile(markerDir,
|
metaClient.getFs().copyFromLocalFile(new Path(markerDir, lastCommitTime),
|
||||||
new Path(metaClient.getMarkerFolderPath(lastCommitTime)));
|
new Path(metaClient.getMarkerFolderPath(lastCommitTime)));
|
||||||
}
|
}
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
|
|||||||
@@ -59,15 +59,18 @@ public class TestDirectWriteMarkers extends TestWriteMarkersBase {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void verifyMarkersInFileSystem() throws IOException {
|
void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException {
|
||||||
List<FileStatus> markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
|
List<FileStatus> markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
|
||||||
.stream().filter(status -> status.getPath().getName().contains(".marker"))
|
.stream().filter(status -> status.getPath().getName().contains(".marker"))
|
||||||
.sorted().collect(Collectors.toList());
|
.sorted().collect(Collectors.toList());
|
||||||
assertEquals(3, markerFiles.size());
|
assertEquals(3, markerFiles.size());
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||||
"file:" + markerFolderPath.toString() + "/2020/06/01/file1.marker.MERGE",
|
"file:" + markerFolderPath.toString()
|
||||||
"file:" + markerFolderPath.toString() + "/2020/06/02/file2.marker.APPEND",
|
+ (isTablePartitioned ? "/2020/06/01" : "") + "/file1.marker.MERGE",
|
||||||
"file:" + markerFolderPath.toString() + "/2020/06/03/file3.marker.CREATE"),
|
"file:" + markerFolderPath.toString()
|
||||||
|
+ (isTablePartitioned ? "/2020/06/02" : "") + "/file2.marker.APPEND",
|
||||||
|
"file:" + markerFolderPath.toString()
|
||||||
|
+ (isTablePartitioned ? "/2020/06/03" : "") + "/file3.marker.CREATE"),
|
||||||
markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList())
|
markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -93,18 +93,20 @@ public class TestTimelineServerBasedWriteMarkers extends TestWriteMarkersBase {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void verifyMarkersInFileSystem() throws IOException {
|
void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException {
|
||||||
// Verifies the markers
|
// Verifies the markers
|
||||||
List<String> allMarkers = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(
|
List<String> allMarkers = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(
|
||||||
markerFolderPath.toString(), fs, context, 1)
|
markerFolderPath.toString(), fs, context, 1)
|
||||||
.values().stream().flatMap(Collection::stream).sorted()
|
.values().stream().flatMap(Collection::stream).sorted()
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
assertEquals(3, allMarkers.size());
|
assertEquals(3, allMarkers.size());
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
List<String> expectedMarkers = isTablePartitioned
|
||||||
"2020/06/01/file1.marker.MERGE",
|
? CollectionUtils.createImmutableList(
|
||||||
"2020/06/02/file2.marker.APPEND",
|
"2020/06/01/file1.marker.MERGE", "2020/06/02/file2.marker.APPEND",
|
||||||
"2020/06/03/file3.marker.CREATE"),
|
"2020/06/03/file3.marker.CREATE")
|
||||||
allMarkers);
|
: CollectionUtils.createImmutableList(
|
||||||
|
"file1.marker.MERGE", "file2.marker.APPEND", "file3.marker.CREATE");
|
||||||
|
assertIterableEquals(expectedMarkers, allMarkers);
|
||||||
// Verifies the marker type file
|
// Verifies the marker type file
|
||||||
Path markerTypeFilePath = new Path(markerFolderPath, MarkerUtils.MARKER_TYPE_FILENAME);
|
Path markerTypeFilePath = new Path(markerFolderPath, MarkerUtils.MARKER_TYPE_FILENAME);
|
||||||
assertTrue(MarkerUtils.doesMarkerTypeFileExist(fs, markerFolderPath.toString()));
|
assertTrue(MarkerUtils.doesMarkerTypeFileExist(fs, markerFolderPath.toString()));
|
||||||
|
|||||||
@@ -31,8 +31,11 @@ import org.apache.hadoop.fs.FileSystem;
|
|||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
|
import org.junit.jupiter.params.provider.ValueSource;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
@@ -48,10 +51,10 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
|
|||||||
protected JavaSparkContext jsc;
|
protected JavaSparkContext jsc;
|
||||||
protected HoodieSparkEngineContext context;
|
protected HoodieSparkEngineContext context;
|
||||||
|
|
||||||
private void createSomeMarkers() {
|
private void createSomeMarkers(boolean isTablePartitioned) {
|
||||||
writeMarkers.create("2020/06/01", "file1", IOType.MERGE);
|
writeMarkers.create(isTablePartitioned ? "2020/06/01" : "", "file1", IOType.MERGE);
|
||||||
writeMarkers.create("2020/06/02", "file2", IOType.APPEND);
|
writeMarkers.create(isTablePartitioned ? "2020/06/02" : "", "file2", IOType.APPEND);
|
||||||
writeMarkers.create("2020/06/03", "file3", IOType.CREATE);
|
writeMarkers.create(isTablePartitioned ? "2020/06/03" : "", "file3", IOType.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createInvalidFile(String partitionPath, String invalidFileName) {
|
private void createInvalidFile(String partitionPath, String invalidFileName) {
|
||||||
@@ -64,22 +67,24 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract void verifyMarkersInFileSystem() throws IOException;
|
abstract void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException;
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void testCreation() throws Exception {
|
@ValueSource(booleans = {true, false})
|
||||||
|
public void testCreation(boolean isTablePartitioned) throws Exception {
|
||||||
// when
|
// when
|
||||||
createSomeMarkers();
|
createSomeMarkers(isTablePartitioned);
|
||||||
|
|
||||||
// then
|
// then
|
||||||
assertTrue(fs.exists(markerFolderPath));
|
assertTrue(fs.exists(markerFolderPath));
|
||||||
verifyMarkersInFileSystem();
|
verifyMarkersInFileSystem(isTablePartitioned);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void testDeletionWhenMarkerDirExists() throws IOException {
|
@ValueSource(booleans = {true, false})
|
||||||
|
public void testDeletionWhenMarkerDirExists(boolean isTablePartitioned) throws IOException {
|
||||||
//when
|
//when
|
||||||
writeMarkers.create("2020/06/01", "file1", IOType.MERGE);
|
writeMarkers.create(isTablePartitioned ? "2020/06/01" : "", "file1", IOType.MERGE);
|
||||||
|
|
||||||
// then
|
// then
|
||||||
assertTrue(writeMarkers.doesMarkerDirExist());
|
assertTrue(writeMarkers.doesMarkerDirExist());
|
||||||
@@ -95,32 +100,40 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
|
|||||||
assertFalse(writeMarkers.deleteMarkerDir(context, 2));
|
assertFalse(writeMarkers.deleteMarkerDir(context, 2));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void testDataPathsWhenCreatingOrMerging() throws IOException {
|
@ValueSource(booleans = {true, false})
|
||||||
|
public void testDataPathsWhenCreatingOrMerging(boolean isTablePartitioned) throws IOException {
|
||||||
// add markfiles
|
// add markfiles
|
||||||
createSomeMarkers();
|
createSomeMarkers(isTablePartitioned);
|
||||||
// add invalid file
|
// add invalid file
|
||||||
createInvalidFile("2020/06/01", "invalid_file3");
|
createInvalidFile(isTablePartitioned ? "2020/06/01" : "", "invalid_file3");
|
||||||
long fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).stream()
|
long fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).stream()
|
||||||
.filter(fileStatus -> !fileStatus.getPath().getName().contains(MarkerUtils.MARKER_TYPE_FILENAME))
|
.filter(fileStatus -> !fileStatus.getPath().getName().contains(MarkerUtils.MARKER_TYPE_FILENAME))
|
||||||
.count();
|
.count();
|
||||||
assertEquals(fileSize, 4);
|
assertEquals(fileSize, 4);
|
||||||
|
|
||||||
|
List<String> expectedPaths = isTablePartitioned
|
||||||
|
? CollectionUtils.createImmutableList("2020/06/01/file1", "2020/06/03/file3")
|
||||||
|
: CollectionUtils.createImmutableList("file1", "file3");
|
||||||
// then
|
// then
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
assertIterableEquals(expectedPaths,
|
||||||
"2020/06/01/file1", "2020/06/03/file3"),
|
|
||||||
writeMarkers.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
|
writeMarkers.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
public void testAllMarkerPaths() throws IOException {
|
@ValueSource(booleans = {true, false})
|
||||||
|
public void testAllMarkerPaths(boolean isTablePartitioned) throws IOException {
|
||||||
// given
|
// given
|
||||||
createSomeMarkers();
|
createSomeMarkers(isTablePartitioned);
|
||||||
|
|
||||||
|
List<String> expectedPaths = isTablePartitioned
|
||||||
|
? CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
|
||||||
|
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE")
|
||||||
|
: CollectionUtils.createImmutableList(
|
||||||
|
"file1.marker.MERGE", "file2.marker.APPEND", "file3.marker.CREATE");
|
||||||
// then
|
// then
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
|
assertIterableEquals(expectedPaths,
|
||||||
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE"),
|
|
||||||
writeMarkers.allMarkerFilePaths().stream()
|
writeMarkers.allMarkerFilePaths().stream()
|
||||||
.filter(path -> !path.contains(MarkerUtils.MARKER_TYPE_FILENAME))
|
.filter(path -> !path.contains(MarkerUtils.MARKER_TYPE_FILENAME))
|
||||||
.sorted().collect(Collectors.toList())
|
.sorted().collect(Collectors.toList())
|
||||||
|
|||||||
@@ -326,9 +326,11 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
|
|||||||
new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance())
|
new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance())
|
||||||
.run(toVersion, null);
|
.run(toVersion, null);
|
||||||
|
|
||||||
// assert marker files
|
if (fromVersion == HoodieTableVersion.TWO) {
|
||||||
assertMarkerFilesForDowngrade(table, commitInstant, toVersion == HoodieTableVersion.ONE);
|
// assert marker files
|
||||||
|
assertMarkerFilesForDowngrade(table, commitInstant, toVersion == HoodieTableVersion.ONE);
|
||||||
|
}
|
||||||
|
|
||||||
// verify hoodie.table.version got downgraded
|
// verify hoodie.table.version got downgraded
|
||||||
metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(cfg.getBasePath())
|
metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(cfg.getBasePath())
|
||||||
.setLayoutVersion(Option.of(new TimelineLayoutVersion(cfg.getTimelineLayoutVersion()))).build();
|
.setLayoutVersion(Option.of(new TimelineLayoutVersion(cfg.getTimelineLayoutVersion()))).build();
|
||||||
|
|||||||
@@ -152,6 +152,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
|||||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
|
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
|
||||||
|
.withRemoteServerPort(timelineServicePort)
|
||||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -32,6 +32,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
|||||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||||
import org.apache.hudi.common.table.view.TableFileSystemView;
|
import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||||
@@ -56,6 +57,7 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
|||||||
import org.apache.hudi.table.HoodieSparkTable;
|
import org.apache.hudi.table.HoodieSparkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.WorkloadStat;
|
import org.apache.hudi.table.WorkloadStat;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
@@ -102,6 +104,8 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
|||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(HoodieClientTestHarness.class);
|
private static final Logger LOG = LogManager.getLogger(HoodieClientTestHarness.class);
|
||||||
|
|
||||||
|
protected static int timelineServicePort =
|
||||||
|
FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue();
|
||||||
private String testMethodName;
|
private String testMethodName;
|
||||||
protected transient JavaSparkContext jsc = null;
|
protected transient JavaSparkContext jsc = null;
|
||||||
protected transient HoodieSparkEngineContext context = null;
|
protected transient HoodieSparkEngineContext context = null;
|
||||||
@@ -113,6 +117,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
|||||||
protected transient SparkRDDWriteClient writeClient;
|
protected transient SparkRDDWriteClient writeClient;
|
||||||
protected transient HoodieReadClient readClient;
|
protected transient HoodieReadClient readClient;
|
||||||
protected transient HoodieTableFileSystemView tableView;
|
protected transient HoodieTableFileSystemView tableView;
|
||||||
|
protected transient TimelineService timelineService;
|
||||||
|
|
||||||
protected final SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
protected final SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||||
|
|
||||||
@@ -145,12 +150,14 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
|||||||
initTestDataGenerator();
|
initTestDataGenerator();
|
||||||
initFileSystem();
|
initFileSystem();
|
||||||
initMetaClient();
|
initMetaClient();
|
||||||
|
initTimelineService();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cleanups resource group for the subclasses of {@link HoodieClientTestBase}.
|
* Cleanups resource group for the subclasses of {@link HoodieClientTestBase}.
|
||||||
*/
|
*/
|
||||||
public void cleanupResources() throws IOException {
|
public void cleanupResources() throws IOException {
|
||||||
|
cleanupTimelineService();
|
||||||
cleanupClients();
|
cleanupClients();
|
||||||
cleanupSparkContexts();
|
cleanupSparkContexts();
|
||||||
cleanupTestDataGenerator();
|
cleanupTestDataGenerator();
|
||||||
@@ -245,6 +252,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
|||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
protected void initMetaClient() throws IOException {
|
protected void initMetaClient() throws IOException {
|
||||||
initMetaClient(getTableType());
|
initMetaClient(getTableType());
|
||||||
}
|
}
|
||||||
@@ -272,6 +280,28 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
|||||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, properties);
|
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, properties);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes timeline service based on the write config.
|
||||||
|
*/
|
||||||
|
protected void initTimelineService() {
|
||||||
|
timelineService = HoodieClientTestUtils.initTimelineService(
|
||||||
|
context, basePath, incrementTimelineServicePortToUse());
|
||||||
|
timelineServicePort = timelineService.getServerPort();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void cleanupTimelineService() {
|
||||||
|
if (timelineService != null) {
|
||||||
|
timelineService.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int incrementTimelineServicePortToUse() {
|
||||||
|
// Increment the timeline service port for each individual test
|
||||||
|
// to avoid port reuse causing failures
|
||||||
|
timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024;
|
||||||
|
return timelineServicePort;
|
||||||
|
}
|
||||||
|
|
||||||
protected Properties getPropertiesForKeyGen() {
|
protected Properties getPropertiesForKeyGen() {
|
||||||
Properties properties = new Properties();
|
Properties properties = new Properties();
|
||||||
properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
|
properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
|
||||||
|
|||||||
@@ -20,6 +20,7 @@ package org.apache.hudi.testutils;
|
|||||||
|
|
||||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||||
import org.apache.hudi.client.HoodieReadClient;
|
import org.apache.hudi.client.HoodieReadClient;
|
||||||
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||||
@@ -30,13 +31,18 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
@@ -257,6 +263,36 @@ public class HoodieClientTestUtils {
|
|||||||
return valuesAsList.stream();
|
return valuesAsList.stream();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes timeline service based on the write config.
|
||||||
|
*
|
||||||
|
* @param context {@link HoodieEngineContext} instance to use.
|
||||||
|
* @param basePath Base path of the table.
|
||||||
|
* @param timelineServicePort Port number to use for timeline service.
|
||||||
|
* @return started {@link TimelineService} instance.
|
||||||
|
*/
|
||||||
|
public static TimelineService initTimelineService(
|
||||||
|
HoodieEngineContext context, String basePath, int timelineServicePort) {
|
||||||
|
try {
|
||||||
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||||
|
.withPath(basePath)
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
|
.build();
|
||||||
|
TimelineService timelineService = new TimelineService(context, new Configuration(),
|
||||||
|
TimelineService.Config.builder().enableMarkerRequests(true)
|
||||||
|
.serverPort(config.getViewStorageConfig().getRemoteViewServerPort()).build(),
|
||||||
|
FileSystem.get(new Configuration()),
|
||||||
|
FileSystemViewManager.createViewManager(context, config.getMetadataConfig(),
|
||||||
|
config.getViewStorageConfig(), config.getCommonConfig()));
|
||||||
|
timelineService.startService();
|
||||||
|
LOG.info("Timeline service server port: " + timelineServicePort);
|
||||||
|
return timelineService;
|
||||||
|
} catch (Exception ex) {
|
||||||
|
throw new RuntimeException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static Option<HoodieCommitMetadata> getCommitMetadataForLatestInstant(HoodieTableMetaClient metaClient) {
|
public static Option<HoodieCommitMetadata> getCommitMetadataForLatestInstant(HoodieTableMetaClient metaClient) {
|
||||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||||
if (timeline.lastInstant().isPresent()) {
|
if (timeline.lastInstant().isPresent()) {
|
||||||
|
|||||||
@@ -49,6 +49,7 @@ import org.apache.hudi.table.HoodieTable;
|
|||||||
import org.apache.hudi.testutils.providers.HoodieMetaClientProvider;
|
import org.apache.hudi.testutils.providers.HoodieMetaClientProvider;
|
||||||
import org.apache.hudi.testutils.providers.HoodieWriteClientProvider;
|
import org.apache.hudi.testutils.providers.HoodieWriteClientProvider;
|
||||||
import org.apache.hudi.testutils.providers.SparkProvider;
|
import org.apache.hudi.testutils.providers.SparkProvider;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
@@ -81,10 +82,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||||||
|
|
||||||
public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMetaClientProvider, HoodieWriteClientProvider {
|
public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMetaClientProvider, HoodieWriteClientProvider {
|
||||||
|
|
||||||
|
protected static int timelineServicePort =
|
||||||
|
FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue();
|
||||||
private static transient SparkSession spark;
|
private static transient SparkSession spark;
|
||||||
private static transient SQLContext sqlContext;
|
private static transient SQLContext sqlContext;
|
||||||
private static transient JavaSparkContext jsc;
|
private static transient JavaSparkContext jsc;
|
||||||
private static transient HoodieSparkEngineContext context;
|
private static transient HoodieSparkEngineContext context;
|
||||||
|
private static transient TimelineService timelineService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An indicator of the initialization status.
|
* An indicator of the initialization status.
|
||||||
@@ -174,6 +178,9 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
|
|||||||
sqlContext = spark.sqlContext();
|
sqlContext = spark.sqlContext();
|
||||||
jsc = new JavaSparkContext(spark.sparkContext());
|
jsc = new JavaSparkContext(spark.sparkContext());
|
||||||
context = new HoodieSparkEngineContext(jsc);
|
context = new HoodieSparkEngineContext(jsc);
|
||||||
|
timelineService = HoodieClientTestUtils.initTimelineService(
|
||||||
|
context, basePath(), incrementTimelineServicePortToUse());
|
||||||
|
timelineServicePort = timelineService.getServerPort();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -189,6 +196,9 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
|
|||||||
spark.close();
|
spark.close();
|
||||||
spark = null;
|
spark = null;
|
||||||
}
|
}
|
||||||
|
if (timelineService != null) {
|
||||||
|
timelineService.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected JavaRDD<HoodieRecord> tagLocation(
|
protected JavaRDD<HoodieRecord> tagLocation(
|
||||||
@@ -312,9 +322,17 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
|
|||||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())
|
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())
|
||||||
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
|
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
|
||||||
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
|
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
|
||||||
|
.withRemoteServerPort(timelineServicePort)
|
||||||
.withEnableBackupForRemoteFileSystemView(false).build())
|
.withEnableBackupForRemoteFileSystemView(false).build())
|
||||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||||
.withClusteringConfig(clusteringConfig)
|
.withClusteringConfig(clusteringConfig)
|
||||||
.withRollbackUsingMarkers(rollbackUsingMarkers);
|
.withRollbackUsingMarkers(rollbackUsingMarkers);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected int incrementTimelineServicePortToUse() {
|
||||||
|
// Increment the timeline service port for each individual test
|
||||||
|
// to avoid port reuse causing failures
|
||||||
|
timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024;
|
||||||
|
return timelineServicePort;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,6 +19,7 @@
|
|||||||
package org.apache.hudi.testutils;
|
package org.apache.hudi.testutils;
|
||||||
|
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||||
import org.apache.hudi.config.HoodieIndexConfig;
|
import org.apache.hudi.config.HoodieIndexConfig;
|
||||||
@@ -176,12 +177,14 @@ public class SparkDatasetTestUtils {
|
|||||||
return new GenericInternalRow(values);
|
return new GenericInternalRow(values);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) {
|
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath, int timelineServicePort) {
|
||||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||||
.withParallelism(2, 2)
|
.withParallelism(2, 2)
|
||||||
.withDeleteParallelism(2)
|
.withDeleteParallelism(2)
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||||
|
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||||
|
.withRemoteServerPort(timelineServicePort).build())
|
||||||
.forTable("test-trip-table")
|
.forTable("test-trip-table")
|
||||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||||
.withBulkInsertParallelism(2);
|
.withBulkInsertParallelism(2);
|
||||||
|
|||||||
@@ -77,7 +77,7 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
|
|||||||
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
|
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
|
||||||
properties.setProperty(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
|
properties.setProperty(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
|
||||||
}
|
}
|
||||||
return getConfigBuilder(basePath).withProperties(properties).build();
|
return getConfigBuilder(basePath, timelineServicePort).withProperties(properties).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size,
|
protected void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size,
|
||||||
|
|||||||
@@ -16,18 +16,16 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hudi
|
package org.apache.hudi
|
||||||
|
|
||||||
import java.lang
|
|
||||||
import java.util.function.Function
|
|
||||||
|
|
||||||
import org.apache.hudi.async.{AsyncClusteringService, AsyncCompactService, SparkStreamingAsyncClusteringService, SparkStreamingAsyncCompactService}
|
import org.apache.hudi.async.{AsyncClusteringService, AsyncCompactService, SparkStreamingAsyncClusteringService, SparkStreamingAsyncCompactService}
|
||||||
import org.apache.hudi.client.SparkRDDWriteClient
|
import org.apache.hudi.client.SparkRDDWriteClient
|
||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload
|
import org.apache.hudi.common.model.HoodieRecordPayload
|
||||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
import org.apache.hudi.common.table.marker.MarkerType
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||||
import org.apache.hudi.common.util.CompactionUtils
|
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||||
import org.apache.hudi.common.util.ClusteringUtils
|
import org.apache.hudi.common.util.{ClusteringUtils, CompactionUtils}
|
||||||
|
import org.apache.hudi.config.HoodieWriteConfig
|
||||||
import org.apache.hudi.exception.HoodieCorruptedDataException
|
import org.apache.hudi.exception.HoodieCorruptedDataException
|
||||||
import org.apache.log4j.LogManager
|
import org.apache.log4j.LogManager
|
||||||
import org.apache.spark.api.java.JavaSparkContext
|
import org.apache.spark.api.java.JavaSparkContext
|
||||||
@@ -35,8 +33,10 @@ import org.apache.spark.sql.execution.streaming.Sink
|
|||||||
import org.apache.spark.sql.streaming.OutputMode
|
import org.apache.spark.sql.streaming.OutputMode
|
||||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
||||||
|
|
||||||
import scala.util.{Failure, Success, Try}
|
import java.lang
|
||||||
|
import java.util.function.Function
|
||||||
import scala.collection.JavaConversions._
|
import scala.collection.JavaConversions._
|
||||||
|
import scala.util.{Failure, Success, Try}
|
||||||
|
|
||||||
class HoodieStreamingSink(sqlContext: SQLContext,
|
class HoodieStreamingSink(sqlContext: SQLContext,
|
||||||
options: Map[String, String],
|
options: Map[String, String],
|
||||||
@@ -71,25 +71,29 @@ class HoodieStreamingSink(sqlContext: SQLContext,
|
|||||||
private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty
|
private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty
|
||||||
|
|
||||||
override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized {
|
override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized {
|
||||||
if (isAsyncCompactorServiceShutdownAbnormally) {
|
if (isAsyncCompactorServiceShutdownAbnormally) {
|
||||||
throw new IllegalStateException("Async Compactor shutdown unexpectedly")
|
throw new IllegalStateException("Async Compactor shutdown unexpectedly")
|
||||||
}
|
}
|
||||||
if (isAsyncClusteringServiceShutdownAbnormally) {
|
if (isAsyncClusteringServiceShutdownAbnormally) {
|
||||||
log.error("Async clustering service shutdown unexpectedly")
|
log.error("Async clustering service shutdown unexpectedly")
|
||||||
throw new IllegalStateException("Async clustering service shutdown unexpectedly")
|
throw new IllegalStateException("Async clustering service shutdown unexpectedly")
|
||||||
}
|
}
|
||||||
|
// Override to use direct markers. In Structured streaming, timeline server is closed after
|
||||||
|
// first micro-batch and subsequent micro-batches do not have timeline server running.
|
||||||
|
// Thus, we can't use timeline-server-based markers.
|
||||||
|
val updatedOptions = options.updated(HoodieWriteConfig.MARKERS_TYPE.key(), MarkerType.DIRECT.name())
|
||||||
|
|
||||||
retry(retryCnt, retryIntervalMs)(
|
retry(retryCnt, retryIntervalMs)(
|
||||||
Try(
|
Try(
|
||||||
HoodieSparkSqlWriter.write(
|
HoodieSparkSqlWriter.write(
|
||||||
sqlContext, mode, options, data, hoodieTableConfig, writeClient, Some(triggerAsyncCompactor), Some(triggerAsyncClustering))
|
sqlContext, mode, updatedOptions, data, hoodieTableConfig, writeClient, Some(triggerAsyncCompactor), Some(triggerAsyncClustering))
|
||||||
) match {
|
) match {
|
||||||
case Success((true, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) =>
|
case Success((true, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) =>
|
||||||
log.info(s"Micro batch id=$batchId succeeded"
|
log.info(s"Micro batch id=$batchId succeeded"
|
||||||
+ (commitOps.isPresent match {
|
+ (commitOps.isPresent match {
|
||||||
case true => s" for commit=${commitOps.get()}"
|
case true => s" for commit=${commitOps.get()}"
|
||||||
case _ => s" with no new commits"
|
case _ => s" with no new commits"
|
||||||
}))
|
}))
|
||||||
writeClient = Some(client)
|
writeClient = Some(client)
|
||||||
hoodieTableConfig = Some(tableConfig)
|
hoodieTableConfig = Some(tableConfig)
|
||||||
if (compactionInstantOps.isPresent) {
|
if (compactionInstantOps.isPresent) {
|
||||||
|
|||||||
@@ -59,9 +59,11 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
|||||||
spark = sqlContext.sparkSession
|
spark = sqlContext.sparkSession
|
||||||
initTestDataGenerator()
|
initTestDataGenerator()
|
||||||
initFileSystem()
|
initFileSystem()
|
||||||
|
initTimelineService()
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterEach override def tearDown() = {
|
@AfterEach override def tearDown() = {
|
||||||
|
cleanupTimelineService()
|
||||||
cleanupSparkContexts()
|
cleanupSparkContexts()
|
||||||
cleanupTestDataGenerator()
|
cleanupTestDataGenerator()
|
||||||
cleanupFileSystem()
|
cleanupFileSystem()
|
||||||
|
|||||||
@@ -321,8 +321,10 @@ public class TimelineService {
|
|||||||
if (requestHandler != null) {
|
if (requestHandler != null) {
|
||||||
this.requestHandler.stop();
|
this.requestHandler.stop();
|
||||||
}
|
}
|
||||||
this.app.stop();
|
if (this.app != null) {
|
||||||
this.app = null;
|
this.app.stop();
|
||||||
|
this.app = null;
|
||||||
|
}
|
||||||
this.fsViewsManager.close();
|
this.fsViewsManager.close();
|
||||||
LOG.info("Closed Timeline Service");
|
LOG.info("Closed Timeline Service");
|
||||||
}
|
}
|
||||||
|
|||||||
Reference in New Issue
Block a user