[HUDI-1138] Add timeline-server-based marker file strategy for improving marker-related latency (#3233)
- Can be enabled for cloud stores like S3. Not supported for hdfs yet, due to partial write failures.
This commit is contained in:
@@ -64,9 +64,9 @@ import org.apache.hudi.metrics.HoodieMetrics;
|
|||||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
import org.apache.hudi.table.action.savepoint.SavepointHelpers;
|
import org.apache.hudi.table.action.savepoint.SavepointHelpers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import com.codahale.metrics.Timer;
|
import com.codahale.metrics.Timer;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
@@ -430,7 +430,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
|||||||
protected void postCommit(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
|
protected void postCommit(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||||
try {
|
try {
|
||||||
// Delete the marker directory for the instant.
|
// Delete the marker directory for the instant.
|
||||||
new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
WriteMarkersFactory.get(config.getMarkersType(), table, instantTime)
|
||||||
|
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
|
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
|
||||||
archiveLog.archiveIfRequired(context);
|
archiveLog.archiveIfRequired(context);
|
||||||
|
|||||||
@@ -67,11 +67,7 @@ public class EmbeddedTimelineServerHelper {
|
|||||||
LOG.info("Starting Timeline service !!");
|
LOG.info("Starting Timeline service !!");
|
||||||
Option<String> hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST);
|
Option<String> hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST);
|
||||||
EmbeddedTimelineService timelineService = new EmbeddedTimelineService(
|
EmbeddedTimelineService timelineService = new EmbeddedTimelineService(
|
||||||
context, hostAddr.orElse(null),config.getEmbeddedTimelineServerPort(),
|
context, hostAddr.orElse(null), config);
|
||||||
config.getMetadataConfig(), config.getCommonConfig(),
|
|
||||||
config.getClientSpecifiedViewStorageConfig(), config.getBasePath(),
|
|
||||||
config.getEmbeddedTimelineServerThreads(), config.getEmbeddedTimelineServerCompressOutput(),
|
|
||||||
config.getEmbeddedTimelineServerUseAsync());
|
|
||||||
timelineService.startServer();
|
timelineService.startServer();
|
||||||
updateWriteConfigWithTimelineServer(timelineService, config);
|
updateWriteConfigWithTimelineServer(timelineService, config);
|
||||||
return timelineService;
|
return timelineService;
|
||||||
|
|||||||
@@ -18,14 +18,15 @@
|
|||||||
|
|
||||||
package org.apache.hudi.client.embedded;
|
package org.apache.hudi.client.embedded;
|
||||||
|
|
||||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
|
||||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
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;
|
||||||
import org.apache.hudi.common.util.NetworkUtils;
|
import org.apache.hudi.common.util.NetworkUtils;
|
||||||
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
import org.apache.hudi.table.marker.MarkerType;
|
||||||
import org.apache.hudi.timeline.service.TimelineService;
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
@@ -41,53 +42,55 @@ public class EmbeddedTimelineService {
|
|||||||
private static final Logger LOG = LogManager.getLogger(EmbeddedTimelineService.class);
|
private static final Logger LOG = LogManager.getLogger(EmbeddedTimelineService.class);
|
||||||
|
|
||||||
private int serverPort;
|
private int serverPort;
|
||||||
private int preferredPort;
|
|
||||||
private String hostAddr;
|
private String hostAddr;
|
||||||
private HoodieEngineContext context;
|
private HoodieEngineContext context;
|
||||||
private final SerializableConfiguration hadoopConf;
|
private final SerializableConfiguration hadoopConf;
|
||||||
private final FileSystemViewStorageConfig config;
|
private final HoodieWriteConfig writeConfig;
|
||||||
private final HoodieMetadataConfig metadataConfig;
|
|
||||||
private final HoodieCommonConfig commonConfig;
|
|
||||||
private final String basePath;
|
private final String basePath;
|
||||||
|
|
||||||
private final int numThreads;
|
|
||||||
private final boolean shouldCompressOutput;
|
|
||||||
private final boolean useAsync;
|
|
||||||
private transient FileSystemViewManager viewManager;
|
private transient FileSystemViewManager viewManager;
|
||||||
private transient TimelineService server;
|
private transient TimelineService server;
|
||||||
|
|
||||||
public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, int embeddedTimelineServerPort,
|
public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig) {
|
||||||
HoodieMetadataConfig metadataConfig, HoodieCommonConfig commonConfig, FileSystemViewStorageConfig config, String basePath,
|
|
||||||
int numThreads, boolean compressOutput, boolean useAsync) {
|
|
||||||
setHostAddr(embeddedTimelineServiceHostAddr);
|
setHostAddr(embeddedTimelineServiceHostAddr);
|
||||||
this.context = context;
|
this.context = context;
|
||||||
this.config = config;
|
this.writeConfig = writeConfig;
|
||||||
this.basePath = basePath;
|
this.basePath = writeConfig.getBasePath();
|
||||||
this.metadataConfig = metadataConfig;
|
|
||||||
this.commonConfig = commonConfig;
|
|
||||||
this.hadoopConf = context.getHadoopConf();
|
this.hadoopConf = context.getHadoopConf();
|
||||||
this.viewManager = createViewManager();
|
this.viewManager = createViewManager();
|
||||||
this.preferredPort = embeddedTimelineServerPort;
|
|
||||||
this.numThreads = numThreads;
|
|
||||||
this.shouldCompressOutput = compressOutput;
|
|
||||||
this.useAsync = useAsync;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private FileSystemViewManager createViewManager() {
|
private FileSystemViewManager createViewManager() {
|
||||||
// Using passed-in configs to build view storage configs
|
// Using passed-in configs to build view storage configs
|
||||||
FileSystemViewStorageConfig.Builder builder =
|
FileSystemViewStorageConfig.Builder builder =
|
||||||
FileSystemViewStorageConfig.newBuilder().fromProperties(config.getProps());
|
FileSystemViewStorageConfig.newBuilder().fromProperties(writeConfig.getClientSpecifiedViewStorageConfig().getProps());
|
||||||
FileSystemViewStorageType storageType = builder.build().getStorageType();
|
FileSystemViewStorageType storageType = builder.build().getStorageType();
|
||||||
if (storageType.equals(FileSystemViewStorageType.REMOTE_ONLY)
|
if (storageType.equals(FileSystemViewStorageType.REMOTE_ONLY)
|
||||||
|| storageType.equals(FileSystemViewStorageType.REMOTE_FIRST)) {
|
|| storageType.equals(FileSystemViewStorageType.REMOTE_FIRST)) {
|
||||||
// Reset to default if set to Remote
|
// Reset to default if set to Remote
|
||||||
builder.withStorageType(FileSystemViewStorageType.MEMORY);
|
builder.withStorageType(FileSystemViewStorageType.MEMORY);
|
||||||
}
|
}
|
||||||
return FileSystemViewManager.createViewManager(context, metadataConfig, builder.build(), commonConfig, basePath);
|
return FileSystemViewManager.createViewManager(context, writeConfig.getMetadataConfig(), builder.build(), writeConfig.getCommonConfig(), basePath);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void startServer() throws IOException {
|
public void startServer() throws IOException {
|
||||||
server = new TimelineService(preferredPort, viewManager, hadoopConf.newCopy(), numThreads, shouldCompressOutput, useAsync);
|
TimelineService.Config.Builder timelineServiceConfBuilder = TimelineService.Config.builder()
|
||||||
|
.serverPort(writeConfig.getEmbeddedTimelineServerPort())
|
||||||
|
.numThreads(writeConfig.getEmbeddedTimelineServerThreads())
|
||||||
|
.compress(writeConfig.getEmbeddedTimelineServerCompressOutput())
|
||||||
|
.async(writeConfig.getEmbeddedTimelineServerUseAsync());
|
||||||
|
// Only passing marker-related write configs to timeline server
|
||||||
|
// if timeline-server-based markers are used.
|
||||||
|
if (writeConfig.getMarkersType() == MarkerType.TIMELINE_SERVER_BASED) {
|
||||||
|
timelineServiceConfBuilder
|
||||||
|
.enableMarkerRequests(true)
|
||||||
|
.markerBatchNumThreads(writeConfig.getMarkersTimelineServerBasedBatchNumThreads())
|
||||||
|
.markerBatchIntervalMs(writeConfig.getMarkersTimelineServerBasedBatchIntervalMs())
|
||||||
|
.markerParallelism(writeConfig.getMarkersDeleteParallelism());
|
||||||
|
}
|
||||||
|
|
||||||
|
server = new TimelineService(context, hadoopConf.newCopy(), timelineServiceConfBuilder.build(),
|
||||||
|
FSUtils.getFs(basePath, hadoopConf.newCopy()), viewManager);
|
||||||
serverPort = server.startService();
|
serverPort = server.startService();
|
||||||
LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
|
LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
|
||||||
}
|
}
|
||||||
@@ -106,7 +109,8 @@ public class EmbeddedTimelineService {
|
|||||||
* Retrieves proper view storage configs for remote clients to access this service.
|
* Retrieves proper view storage configs for remote clients to access this service.
|
||||||
*/
|
*/
|
||||||
public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
|
public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
|
||||||
FileSystemViewStorageType viewStorageType = config.shouldEnableBackupForRemoteFileSystemView()
|
FileSystemViewStorageType viewStorageType = writeConfig.getClientSpecifiedViewStorageConfig()
|
||||||
|
.shouldEnableBackupForRemoteFileSystemView()
|
||||||
? FileSystemViewStorageType.REMOTE_FIRST : FileSystemViewStorageType.REMOTE_ONLY;
|
? FileSystemViewStorageType.REMOTE_FIRST : FileSystemViewStorageType.REMOTE_ONLY;
|
||||||
return FileSystemViewStorageConfig.newBuilder().withStorageType(viewStorageType)
|
return FileSystemViewStorageConfig.newBuilder().withStorageType(viewStorageType)
|
||||||
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
|
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
|
||||||
|
|||||||
@@ -46,6 +46,7 @@ import org.apache.hudi.metrics.MetricsReporterType;
|
|||||||
import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
|
import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
|
||||||
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
|
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
|
||||||
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
||||||
|
import org.apache.hudi.table.marker.MarkerType;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||||
import org.apache.orc.CompressionKind;
|
import org.apache.orc.CompressionKind;
|
||||||
@@ -227,6 +228,30 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
+ "files from lake storage, before committing the write. Reduce this value, if the high number of tasks incur delays for smaller tables "
|
+ "files from lake storage, before committing the write. Reduce this value, if the high number of tasks incur delays for smaller tables "
|
||||||
+ "or low latency writes.");
|
+ "or low latency writes.");
|
||||||
|
|
||||||
|
public static final ConfigProperty<String> MARKERS_TYPE_PROP = ConfigProperty
|
||||||
|
.key("hoodie.write.markers.type")
|
||||||
|
.defaultValue(MarkerType.DIRECT.toString())
|
||||||
|
.sinceVersion("0.9.0")
|
||||||
|
.withDocumentation("Marker type to use. Two modes are supported: "
|
||||||
|
+ "- DIRECT: individual marker file corresponding to each data file is directly "
|
||||||
|
+ "created by the writer. "
|
||||||
|
+ "- 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 "
|
||||||
|
+ "in a limited number of underlying files for efficiency.");
|
||||||
|
|
||||||
|
public static final ConfigProperty<Integer> MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS_PROP = ConfigProperty
|
||||||
|
.key("hoodie.markers.timeline_server_based.batch.num_threads")
|
||||||
|
.defaultValue(20)
|
||||||
|
.sinceVersion("0.9.0")
|
||||||
|
.withDocumentation("Number of threads to use for batch processing marker "
|
||||||
|
+ "creation requests at the timeline server");
|
||||||
|
|
||||||
|
public static final ConfigProperty<Long> MARKERS_TIMELINE_SERVER_BASED_BATCH_INTERVAL_MS_PROP = ConfigProperty
|
||||||
|
.key("hoodie.markers.timeline_server_based.batch.interval_ms")
|
||||||
|
.defaultValue(50L)
|
||||||
|
.sinceVersion("0.9.0")
|
||||||
|
.withDocumentation("The batch interval in milliseconds for marker creation batch processing");
|
||||||
|
|
||||||
public static final ConfigProperty<String> MARKERS_DELETE_PARALLELISM = ConfigProperty
|
public static final ConfigProperty<String> MARKERS_DELETE_PARALLELISM = ConfigProperty
|
||||||
.key("hoodie.markers.delete.parallelism")
|
.key("hoodie.markers.delete.parallelism")
|
||||||
.defaultValue("100")
|
.defaultValue("100")
|
||||||
@@ -542,6 +567,19 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
return getInt(FINALIZE_WRITE_PARALLELISM);
|
return getInt(FINALIZE_WRITE_PARALLELISM);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public MarkerType getMarkersType() {
|
||||||
|
String markerType = getString(MARKERS_TYPE_PROP);
|
||||||
|
return MarkerType.valueOf(markerType.toUpperCase());
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getMarkersTimelineServerBasedBatchNumThreads() {
|
||||||
|
return getInt(MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS_PROP);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getMarkersTimelineServerBasedBatchIntervalMs() {
|
||||||
|
return getLong(MARKERS_TIMELINE_SERVER_BASED_BATCH_INTERVAL_MS_PROP);
|
||||||
|
}
|
||||||
|
|
||||||
public int getMarkersDeleteParallelism() {
|
public int getMarkersDeleteParallelism() {
|
||||||
return getInt(MARKERS_DELETE_PARALLELISM);
|
return getInt(MARKERS_DELETE_PARALLELISM);
|
||||||
}
|
}
|
||||||
@@ -1556,6 +1594,21 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder withMarkersType(String markerType) {
|
||||||
|
writeConfig.setValue(MARKERS_TYPE_PROP, markerType);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withMarkersTimelineServerBasedBatchNumThreads(int numThreads) {
|
||||||
|
writeConfig.setValue(MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS_PROP, String.valueOf(numThreads));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withMarkersTimelineServerBasedBatchIntervalMs(long intervalMs) {
|
||||||
|
writeConfig.setValue(MARKERS_TIMELINE_SERVER_BASED_BATCH_INTERVAL_MS_PROP, String.valueOf(intervalMs));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public Builder withMarkersDeleteParallelism(int parallelism) {
|
public Builder withMarkersDeleteParallelism(int parallelism) {
|
||||||
writeConfig.setValue(MARKERS_DELETE_PARALLELISM, String.valueOf(parallelism));
|
writeConfig.setValue(MARKERS_DELETE_PARALLELISM, String.valueOf(parallelism));
|
||||||
return this;
|
return this;
|
||||||
|
|||||||
@@ -33,7 +33,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
|||||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||||
import org.apache.hudi.io.storage.HoodieFileWriterFactory;
|
import org.apache.hudi.io.storage.HoodieFileWriterFactory;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
@@ -177,8 +177,8 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
* @param partitionPath Partition path
|
* @param partitionPath Partition path
|
||||||
*/
|
*/
|
||||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime)
|
||||||
markerFiles.create(partitionPath, dataFileName, getIOType());
|
.create(partitionPath, dataFileName, getIOType());
|
||||||
}
|
}
|
||||||
|
|
||||||
public Schema getWriterSchemaWithMetaFields() {
|
public Schema getWriterSchemaWithMetaFields() {
|
||||||
|
|||||||
@@ -68,6 +68,9 @@ import org.apache.hudi.index.HoodieIndex;
|
|||||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -482,7 +485,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
|||||||
/**
|
/**
|
||||||
* Returns the possible invalid data file name with given marker files.
|
* Returns the possible invalid data file name with given marker files.
|
||||||
*/
|
*/
|
||||||
protected Set<String> getInvalidDataPaths(MarkerFiles markers) throws IOException {
|
protected Set<String> getInvalidDataPaths(WriteMarkers markers) throws IOException {
|
||||||
return markers.createdAndMergedDataPaths(context, config.getFinalizeWriteParallelism());
|
return markers.createdAndMergedDataPaths(context, config.getFinalizeWriteParallelism());
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -504,7 +507,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
|||||||
// Reconcile marker and data files with WriteStats so that partially written data-files due to failed
|
// Reconcile marker and data files with WriteStats so that partially written data-files due to failed
|
||||||
// (but succeeded on retry) tasks are removed.
|
// (but succeeded on retry) tasks are removed.
|
||||||
String basePath = getMetaClient().getBasePath();
|
String basePath = getMetaClient().getBasePath();
|
||||||
MarkerFiles markers = new MarkerFiles(this, instantTs);
|
WriteMarkers markers = WriteMarkersFactory.get(config.getMarkersType(), this, instantTs);
|
||||||
|
|
||||||
if (!markers.doesMarkerDirExist()) {
|
if (!markers.doesMarkerDirExist()) {
|
||||||
// can happen if it was an empty write say.
|
// can happen if it was an empty write say.
|
||||||
|
|||||||
@@ -49,6 +49,9 @@ import org.apache.hudi.exception.HoodieCommitException;
|
|||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -306,7 +309,7 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
|
|||||||
LOG.warn("Unable to delete file(s) for " + hoodieInstant.getFileName() + ", replaced files possibly deleted by cleaner");
|
LOG.warn("Unable to delete file(s) for " + hoodieInstant.getFileName() + ", replaced files possibly deleted by cleaner");
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
deleteAnyLeftOverMarkerFiles(context, hoodieInstant);
|
deleteAnyLeftOverMarkers(context, hoodieInstant);
|
||||||
records.add(convertToAvroRecord(hoodieInstant));
|
records.add(convertToAvroRecord(hoodieInstant));
|
||||||
if (records.size() >= this.config.getCommitArchivalBatchSize()) {
|
if (records.size() >= this.config.getCommitArchivalBatchSize()) {
|
||||||
writeToFile(wrapperSchema, records);
|
writeToFile(wrapperSchema, records);
|
||||||
@@ -324,9 +327,9 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void deleteAnyLeftOverMarkerFiles(HoodieEngineContext context, HoodieInstant instant) {
|
private void deleteAnyLeftOverMarkers(HoodieEngineContext context, HoodieInstant instant) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, instant.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, instant.getTimestamp());
|
||||||
if (markerFiles.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) {
|
if (writeMarkers.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) {
|
||||||
LOG.info("Cleaned up left over marker directory for instant :" + instant);
|
LOG.info("Cleaned up left over marker directory for instant :" + instant);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -36,8 +36,9 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.exception.HoodieRollbackException;
|
import org.apache.hudi.exception.HoodieRollbackException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -110,8 +111,9 @@ public abstract class BaseRollbackActionExecutor<T extends HoodieRecordPayload,
|
|||||||
finishRollback(rollbackMetadata);
|
finishRollback(rollbackMetadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Finally, remove the marker files post rollback.
|
// Finally, remove the markers post rollback.
|
||||||
new MarkerFiles(table, instantToRollback.getTimestamp()).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp())
|
||||||
|
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||||
|
|
||||||
return rollbackMetadata;
|
return rollbackMetadata;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,16 +16,18 @@
|
|||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hudi.table;
|
package org.apache.hudi.table.marker;
|
||||||
|
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
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.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.IOType;
|
import org.apache.hudi.common.model.IOType;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
@@ -36,7 +38,6 @@ import org.apache.log4j.LogManager;
|
|||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
@@ -45,41 +46,28 @@ import java.util.Set;
|
|||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Operates on marker files for a given write action (commit, delta commit, compaction).
|
* Marker operations of directly accessing the file system to create and delete
|
||||||
|
* marker files. Each data file has a corresponding marker file.
|
||||||
*/
|
*/
|
||||||
public class MarkerFiles implements Serializable {
|
public class DirectWriteMarkers extends WriteMarkers {
|
||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(MarkerFiles.class);
|
private static final Logger LOG = LogManager.getLogger(DirectWriteMarkers.class);
|
||||||
|
|
||||||
private final String instantTime;
|
|
||||||
private final transient FileSystem fs;
|
private final transient FileSystem fs;
|
||||||
private final transient Path markerDirPath;
|
|
||||||
private final String basePath;
|
|
||||||
|
|
||||||
public MarkerFiles(FileSystem fs, String basePath, String markerFolderPath, String instantTime) {
|
public DirectWriteMarkers(FileSystem fs, String basePath, String markerFolderPath, String instantTime) {
|
||||||
this.instantTime = instantTime;
|
super(basePath, markerFolderPath, instantTime);
|
||||||
this.fs = fs;
|
this.fs = fs;
|
||||||
this.markerDirPath = new Path(markerFolderPath);
|
|
||||||
this.basePath = basePath;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public MarkerFiles(HoodieTable table, String instantTime) {
|
public DirectWriteMarkers(HoodieTable table, String instantTime) {
|
||||||
this(table.getMetaClient().getFs(),
|
this(table.getMetaClient().getFs(),
|
||||||
table.getMetaClient().getBasePath(),
|
table.getMetaClient().getBasePath(),
|
||||||
table.getMetaClient().getMarkerFolderPath(instantTime),
|
table.getMetaClient().getMarkerFolderPath(instantTime),
|
||||||
instantTime);
|
instantTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) {
|
|
||||||
try {
|
|
||||||
deleteMarkerDir(context, parallelism);
|
|
||||||
} catch (HoodieIOException ioe) {
|
|
||||||
LOG.warn("Error deleting marker directory for instant " + instantTime, ioe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete Marker directory corresponding to an instant.
|
* Deletes Marker directory corresponding to an instant.
|
||||||
*
|
*
|
||||||
* @param context HoodieEngineContext.
|
* @param context HoodieEngineContext.
|
||||||
* @param parallelism parallelism for deletion.
|
* @param parallelism parallelism for deletion.
|
||||||
@@ -112,10 +100,15 @@ public class MarkerFiles implements Serializable {
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return {@code true} if marker directory exists; {@code false} otherwise.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
public boolean doesMarkerDirExist() throws IOException {
|
public boolean doesMarkerDirExist() throws IOException {
|
||||||
return fs.exists(markerDirPath);
|
return fs.exists(markerDirPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Set<String> createdAndMergedDataPaths(HoodieEngineContext context, int parallelism) throws IOException {
|
public Set<String> createdAndMergedDataPaths(HoodieEngineContext context, int parallelism) throws IOException {
|
||||||
Set<String> dataFiles = new HashSet<>();
|
Set<String> dataFiles = new HashSet<>();
|
||||||
|
|
||||||
@@ -157,15 +150,12 @@ public class MarkerFiles implements Serializable {
|
|||||||
|
|
||||||
private String translateMarkerToDataPath(String markerPath) {
|
private String translateMarkerToDataPath(String markerPath) {
|
||||||
String rPath = stripMarkerFolderPrefix(markerPath);
|
String rPath = stripMarkerFolderPrefix(markerPath);
|
||||||
return MarkerFiles.stripMarkerSuffix(rPath);
|
return stripMarkerSuffix(rPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static String stripMarkerSuffix(String path) {
|
@Override
|
||||||
return path.substring(0, path.indexOf(HoodieTableMetaClient.MARKER_EXTN));
|
public Set<String> allMarkerFilePaths() throws IOException {
|
||||||
}
|
Set<String> markerFiles = new HashSet<>();
|
||||||
|
|
||||||
public List<String> allMarkerFilePaths() throws IOException {
|
|
||||||
List<String> markerFiles = new ArrayList<>();
|
|
||||||
if (doesMarkerDirExist()) {
|
if (doesMarkerDirExist()) {
|
||||||
FSUtils.processFiles(fs, markerDirPath.toString(), fileStatus -> {
|
FSUtils.processFiles(fs, markerDirPath.toString(), fileStatus -> {
|
||||||
markerFiles.add(stripMarkerFolderPrefix(fileStatus.getPath().toString()));
|
markerFiles.add(stripMarkerFolderPrefix(fileStatus.getPath().toString()));
|
||||||
@@ -175,70 +165,30 @@ public class MarkerFiles implements Serializable {
|
|||||||
return markerFiles;
|
return markerFiles;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String stripMarkerFolderPrefix(String fullMarkerPath) {
|
@Override
|
||||||
ValidationUtils.checkArgument(fullMarkerPath.contains(HoodieTableMetaClient.MARKER_EXTN));
|
protected Option<Path> create(String partitionPath, String dataFileName, IOType type, boolean checkIfExists) {
|
||||||
String markerRootPath = Path.getPathWithoutSchemeAndAuthority(
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
new Path(String.format("%s/%s/%s", basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime))).toString();
|
|
||||||
int begin = fullMarkerPath.indexOf(markerRootPath);
|
|
||||||
ValidationUtils.checkArgument(begin >= 0,
|
|
||||||
"Not in marker dir. Marker Path=" + fullMarkerPath + ", Expected Marker Root=" + markerRootPath);
|
|
||||||
return fullMarkerPath.substring(begin + markerRootPath.length() + 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
|
|
||||||
*/
|
|
||||||
public Path create(String partitionPath, String dataFileName, IOType type) {
|
|
||||||
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
|
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
|
||||||
|
Path dirPath = markerPath.getParent();
|
||||||
try {
|
try {
|
||||||
LOG.info("Creating Marker Path=" + markerPath);
|
if (!fs.exists(dirPath)) {
|
||||||
fs.create(markerPath, false).close();
|
fs.mkdirs(dirPath); // create a new partition as needed.
|
||||||
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieException("Failed to create marker file " + markerPath, e);
|
throw new HoodieIOException("Failed to make dir " + dirPath, e);
|
||||||
}
|
}
|
||||||
return markerPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
|
|
||||||
*
|
|
||||||
* @return true if the marker file creates successfully,
|
|
||||||
* false if it already exists
|
|
||||||
*/
|
|
||||||
public boolean createIfNotExists(String partitionPath, String dataFileName, IOType type) {
|
|
||||||
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
|
|
||||||
try {
|
try {
|
||||||
if (fs.exists(markerPath)) {
|
if (checkIfExists && fs.exists(markerPath)) {
|
||||||
LOG.warn("Marker Path=" + markerPath + " already exists, cancel creation");
|
LOG.warn("Marker Path=" + markerPath + " already exists, cancel creation");
|
||||||
return false;
|
return Option.empty();
|
||||||
}
|
}
|
||||||
LOG.info("Creating Marker Path=" + markerPath);
|
LOG.info("Creating Marker Path=" + markerPath);
|
||||||
fs.create(markerPath, false).close();
|
fs.create(markerPath, false).close();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieException("Failed to create marker file " + markerPath, e);
|
throw new HoodieException("Failed to create marker file " + markerPath, e);
|
||||||
}
|
}
|
||||||
return true;
|
LOG.info("[direct] Created marker file " + markerPath.toString()
|
||||||
|
+ " in " + timer.endTimer() + " ms");
|
||||||
|
return Option.of(markerPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the marker path. Would create the partition path first if not exists.
|
|
||||||
*
|
|
||||||
* @param partitionPath The partition path
|
|
||||||
* @param dataFileName The data file name
|
|
||||||
* @param type The IO type
|
|
||||||
* @return path of the marker file
|
|
||||||
*/
|
|
||||||
private Path getMarkerPath(String partitionPath, String dataFileName, IOType type) {
|
|
||||||
Path path = FSUtils.getPartitionPath(markerDirPath, partitionPath);
|
|
||||||
try {
|
|
||||||
if (!fs.exists(path)) {
|
|
||||||
fs.mkdirs(path); // create a new partition as needed.
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
|
||||||
}
|
|
||||||
String markerFileName = String.format("%s%s.%s", dataFileName, HoodieTableMetaClient.MARKER_EXTN, type.name());
|
|
||||||
return new Path(path, markerFileName);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
@@ -0,0 +1,27 @@
|
|||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marker type indicating how markers are stored in the file system.
|
||||||
|
*/
|
||||||
|
public enum MarkerType {
|
||||||
|
DIRECT,
|
||||||
|
TIMELINE_SERVER_BASED
|
||||||
|
}
|
||||||
@@ -0,0 +1,179 @@
|
|||||||
|
/*
|
||||||
|
* 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.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.model.IOType;
|
||||||
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.exception.HoodieRemoteException;
|
||||||
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.http.client.fluent.Request;
|
||||||
|
import org.apache.http.client.fluent.Response;
|
||||||
|
import org.apache.http.client.utils.URIBuilder;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.ALL_MARKERS_URL;
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.CREATE_AND_MERGE_MARKERS_URL;
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.CREATE_MARKER_URL;
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.DELETE_MARKER_DIR_URL;
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.MARKERS_DIR_EXISTS_URL;
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_DIR_PATH_PARAM;
|
||||||
|
import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_NAME_PARAM;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marker operations of using timeline server as a proxy to create and delete markers.
|
||||||
|
* Each data file has a corresponding marker entry, which is stored in a limited number of
|
||||||
|
* underlying files maintained by the timeline server (each file contains multiple marker
|
||||||
|
* entries).
|
||||||
|
*/
|
||||||
|
public class TimelineServerBasedWriteMarkers extends WriteMarkers {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(TimelineServerBasedWriteMarkers.class);
|
||||||
|
private final ObjectMapper mapper;
|
||||||
|
private final String timelineServerHost;
|
||||||
|
private final int timelineServerPort;
|
||||||
|
private final int timeoutSecs;
|
||||||
|
|
||||||
|
public TimelineServerBasedWriteMarkers(HoodieTable table, String instantTime) {
|
||||||
|
this(table.getMetaClient().getBasePath(),
|
||||||
|
table.getMetaClient().getMarkerFolderPath(instantTime), instantTime,
|
||||||
|
table.getConfig().getViewStorageConfig().getRemoteViewServerHost(),
|
||||||
|
table.getConfig().getViewStorageConfig().getRemoteViewServerPort(),
|
||||||
|
table.getConfig().getViewStorageConfig().getRemoteTimelineClientTimeoutSecs());
|
||||||
|
}
|
||||||
|
|
||||||
|
TimelineServerBasedWriteMarkers(String basePath, String markerFolderPath, String instantTime,
|
||||||
|
String timelineServerHost, int timelineServerPort, int timeoutSecs) {
|
||||||
|
super(basePath, markerFolderPath, instantTime);
|
||||||
|
this.mapper = new ObjectMapper();
|
||||||
|
this.timelineServerHost = timelineServerHost;
|
||||||
|
this.timelineServerPort = timelineServerPort;
|
||||||
|
this.timeoutSecs = timeoutSecs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean deleteMarkerDir(HoodieEngineContext context, int parallelism) {
|
||||||
|
Map<String, String> paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
||||||
|
try {
|
||||||
|
return executeRequestToTimelineServer(
|
||||||
|
DELETE_MARKER_DIR_URL, paramsMap, new TypeReference<Boolean>() {}, RequestMethod.POST);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieRemoteException("Failed to delete marker directory " + markerDirPath.toString(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean doesMarkerDirExist() {
|
||||||
|
Map<String, String> paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
||||||
|
try {
|
||||||
|
return executeRequestToTimelineServer(
|
||||||
|
MARKERS_DIR_EXISTS_URL, paramsMap, new TypeReference<Boolean>() {}, RequestMethod.GET);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieRemoteException("Failed to check marker directory " + markerDirPath.toString(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> createdAndMergedDataPaths(HoodieEngineContext context, int parallelism) throws IOException {
|
||||||
|
Map<String, String> paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
||||||
|
try {
|
||||||
|
Set<String> markerPaths = executeRequestToTimelineServer(
|
||||||
|
CREATE_AND_MERGE_MARKERS_URL, paramsMap, new TypeReference<Set<String>>() {}, RequestMethod.GET);
|
||||||
|
return markerPaths.stream().map(WriteMarkers::stripMarkerSuffix).collect(Collectors.toSet());
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieRemoteException("Failed to get CREATE and MERGE data file paths in "
|
||||||
|
+ markerDirPath.toString(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> allMarkerFilePaths() {
|
||||||
|
Map<String, String> paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
||||||
|
try {
|
||||||
|
return executeRequestToTimelineServer(
|
||||||
|
ALL_MARKERS_URL, paramsMap, new TypeReference<Set<String>>() {}, RequestMethod.GET);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieRemoteException("Failed to get all markers in " + markerDirPath.toString(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Option<Path> create(String partitionPath, String dataFileName, IOType type, boolean checkIfExists) {
|
||||||
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
|
String markerFileName = getMarkerFileName(dataFileName, type);
|
||||||
|
|
||||||
|
Map<String, String> paramsMap = new HashMap<>();
|
||||||
|
paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString());
|
||||||
|
paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName);
|
||||||
|
boolean success;
|
||||||
|
try {
|
||||||
|
success = executeRequestToTimelineServer(
|
||||||
|
CREATE_MARKER_URL, paramsMap, new TypeReference<Boolean>() {}, RequestMethod.POST);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieRemoteException("Failed to create marker file " + partitionPath + "/" + markerFileName, e);
|
||||||
|
}
|
||||||
|
LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName
|
||||||
|
+ " in " + timer.endTimer() + " ms");
|
||||||
|
if (success) {
|
||||||
|
return Option.of(new Path(new Path(markerDirPath, partitionPath), markerFileName));
|
||||||
|
} else {
|
||||||
|
return Option.empty();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private <T> T executeRequestToTimelineServer(String requestPath, Map<String, String> queryParameters,
|
||||||
|
TypeReference reference, RequestMethod method) throws IOException {
|
||||||
|
URIBuilder builder =
|
||||||
|
new URIBuilder().setHost(timelineServerHost).setPort(timelineServerPort).setPath(requestPath).setScheme("http");
|
||||||
|
|
||||||
|
queryParameters.forEach(builder::addParameter);
|
||||||
|
|
||||||
|
String url = builder.toString();
|
||||||
|
LOG.info("Sending request : (" + url + ")");
|
||||||
|
Response response;
|
||||||
|
int timeout = this.timeoutSecs * 1000; // msec
|
||||||
|
switch (method) {
|
||||||
|
case GET:
|
||||||
|
response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
|
||||||
|
break;
|
||||||
|
case POST:
|
||||||
|
default:
|
||||||
|
response = Request.Post(url).connectTimeout(timeout).socketTimeout(timeout).execute();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
String content = response.returnContent().asString();
|
||||||
|
return (T) mapper.readValue(content, reference);
|
||||||
|
}
|
||||||
|
|
||||||
|
private enum RequestMethod {
|
||||||
|
GET, POST
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,186 @@
|
|||||||
|
/*
|
||||||
|
* 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.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.model.IOType;
|
||||||
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Operates on markers for a given write action (commit, delta commit, compaction).
|
||||||
|
*
|
||||||
|
* This abstract class provides abstract methods of different marker operations, so that
|
||||||
|
* different marker write mechanism can be implemented.
|
||||||
|
*/
|
||||||
|
public abstract class WriteMarkers implements Serializable {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(WriteMarkers.class);
|
||||||
|
|
||||||
|
protected final String basePath;
|
||||||
|
protected final transient Path markerDirPath;
|
||||||
|
protected final String instantTime;
|
||||||
|
|
||||||
|
public WriteMarkers(String basePath, String markerFolderPath, String instantTime) {
|
||||||
|
this.basePath = basePath;
|
||||||
|
this.markerDirPath = new Path(markerFolderPath);
|
||||||
|
this.instantTime = instantTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a marker without checking if the marker already exists.
|
||||||
|
*
|
||||||
|
* @param partitionPath partition path in the table
|
||||||
|
* @param dataFileName data file name
|
||||||
|
* @param type write IO type
|
||||||
|
* @return the marker path
|
||||||
|
*/
|
||||||
|
public Option<Path> create(String partitionPath, String dataFileName, IOType type) {
|
||||||
|
return create(partitionPath, dataFileName, type, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a marker if the marker does not exist.
|
||||||
|
*
|
||||||
|
* @param partitionPath partition path in the table
|
||||||
|
* @param dataFileName data file name
|
||||||
|
* @param type write IO type
|
||||||
|
* @return the marker path or empty option if already exists
|
||||||
|
*/
|
||||||
|
public Option<Path> createIfNotExists(String partitionPath, String dataFileName, IOType type) {
|
||||||
|
return create(partitionPath, dataFileName, type, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Quietly deletes the marker directory.
|
||||||
|
*
|
||||||
|
* @param context {@code HoodieEngineContext} instance.
|
||||||
|
* @param parallelism parallelism for deleting the marker files in the directory.
|
||||||
|
*/
|
||||||
|
public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) {
|
||||||
|
try {
|
||||||
|
deleteMarkerDir(context, parallelism);
|
||||||
|
} catch (HoodieIOException ioe) {
|
||||||
|
LOG.warn("Error deleting marker directory for instant " + instantTime, ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strips the marker file suffix from the input path, i.e., ".marker.[IO_type]".
|
||||||
|
*
|
||||||
|
* @param path file path
|
||||||
|
* @return Stripped path
|
||||||
|
*/
|
||||||
|
public static String stripMarkerSuffix(String path) {
|
||||||
|
return path.substring(0, path.indexOf(HoodieTableMetaClient.MARKER_EXTN));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the marker file name, in the format of "[data_file_name].marker.[IO_type]".
|
||||||
|
*
|
||||||
|
* @param dataFileName data file name
|
||||||
|
* @param type IO type
|
||||||
|
* @return the marker file name
|
||||||
|
*/
|
||||||
|
protected String getMarkerFileName(String dataFileName, IOType type) {
|
||||||
|
return String.format("%s%s.%s", dataFileName, HoodieTableMetaClient.MARKER_EXTN, type.name());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the marker path. Would create the partition path first if not exists
|
||||||
|
*
|
||||||
|
* @param partitionPath The partition path
|
||||||
|
* @param dataFileName The data file name
|
||||||
|
* @param type The IO type
|
||||||
|
* @return path of the marker file
|
||||||
|
*/
|
||||||
|
protected Path getMarkerPath(String partitionPath, String dataFileName, IOType type) {
|
||||||
|
Path path = FSUtils.getPartitionPath(markerDirPath, partitionPath);
|
||||||
|
String markerFileName = getMarkerFileName(dataFileName, type);
|
||||||
|
return new Path(path, markerFileName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strips the folder prefix of the marker file path.
|
||||||
|
*
|
||||||
|
* @param fullMarkerPath the full path of the marker file
|
||||||
|
* @return marker file name
|
||||||
|
*/
|
||||||
|
protected String stripMarkerFolderPrefix(String fullMarkerPath) {
|
||||||
|
ValidationUtils.checkArgument(fullMarkerPath.contains(HoodieTableMetaClient.MARKER_EXTN));
|
||||||
|
String markerRootPath = Path.getPathWithoutSchemeAndAuthority(
|
||||||
|
new Path(String.format("%s/%s/%s", basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime))).toString();
|
||||||
|
int begin =
|
||||||
|
fullMarkerPath.indexOf(markerRootPath);
|
||||||
|
ValidationUtils.checkArgument(begin >= 0,
|
||||||
|
"Not in marker dir. Marker Path=" + fullMarkerPath + ", Expected Marker Root=" + markerRootPath);
|
||||||
|
return fullMarkerPath.substring(begin + markerRootPath.length() + 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deletes the marker directory.
|
||||||
|
*
|
||||||
|
* @param context {@code HoodieEngineContext} instance.
|
||||||
|
* @param parallelism parallelism for deleting the marker files in the directory.
|
||||||
|
* @return {@true} if successful; {@false} otherwise.
|
||||||
|
*/
|
||||||
|
public abstract boolean deleteMarkerDir(HoodieEngineContext context, int parallelism);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return {@true} if the marker directory exists in the file system; {@false} otherwise.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public abstract boolean doesMarkerDirExist() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param context {@code HoodieEngineContext} instance.
|
||||||
|
* @param parallelism parallelism for reading the marker files in the directory.
|
||||||
|
* @return all the data file paths of write IO type "CREATE" and "MERGE"
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public abstract Set<String> createdAndMergedDataPaths(HoodieEngineContext context, int parallelism) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return all the marker paths
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public abstract Set<String> allMarkerFilePaths() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a marker.
|
||||||
|
*
|
||||||
|
* @param partitionPath partition path in the table
|
||||||
|
* @param dataFileName data file name
|
||||||
|
* @param type write IO type
|
||||||
|
* @param checkIfExists whether to check if the marker already exists
|
||||||
|
* @return the marker path or empty option if already exists and {@code checkIfExists} is true
|
||||||
|
*/
|
||||||
|
abstract Option<Path> create(String partitionPath, String dataFileName, IOType type, boolean checkIfExists);
|
||||||
|
}
|
||||||
@@ -0,0 +1,58 @@
|
|||||||
|
/*
|
||||||
|
* 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.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.fs.StorageSchemes;
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A factory to generate {@code WriteMarkers} instance based on the {@code MarkerType}.
|
||||||
|
*/
|
||||||
|
public class WriteMarkersFactory {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(WriteMarkersFactory.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param markerType the type of markers to use
|
||||||
|
* @param table {@code HoodieTable} instance
|
||||||
|
* @param instantTime current instant time
|
||||||
|
* @return {@code WriteMarkers} instance based on the {@code MarkerType}
|
||||||
|
*/
|
||||||
|
public static WriteMarkers get(MarkerType markerType, HoodieTable table, String instantTime) {
|
||||||
|
LOG.debug("Instantiated MarkerFiles with marker type: " + markerType.toString());
|
||||||
|
switch (markerType) {
|
||||||
|
case DIRECT:
|
||||||
|
return new DirectWriteMarkers(table, instantTime);
|
||||||
|
case TIMELINE_SERVER_BASED:
|
||||||
|
String basePath = table.getMetaClient().getBasePath();
|
||||||
|
if (StorageSchemes.HDFS.getScheme().equals(
|
||||||
|
FSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) {
|
||||||
|
throw new HoodieException("Timeline-server-based markers are not supported for HDFS: "
|
||||||
|
+ "base path " + basePath);
|
||||||
|
}
|
||||||
|
return new TimelineServerBasedWriteMarkers(table, instantTime);
|
||||||
|
default:
|
||||||
|
throw new HoodieException("The marker type \"" + markerType.name() + "\" is not supported.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -58,9 +58,9 @@ import org.apache.hudi.table.BulkInsertPartitioner;
|
|||||||
import org.apache.hudi.table.HoodieFlinkTable;
|
import org.apache.hudi.table.HoodieFlinkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
|
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
|
import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
|
||||||
import org.apache.hudi.util.FlinkClientUtil;
|
import org.apache.hudi.util.FlinkClientUtil;
|
||||||
|
|
||||||
@@ -316,7 +316,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
|||||||
Option<Map<String, String>> extraMetadata) {
|
Option<Map<String, String>> extraMetadata) {
|
||||||
try {
|
try {
|
||||||
// Delete the marker directory for the instant.
|
// Delete the marker directory for the instant.
|
||||||
new MarkerFiles(createTable(config, hadoopConf), instantTime)
|
WriteMarkersFactory.get(config.getMarkersType(), createTable(config, hadoopConf), instantTime)
|
||||||
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
|
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
|
||||||
|
|||||||
@@ -24,7 +24,8 @@ import org.apache.hudi.common.model.HoodieRecord;
|
|||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@@ -49,6 +50,7 @@ public class FlinkAppendHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
private static final Logger LOG = LoggerFactory.getLogger(FlinkAppendHandle.class);
|
private static final Logger LOG = LoggerFactory.getLogger(FlinkAppendHandle.class);
|
||||||
|
|
||||||
private boolean isClosed = false;
|
private boolean isClosed = false;
|
||||||
|
private final WriteMarkers writeMarkers;
|
||||||
|
|
||||||
public FlinkAppendHandle(
|
public FlinkAppendHandle(
|
||||||
HoodieWriteConfig config,
|
HoodieWriteConfig config,
|
||||||
@@ -59,6 +61,7 @@ public class FlinkAppendHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
Iterator<HoodieRecord<T>> recordItr,
|
Iterator<HoodieRecord<T>> recordItr,
|
||||||
TaskContextSupplier taskContextSupplier) {
|
TaskContextSupplier taskContextSupplier) {
|
||||||
super(config, instantTime, hoodieTable, partitionPath, fileId, recordItr, taskContextSupplier);
|
super(config, instantTime, hoodieTable, partitionPath, fileId, recordItr, taskContextSupplier);
|
||||||
|
this.writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -66,10 +69,9 @@ public class FlinkAppendHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
// In some rare cases, the task was pulled up again with same write file name,
|
// In some rare cases, the task was pulled up again with same write file name,
|
||||||
// for e.g, reuse the small log files from last commit instant.
|
// for e.g, reuse the small log files from last commit instant.
|
||||||
|
|
||||||
// Just skip the marker file creation if it already exists, the new data would append to
|
// Just skip the marker creation if it already exists, the new data would append to
|
||||||
// the file directly.
|
// the file directly.
|
||||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
writeMarkers.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||||
markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -27,7 +27,8 @@ import org.apache.hudi.common.util.Option;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
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 org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -102,8 +103,8 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime);
|
||||||
markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
writeMarkers.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -29,7 +29,8 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
@@ -104,8 +105,8 @@ public class FlinkMergeAndReplaceHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime);
|
||||||
markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
writeMarkers.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -28,7 +28,8 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
@@ -116,8 +117,8 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime);
|
||||||
markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
writeMarkers.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -31,7 +31,8 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.exception.HoodieInsertException;
|
import org.apache.hudi.exception.HoodieInsertException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.flink.table.data.RowData;
|
import org.apache.flink.table.data.RowData;
|
||||||
import org.apache.flink.table.types.logical.RowType;
|
import org.apache.flink.table.types.logical.RowType;
|
||||||
@@ -188,8 +189,8 @@ public class HoodieRowDataCreateHandle implements Serializable {
|
|||||||
* @param partitionPath Partition path
|
* @param partitionPath Partition path
|
||||||
*/
|
*/
|
||||||
private void createMarkerFile(String partitionPath, String dataFileName) {
|
private void createMarkerFile(String partitionPath, String dataFileName) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, instantTime);
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(writeConfig.getMarkersType(), table, instantTime);
|
||||||
markerFiles.create(partitionPath, dataFileName, IOType.CREATE);
|
writeMarkers.create(partitionPath, dataFileName, IOType.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getWriteToken() {
|
private String getWriteToken() {
|
||||||
|
|||||||
@@ -32,9 +32,11 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieRollbackException;
|
import org.apache.hudi.exception.HoodieRollbackException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@@ -52,17 +54,17 @@ public class FlinkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> ext
|
|||||||
@Override
|
@Override
|
||||||
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
||||||
try {
|
try {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp());
|
||||||
List<HoodieRollbackStat> rollbackStats = context.map(markerFiles.allMarkerFilePaths(), markerFilePath -> {
|
List<HoodieRollbackStat> rollbackStats = context.map(new ArrayList<>(writeMarkers.allMarkerFilePaths()), markerFilePath -> {
|
||||||
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
|
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
|
||||||
IOType type = IOType.valueOf(typeStr);
|
IOType type = IOType.valueOf(typeStr);
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case MERGE:
|
case MERGE:
|
||||||
return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
return undoMerge(WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
case APPEND:
|
case APPEND:
|
||||||
return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
|
return undoAppend(WriteMarkers.stripMarkerSuffix(markerFilePath), instantToRollback);
|
||||||
case CREATE:
|
case CREATE:
|
||||||
return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
return undoCreate(WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
default:
|
default:
|
||||||
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,7 +24,8 @@ 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.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieFlinkTable;
|
import org.apache.hudi.table.HoodieFlinkTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@@ -41,9 +42,9 @@ public class OneToZeroDowngradeHandler implements DowngradeHandler {
|
|||||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||||
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
|
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
|
||||||
for (HoodieInstant commitInstant : commits) {
|
for (HoodieInstant commitInstant : commits) {
|
||||||
// delete existing marker files
|
// delete existing markers
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, commitInstant.getTimestamp());
|
||||||
markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
writeMarkers.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -32,10 +32,12 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieRollbackException;
|
import org.apache.hudi.exception.HoodieRollbackException;
|
||||||
import org.apache.hudi.table.HoodieFlinkTable;
|
import org.apache.hudi.table.HoodieFlinkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
|
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
|
||||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
|
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
|
||||||
import org.apache.hudi.table.action.rollback.RollbackUtils;
|
import org.apache.hudi.table.action.rollback.RollbackUtils;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
import org.apache.hudi.table.marker.MarkerType;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -60,25 +62,27 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
|||||||
commits.remove(instantTime);
|
commits.remove(instantTime);
|
||||||
}
|
}
|
||||||
for (String commit : commits) {
|
for (String commit : commits) {
|
||||||
// for every pending commit, delete old marker files and re-create marker files in new format
|
// for every pending commit, delete old markers and re-create markers in new format
|
||||||
recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism());
|
recreateMarkers(commit, table, context, config.getMarkersType(), config.getMarkersDeleteParallelism());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Recreate marker files in new format.
|
* Recreate markers in new format.
|
||||||
* Step1: Delete existing marker files
|
* Step1: Delete existing markers
|
||||||
* Step2: Collect all rollback file info.
|
* Step2: Collect all rollback file info.
|
||||||
* Step3: recreate marker files for all interested files.
|
* Step3: recreate markers for all interested files.
|
||||||
*
|
*
|
||||||
* @param commitInstantTime instant of interest for which marker files need to be recreated.
|
* @param commitInstantTime instant of interest for which markers need to be recreated.
|
||||||
* @param table instance of {@link HoodieFlinkTable} to use
|
* @param table instance of {@link HoodieFlinkTable} to use
|
||||||
* @param context instance of {@link HoodieEngineContext} to use
|
* @param context instance of {@link HoodieEngineContext} to use
|
||||||
|
* @param markerType marker type to use
|
||||||
* @throws HoodieRollbackException on any exception during upgrade.
|
* @throws HoodieRollbackException on any exception during upgrade.
|
||||||
*/
|
*/
|
||||||
private static void recreateMarkerFiles(final String commitInstantTime,
|
private static void recreateMarkers(final String commitInstantTime,
|
||||||
HoodieFlinkTable table,
|
HoodieFlinkTable table,
|
||||||
HoodieEngineContext context,
|
HoodieEngineContext context,
|
||||||
|
MarkerType markerType,
|
||||||
int parallelism) throws HoodieRollbackException {
|
int parallelism) throws HoodieRollbackException {
|
||||||
try {
|
try {
|
||||||
// fetch hoodie instant
|
// fetch hoodie instant
|
||||||
@@ -86,9 +90,9 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
|||||||
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
||||||
.findFirst());
|
.findFirst());
|
||||||
if (commitInstantOpt.isPresent()) {
|
if (commitInstantOpt.isPresent()) {
|
||||||
// delete existing marker files
|
// delete existing markers
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime);
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(markerType, table, commitInstantTime);
|
||||||
markerFiles.quietDeleteMarkerDir(context, parallelism);
|
writeMarkers.quietDeleteMarkerDir(context, parallelism);
|
||||||
|
|
||||||
// generate rollback stats
|
// generate rollback stats
|
||||||
List<ListingBasedRollbackRequest> rollbackRequests;
|
List<ListingBasedRollbackRequest> rollbackRequests;
|
||||||
@@ -100,15 +104,15 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
|||||||
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
|
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
|
||||||
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
|
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
|
||||||
|
|
||||||
// recreate marker files adhering to marker based rollback
|
// recreate markers adhering to marker based rollback
|
||||||
for (HoodieRollbackStat rollbackStat : rollbackStats) {
|
for (HoodieRollbackStat rollbackStat : rollbackStats) {
|
||||||
for (String path : rollbackStat.getSuccessDeleteFiles()) {
|
for (String path : rollbackStat.getSuccessDeleteFiles()) {
|
||||||
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
|
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
|
||||||
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
|
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
|
||||||
markerFiles.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
|
writeMarkers.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
|
||||||
}
|
}
|
||||||
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
|
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
|
||||||
markerFiles.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
|
writeMarkers.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -30,8 +30,10 @@ import org.apache.hudi.common.util.collection.Pair;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieRollbackException;
|
import org.apache.hudi.exception.HoodieRollbackException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
@@ -47,17 +49,18 @@ public class JavaMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> exte
|
|||||||
@Override
|
@Override
|
||||||
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
||||||
try {
|
try {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
|
WriteMarkers writeMarkers =
|
||||||
List<HoodieRollbackStat> rollbackStats = context.map(markerFiles.allMarkerFilePaths(), markerFilePath -> {
|
WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp());
|
||||||
|
List<HoodieRollbackStat> rollbackStats = context.map(new ArrayList<>(writeMarkers.allMarkerFilePaths()), markerFilePath -> {
|
||||||
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
|
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
|
||||||
IOType type = IOType.valueOf(typeStr);
|
IOType type = IOType.valueOf(typeStr);
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case MERGE:
|
case MERGE:
|
||||||
return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
return undoMerge(WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
case APPEND:
|
case APPEND:
|
||||||
return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
|
return undoAppend(WriteMarkers.stripMarkerSuffix(markerFilePath), instantToRollback);
|
||||||
case CREATE:
|
case CREATE:
|
||||||
return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
return undoCreate(WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
default:
|
default:
|
||||||
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -51,9 +51,9 @@ import org.apache.hudi.metrics.DistributedRegistry;
|
|||||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||||
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.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
|
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade;
|
import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade;
|
||||||
import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
|
import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
|
||||||
|
|
||||||
@@ -375,7 +375,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
|||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
|
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
|
||||||
}
|
}
|
||||||
new MarkerFiles(table, clusteringCommitTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
WriteMarkersFactory.get(config.getMarkersType(), table, clusteringCommitTime)
|
||||||
|
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||||
if (clusteringTimer != null) {
|
if (clusteringTimer != null) {
|
||||||
long durationInMs = metrics.getDurationInMs(clusteringTimer.stop());
|
long durationInMs = metrics.getDurationInMs(clusteringTimer.stop());
|
||||||
try {
|
try {
|
||||||
|
|||||||
@@ -31,7 +31,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.exception.HoodieInsertException;
|
import org.apache.hudi.exception.HoodieInsertException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -187,8 +187,8 @@ public class HoodieRowCreateHandle implements Serializable {
|
|||||||
* @param partitionPath Partition path
|
* @param partitionPath Partition path
|
||||||
*/
|
*/
|
||||||
private void createMarkerFile(String partitionPath, String dataFileName) {
|
private void createMarkerFile(String partitionPath, String dataFileName) {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, instantTime);
|
WriteMarkersFactory.get(writeConfig.getMarkersType(), table, instantTime)
|
||||||
markerFiles.create(partitionPath, dataFileName, IOType.CREATE);
|
.create(partitionPath, dataFileName, IOType.CREATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getWriteToken() {
|
private String getWriteToken() {
|
||||||
|
|||||||
@@ -33,13 +33,15 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
|||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieRollbackException;
|
import org.apache.hudi.exception.HoodieRollbackException;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@@ -56,8 +58,8 @@ public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> ext
|
|||||||
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
||||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||||
try {
|
try {
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp());
|
||||||
List<String> markerFilePaths = markerFiles.allMarkerFilePaths();
|
List<String> markerFilePaths = new ArrayList<>(writeMarkers.allMarkerFilePaths());
|
||||||
int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1);
|
int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1);
|
||||||
jsc.setJobGroup(this.getClass().getSimpleName(), "Rolling back using marker files");
|
jsc.setJobGroup(this.getClass().getSimpleName(), "Rolling back using marker files");
|
||||||
return jsc.parallelize(markerFilePaths, parallelism)
|
return jsc.parallelize(markerFilePaths, parallelism)
|
||||||
@@ -66,11 +68,11 @@ public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> ext
|
|||||||
IOType type = IOType.valueOf(typeStr);
|
IOType type = IOType.valueOf(typeStr);
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case MERGE:
|
case MERGE:
|
||||||
return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
return undoMerge(WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
case APPEND:
|
case APPEND:
|
||||||
return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
|
return undoAppend(WriteMarkers.stripMarkerSuffix(markerFilePath), instantToRollback);
|
||||||
case CREATE:
|
case CREATE:
|
||||||
return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
return undoCreate(WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
default:
|
default:
|
||||||
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -23,7 +23,8 @@ 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.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieSparkTable;
|
import org.apache.hudi.table.HoodieSparkTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@@ -40,9 +41,9 @@ public class OneToZeroDowngradeHandler implements DowngradeHandler {
|
|||||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||||
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
|
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
|
||||||
for (HoodieInstant commitInstant : commits) {
|
for (HoodieInstant commitInstant : commits) {
|
||||||
// delete existing marker files
|
// delete existing markers
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, commitInstant.getTimestamp());
|
||||||
markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
writeMarkers.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -31,13 +31,15 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
|||||||
import org.apache.hudi.exception.HoodieRollbackException;
|
import org.apache.hudi.exception.HoodieRollbackException;
|
||||||
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.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
|
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
|
||||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
|
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hudi.table.action.rollback.RollbackUtils;
|
import org.apache.hudi.table.action.rollback.RollbackUtils;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
import org.apache.hudi.table.marker.MarkerType;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@@ -59,25 +61,27 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
|||||||
commits.remove(instantTime);
|
commits.remove(instantTime);
|
||||||
}
|
}
|
||||||
for (String commit : commits) {
|
for (String commit : commits) {
|
||||||
// for every pending commit, delete old marker files and re-create marker files in new format
|
// for every pending commit, delete old markers and re-create markers in new format
|
||||||
recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism());
|
recreateMarkers(commit, table, context, config.getMarkersType(), config.getMarkersDeleteParallelism());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Recreate marker files in new format.
|
* Recreate markers in new format.
|
||||||
* Step1: Delete existing marker files
|
* Step1: Delete existing markers
|
||||||
* Step2: Collect all rollback file info.
|
* Step2: Collect all rollback file info.
|
||||||
* Step3: recreate marker files for all interested files.
|
* Step3: recreate markers for all interested files.
|
||||||
*
|
*
|
||||||
* @param commitInstantTime instant of interest for which marker files need to be recreated.
|
* @param commitInstantTime instant of interest for which markers need to be recreated.
|
||||||
* @param table instance of {@link HoodieSparkTable} to use
|
* @param table instance of {@link HoodieSparkTable} to use
|
||||||
* @param context instance of {@link HoodieEngineContext} to use
|
* @param context instance of {@link HoodieEngineContext} to use
|
||||||
|
* @param markerType marker type to use
|
||||||
* @throws HoodieRollbackException on any exception during upgrade.
|
* @throws HoodieRollbackException on any exception during upgrade.
|
||||||
*/
|
*/
|
||||||
private static void recreateMarkerFiles(final String commitInstantTime,
|
private static void recreateMarkers(final String commitInstantTime,
|
||||||
HoodieSparkTable table,
|
HoodieSparkTable table,
|
||||||
HoodieEngineContext context,
|
HoodieEngineContext context,
|
||||||
|
MarkerType markerType,
|
||||||
int parallelism) throws HoodieRollbackException {
|
int parallelism) throws HoodieRollbackException {
|
||||||
try {
|
try {
|
||||||
// fetch hoodie instant
|
// fetch hoodie instant
|
||||||
@@ -85,9 +89,9 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
|||||||
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
||||||
.findFirst());
|
.findFirst());
|
||||||
if (commitInstantOpt.isPresent()) {
|
if (commitInstantOpt.isPresent()) {
|
||||||
// delete existing marker files
|
// delete existing markers
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime);
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(markerType, table, commitInstantTime);
|
||||||
markerFiles.quietDeleteMarkerDir(context, parallelism);
|
writeMarkers.quietDeleteMarkerDir(context, parallelism);
|
||||||
|
|
||||||
// generate rollback stats
|
// generate rollback stats
|
||||||
List<ListingBasedRollbackRequest> rollbackRequests;
|
List<ListingBasedRollbackRequest> rollbackRequests;
|
||||||
@@ -99,15 +103,15 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
|||||||
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
|
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
|
||||||
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
|
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
|
||||||
|
|
||||||
// recreate marker files adhering to marker based rollback
|
// recreate markers adhering to marker based rollback
|
||||||
for (HoodieRollbackStat rollbackStat : rollbackStats) {
|
for (HoodieRollbackStat rollbackStat : rollbackStats) {
|
||||||
for (String path : rollbackStat.getSuccessDeleteFiles()) {
|
for (String path : rollbackStat.getSuccessDeleteFiles()) {
|
||||||
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
|
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
|
||||||
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
|
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
|
||||||
markerFiles.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
|
writeMarkers.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
|
||||||
}
|
}
|
||||||
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
|
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
|
||||||
markerFiles.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
|
writeMarkers.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -87,9 +87,9 @@ import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
|
|||||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||||
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.MarkerFiles;
|
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
import org.apache.hudi.table.action.commit.SparkWriteHelper;
|
import org.apache.hudi.table.action.commit.SparkWriteHelper;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||||
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
|
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
|
||||||
@@ -2220,11 +2220,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
|||||||
path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN)))
|
path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN)))
|
||||||
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
|
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
|
||||||
|
|
||||||
Path markerFilePath = new MarkerFiles(fs, basePath, metaClient.getMarkerFolderPath(instantTime), instantTime)
|
Option<Path> markerFilePath = WriteMarkersFactory.get(
|
||||||
|
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);
|
||||||
LOG.info("Created a dummy marker path=" + markerFilePath);
|
LOG.info("Created a dummy marker path=" + markerFilePath.get());
|
||||||
|
|
||||||
if (!enableOptimisticConsistencyGuard) {
|
if (!enableOptimisticConsistencyGuard) {
|
||||||
Exception e = assertThrows(HoodieCommitException.class, () -> {
|
Exception e = assertThrows(HoodieCommitException.class, () -> {
|
||||||
@@ -2235,7 +2236,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
|||||||
// with optimistic CG, commit should succeed
|
// with optimistic CG, commit should succeed
|
||||||
client.commit(instantTime, result);
|
client.commit(instantTime, result);
|
||||||
}
|
}
|
||||||
return Pair.of(markerFilePath, result);
|
return Pair.of(markerFilePath.get(), result);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
|
|||||||
@@ -27,7 +27,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
|||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
@@ -213,6 +214,6 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT
|
|||||||
String.format("%s:%s/%s", this.fs.getScheme(), basePath, rollbackMetadata.get(DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().get(0)));
|
String.format("%s:%s/%s", this.fs.getScheme(), basePath, rollbackMetadata.get(DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().get(0)));
|
||||||
}
|
}
|
||||||
|
|
||||||
assertFalse(new MarkerFiles(table, commitInstant.getTimestamp()).doesMarkerDirExist());
|
assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, commitInstant.getTimestamp()).doesMarkerDirExist());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -29,7 +29,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
|||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.Assertions;
|
import org.junit.jupiter.api.Assertions;
|
||||||
@@ -138,7 +138,7 @@ public class TestMergeOnReadRollbackActionExecutor extends HoodieClientRollbackT
|
|||||||
secondPartitionRollBackLogFiles.removeAll(secondPartitionCommit2LogFiles);
|
secondPartitionRollBackLogFiles.removeAll(secondPartitionCommit2LogFiles);
|
||||||
assertEquals(1, secondPartitionRollBackLogFiles.size());
|
assertEquals(1, secondPartitionRollBackLogFiles.size());
|
||||||
|
|
||||||
assertFalse(new MarkerFiles(table, "002").doesMarkerDirExist());
|
assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, "002").doesMarkerDirExist());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|||||||
@@ -0,0 +1,74 @@
|
|||||||
|
/*
|
||||||
|
* 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.client.common.HoodieSparkEngineContext;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.testutils.FileSystemTestUtils;
|
||||||
|
import org.apache.hudi.common.util.CollectionUtils;
|
||||||
|
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
||||||
|
|
||||||
|
public class TestDirectWriteMarkers extends TestWriteMarkersBase {
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setup() throws IOException {
|
||||||
|
initPath();
|
||||||
|
initMetaClient();
|
||||||
|
this.jsc = new JavaSparkContext(
|
||||||
|
HoodieClientTestUtils.getSparkConfForTest(TestDirectWriteMarkers.class.getName()));
|
||||||
|
this.context = new HoodieSparkEngineContext(jsc);
|
||||||
|
this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf());
|
||||||
|
this.markerFolderPath = new Path(metaClient.getMarkerFolderPath("000"));
|
||||||
|
this.writeMarkers = new DirectWriteMarkers(
|
||||||
|
fs, metaClient.getBasePath(), markerFolderPath.toString(), "000");
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void cleanup() {
|
||||||
|
jsc.stop();
|
||||||
|
context = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void verifyMarkersInFileSystem() throws IOException {
|
||||||
|
List<FileStatus> markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
|
||||||
|
.stream().filter(status -> status.getPath().getName().contains(".marker"))
|
||||||
|
.sorted().collect(Collectors.toList());
|
||||||
|
assertEquals(3, markerFiles.size());
|
||||||
|
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||||
|
"file:" + markerFolderPath.toString() + "/2020/06/01/file1.marker.MERGE",
|
||||||
|
"file:" + markerFolderPath.toString() + "/2020/06/02/file2.marker.APPEND",
|
||||||
|
"file:" + markerFolderPath.toString() + "/2020/06/03/file3.marker.CREATE"),
|
||||||
|
markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList())
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,141 @@
|
|||||||
|
/*
|
||||||
|
* 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.client.common.HoodieSparkEngineContext;
|
||||||
|
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||||
|
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||||
|
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||||
|
import org.apache.hudi.common.testutils.FileSystemTestUtils;
|
||||||
|
import org.apache.hudi.common.util.CollectionUtils;
|
||||||
|
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
import org.apache.hudi.timeline.service.handlers.marker.MarkerDirState;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
|
||||||
|
import java.io.BufferedReader;
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStreamReader;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
||||||
|
|
||||||
|
public class TestTimelineServerBasedWriteMarkers extends TestWriteMarkersBase {
|
||||||
|
TimelineService timelineService;
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setup() throws IOException {
|
||||||
|
initPath();
|
||||||
|
initMetaClient();
|
||||||
|
this.jsc = new JavaSparkContext(
|
||||||
|
HoodieClientTestUtils.getSparkConfForTest(TestTimelineServerBasedWriteMarkers.class.getName()));
|
||||||
|
this.context = new HoodieSparkEngineContext(jsc);
|
||||||
|
this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf());
|
||||||
|
this.markerFolderPath = new Path(metaClient.getMarkerFolderPath("000"));
|
||||||
|
|
||||||
|
FileSystemViewStorageConfig storageConf =
|
||||||
|
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).build();
|
||||||
|
HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().build();
|
||||||
|
HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
|
||||||
|
|
||||||
|
try {
|
||||||
|
timelineService = new TimelineService(localEngineContext, new Configuration(),
|
||||||
|
TimelineService.Config.builder().serverPort(0).enableMarkerRequests(true).build(),
|
||||||
|
FileSystem.get(new Configuration()),
|
||||||
|
FileSystemViewManager.createViewManager(
|
||||||
|
localEngineContext, metadataConfig, storageConf, HoodieCommonConfig.newBuilder().build()));
|
||||||
|
timelineService.startService();
|
||||||
|
} catch (Exception ex) {
|
||||||
|
throw new RuntimeException(ex);
|
||||||
|
}
|
||||||
|
this.writeMarkers = new TimelineServerBasedWriteMarkers(
|
||||||
|
metaClient.getBasePath(), markerFolderPath.toString(), "000", "localhost", timelineService.getServerPort(), 300);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void cleanup() {
|
||||||
|
if (timelineService != null) {
|
||||||
|
timelineService.close();
|
||||||
|
}
|
||||||
|
jsc.stop();
|
||||||
|
context = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void verifyMarkersInFileSystem() throws IOException {
|
||||||
|
List<String> allMarkers = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
|
||||||
|
.stream().filter(status -> status.getPath().getName().contains(MarkerDirState.MARKERS_FILENAME_PREFIX))
|
||||||
|
.flatMap(status -> {
|
||||||
|
// Read all markers stored in each marker file maintained by the timeline service
|
||||||
|
FSDataInputStream fsDataInputStream = null;
|
||||||
|
BufferedReader bufferedReader = null;
|
||||||
|
List<String> markers = null;
|
||||||
|
try {
|
||||||
|
fsDataInputStream = fs.open(status.getPath());
|
||||||
|
bufferedReader = new BufferedReader(new InputStreamReader(fsDataInputStream, StandardCharsets.UTF_8));
|
||||||
|
markers = bufferedReader.lines().collect(Collectors.toList());
|
||||||
|
} catch (IOException e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
} finally {
|
||||||
|
closeQuietly(bufferedReader);
|
||||||
|
closeQuietly(fsDataInputStream);
|
||||||
|
}
|
||||||
|
return markers.stream();
|
||||||
|
})
|
||||||
|
.sorted()
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
assertEquals(3, allMarkers.size());
|
||||||
|
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||||
|
"2020/06/01/file1.marker.MERGE",
|
||||||
|
"2020/06/02/file2.marker.APPEND",
|
||||||
|
"2020/06/03/file3.marker.CREATE"),
|
||||||
|
allMarkers);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Closes {@code Closeable} quietly.
|
||||||
|
*
|
||||||
|
* @param closeable {@code Closeable} to close
|
||||||
|
*/
|
||||||
|
private void closeQuietly(Closeable closeable) {
|
||||||
|
if (closeable == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
closeable.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
// Ignore
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -16,7 +16,7 @@
|
|||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hudi.table;
|
package org.apache.hudi.table.marker;
|
||||||
|
|
||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
@@ -25,18 +25,13 @@ import org.apache.hudi.common.testutils.FileSystemTestUtils;
|
|||||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||||
import org.apache.hudi.common.util.CollectionUtils;
|
import org.apache.hudi.common.util.CollectionUtils;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
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.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.junit.jupiter.api.AfterEach;
|
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
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;
|
||||||
@@ -44,35 +39,18 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
|
|||||||
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
public class TestMarkerFiles extends HoodieCommonTestHarness {
|
public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
|
||||||
|
|
||||||
private MarkerFiles markerFiles;
|
protected WriteMarkers writeMarkers;
|
||||||
private FileSystem fs;
|
protected FileSystem fs;
|
||||||
private Path markerFolderPath;
|
protected Path markerFolderPath;
|
||||||
private JavaSparkContext jsc;
|
protected JavaSparkContext jsc;
|
||||||
private HoodieSparkEngineContext context;
|
protected HoodieSparkEngineContext context;
|
||||||
|
|
||||||
@BeforeEach
|
private void createSomeMarkers() {
|
||||||
public void setup() throws IOException {
|
writeMarkers.create("2020/06/01", "file1", IOType.MERGE);
|
||||||
initPath();
|
writeMarkers.create("2020/06/02", "file2", IOType.APPEND);
|
||||||
initMetaClient();
|
writeMarkers.create("2020/06/03", "file3", IOType.CREATE);
|
||||||
this.jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(TestMarkerFiles.class.getName()));
|
|
||||||
this.context = new HoodieSparkEngineContext(jsc);
|
|
||||||
this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf());
|
|
||||||
this.markerFolderPath = new Path(metaClient.getMarkerFolderPath("000"));
|
|
||||||
this.markerFiles = new MarkerFiles(fs, metaClient.getBasePath(), markerFolderPath.toString(), "000");
|
|
||||||
}
|
|
||||||
|
|
||||||
@AfterEach
|
|
||||||
public void cleanup() {
|
|
||||||
jsc.stop();
|
|
||||||
context = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void createSomeMarkerFiles() {
|
|
||||||
markerFiles.create("2020/06/01", "file1", IOType.MERGE);
|
|
||||||
markerFiles.create("2020/06/02", "file2", IOType.APPEND);
|
|
||||||
markerFiles.create("2020/06/03", "file3", IOType.CREATE);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createInvalidFile(String partitionPath, String invalidFileName) {
|
private void createInvalidFile(String partitionPath, String invalidFileName) {
|
||||||
@@ -85,48 +63,41 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
abstract void verifyMarkersInFileSystem() throws IOException;
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreation() throws Exception {
|
public void testCreation() throws Exception {
|
||||||
// when
|
// when
|
||||||
createSomeMarkerFiles();
|
createSomeMarkers();
|
||||||
|
|
||||||
// then
|
// then
|
||||||
assertTrue(fs.exists(markerFolderPath));
|
assertTrue(fs.exists(markerFolderPath));
|
||||||
List<FileStatus> markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
|
verifyMarkersInFileSystem();
|
||||||
.stream().filter(status -> status.getPath().getName().contains(".marker"))
|
|
||||||
.sorted().collect(Collectors.toList());
|
|
||||||
assertEquals(3, markerFiles.size());
|
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
|
||||||
"file:" + markerFolderPath.toString() + "/2020/06/01/file1.marker.MERGE",
|
|
||||||
"file:" + markerFolderPath.toString() + "/2020/06/02/file2.marker.APPEND",
|
|
||||||
"file:" + markerFolderPath.toString() + "/2020/06/03/file3.marker.CREATE"),
|
|
||||||
markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList())
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDeletionWhenMarkerDirExists() throws IOException {
|
public void testDeletionWhenMarkerDirExists() throws IOException {
|
||||||
//when
|
//when
|
||||||
markerFiles.create("2020/06/01", "file1", IOType.MERGE);
|
writeMarkers.create("2020/06/01", "file1", IOType.MERGE);
|
||||||
|
|
||||||
// then
|
// then
|
||||||
assertTrue(markerFiles.doesMarkerDirExist());
|
assertTrue(writeMarkers.doesMarkerDirExist());
|
||||||
assertTrue(markerFiles.deleteMarkerDir(context, 2));
|
assertTrue(writeMarkers.deleteMarkerDir(context, 2));
|
||||||
assertFalse(markerFiles.doesMarkerDirExist());
|
assertFalse(writeMarkers.doesMarkerDirExist());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDeletionWhenMarkerDirNotExists() throws IOException {
|
public void testDeletionWhenMarkerDirNotExists() throws IOException {
|
||||||
// then
|
// then
|
||||||
assertFalse(markerFiles.doesMarkerDirExist());
|
assertFalse(writeMarkers.doesMarkerDirExist());
|
||||||
assertTrue(markerFiles.allMarkerFilePaths().isEmpty());
|
assertTrue(writeMarkers.allMarkerFilePaths().isEmpty());
|
||||||
assertFalse(markerFiles.deleteMarkerDir(context, 2));
|
assertFalse(writeMarkers.deleteMarkerDir(context, 2));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDataPathsWhenCreatingOrMerging() throws IOException {
|
public void testDataPathsWhenCreatingOrMerging() throws IOException {
|
||||||
// add markfiles
|
// add markfiles
|
||||||
createSomeMarkerFiles();
|
createSomeMarkers();
|
||||||
// add invalid file
|
// add invalid file
|
||||||
createInvalidFile("2020/06/01", "invalid_file3");
|
createInvalidFile("2020/06/01", "invalid_file3");
|
||||||
int fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).size();
|
int fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).size();
|
||||||
@@ -135,19 +106,19 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
|||||||
// then
|
// then
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||||
"2020/06/01/file1", "2020/06/03/file3"),
|
"2020/06/01/file1", "2020/06/03/file3"),
|
||||||
markerFiles.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
|
writeMarkers.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllMarkerPaths() throws IOException {
|
public void testAllMarkerPaths() throws IOException {
|
||||||
// given
|
// given
|
||||||
createSomeMarkerFiles();
|
createSomeMarkers();
|
||||||
|
|
||||||
// then
|
// then
|
||||||
assertIterableEquals(CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
|
assertIterableEquals(CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
|
||||||
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE"),
|
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE"),
|
||||||
markerFiles.allMarkerFilePaths().stream().sorted().collect(Collectors.toList())
|
writeMarkers.allMarkerFilePaths().stream().sorted().collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -158,6 +129,6 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
|||||||
final String markerFilePath = pathPrefix + ".marker.APPEND";
|
final String markerFilePath = pathPrefix + ".marker.APPEND";
|
||||||
|
|
||||||
// when-then
|
// when-then
|
||||||
assertEquals(pathPrefix, MarkerFiles.stripMarkerSuffix(markerFilePath));
|
assertEquals(pathPrefix, WriteMarkers.stripMarkerSuffix(markerFilePath));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -39,7 +39,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
|
|||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.MarkerFiles;
|
import org.apache.hudi.table.marker.WriteMarkers;
|
||||||
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
import org.apache.hudi.testutils.Assertions;
|
import org.apache.hudi.testutils.Assertions;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||||
@@ -58,6 +59,7 @@ import java.util.ArrayList;
|
|||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
@@ -126,8 +128,9 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
|
|||||||
HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
|
HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
|
||||||
|
|
||||||
// delete one of the marker files in 2nd commit if need be.
|
// delete one of the marker files in 2nd commit if need be.
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
WriteMarkers writeMarkers =
|
||||||
List<String> markerPaths = markerFiles.allMarkerFilePaths();
|
WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.getTimestamp());
|
||||||
|
List<String> markerPaths = new ArrayList<>(writeMarkers.allMarkerFilePaths());
|
||||||
if (deletePartialMarkerFiles) {
|
if (deletePartialMarkerFiles) {
|
||||||
String toDeleteMarkerFile = markerPaths.get(0);
|
String toDeleteMarkerFile = markerPaths.get(0);
|
||||||
table.getMetaClient().getFs().delete(new Path(table.getMetaClient().getTempFolderPath() + "/" + commitInstant.getTimestamp() + "/" + toDeleteMarkerFile));
|
table.getMetaClient().getFs().delete(new Path(table.getMetaClient().getTempFolderPath() + "/" + commitInstant.getTimestamp() + "/" + toDeleteMarkerFile));
|
||||||
@@ -182,8 +185,8 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
|
|||||||
HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
|
HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
|
||||||
|
|
||||||
// delete one of the marker files in 2nd commit if need be.
|
// delete one of the marker files in 2nd commit if need be.
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.getTimestamp());
|
||||||
List<String> markerPaths = markerFiles.allMarkerFilePaths();
|
List<String> markerPaths = new ArrayList<>(writeMarkers.allMarkerFilePaths());
|
||||||
if (deletePartialMarkerFiles) {
|
if (deletePartialMarkerFiles) {
|
||||||
String toDeleteMarkerFile = markerPaths.get(0);
|
String toDeleteMarkerFile = markerPaths.get(0);
|
||||||
table.getMetaClient().getFs().delete(new Path(table.getMetaClient().getTempFolderPath() + "/" + commitInstant.getTimestamp() + "/" + toDeleteMarkerFile));
|
table.getMetaClient().getFs().delete(new Path(table.getMetaClient().getTempFolderPath() + "/" + commitInstant.getTimestamp() + "/" + toDeleteMarkerFile));
|
||||||
@@ -212,21 +215,21 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
|
|||||||
|
|
||||||
private void assertMarkerFilesForDowngrade(HoodieTable table, HoodieInstant commitInstant) throws IOException {
|
private void assertMarkerFilesForDowngrade(HoodieTable table, HoodieInstant commitInstant) throws IOException {
|
||||||
// Verify recreated marker files are as expected
|
// Verify recreated marker files are as expected
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.getTimestamp());
|
||||||
assertFalse(markerFiles.doesMarkerDirExist());
|
assertFalse(writeMarkers.doesMarkerDirExist());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertMarkerFilesForUpgrade(HoodieTable table, HoodieInstant commitInstant, List<FileSlice> firstPartitionCommit2FileSlices,
|
private void assertMarkerFilesForUpgrade(HoodieTable table, HoodieInstant commitInstant, List<FileSlice> firstPartitionCommit2FileSlices,
|
||||||
List<FileSlice> secondPartitionCommit2FileSlices) throws IOException {
|
List<FileSlice> secondPartitionCommit2FileSlices) throws IOException {
|
||||||
// Verify recreated marker files are as expected
|
// Verify recreated marker files are as expected
|
||||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.getTimestamp());
|
||||||
assertTrue(markerFiles.doesMarkerDirExist());
|
assertTrue(writeMarkers.doesMarkerDirExist());
|
||||||
List<String> files = markerFiles.allMarkerFilePaths();
|
Set<String> files = writeMarkers.allMarkerFilePaths();
|
||||||
|
|
||||||
assertEquals(2, files.size());
|
assertEquals(2, files.size());
|
||||||
List<String> actualFiles = new ArrayList<>();
|
List<String> actualFiles = new ArrayList<>();
|
||||||
for (String file : files) {
|
for (String file : files) {
|
||||||
String fileName = MarkerFiles.stripMarkerSuffix(file);
|
String fileName = WriteMarkers.stripMarkerSuffix(file);
|
||||||
actualFiles.add(fileName);
|
actualFiles.add(fileName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -0,0 +1,41 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hudi.common.table.marker;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stores URLs to timeline server for marker-related operations.
|
||||||
|
*/
|
||||||
|
public class MarkerOperation implements Serializable {
|
||||||
|
|
||||||
|
private static final String BASE_URL = "/v1/hoodie/marker";
|
||||||
|
|
||||||
|
public static final String MARKER_DIR_PATH_PARAM = "markerdirpath";
|
||||||
|
public static final String MARKER_NAME_PARAM = "markername";
|
||||||
|
|
||||||
|
// GET requests
|
||||||
|
public static final String ALL_MARKERS_URL = String.format("%s/%s", BASE_URL, "all");
|
||||||
|
public static final String CREATE_AND_MERGE_MARKERS_URL = String.format("%s/%s", BASE_URL, "create-and-merge");
|
||||||
|
public static final String MARKERS_DIR_EXISTS_URL = String.format("%s/%s", BASE_URL, "dir/exists");
|
||||||
|
|
||||||
|
// POST requests
|
||||||
|
public static final String CREATE_MARKER_URL = String.format("%s/%s", BASE_URL, "create");
|
||||||
|
public static final String DELETE_MARKER_DIR_URL = String.format("%s/%s", BASE_URL, "dir/delete");
|
||||||
|
}
|
||||||
@@ -18,7 +18,11 @@
|
|||||||
|
|
||||||
package org.apache.hudi.common.util;
|
package org.apache.hudi.common.util;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
import java.io.FileOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@@ -34,7 +38,7 @@ import java.util.Comparator;
|
|||||||
* Bunch of utility methods for working with files and byte streams.
|
* Bunch of utility methods for working with files and byte streams.
|
||||||
*/
|
*/
|
||||||
public class FileIOUtils {
|
public class FileIOUtils {
|
||||||
|
public static final Logger LOG = LogManager.getLogger(FileIOUtils.class);
|
||||||
public static final long KB = 1024;
|
public static final long KB = 1024;
|
||||||
|
|
||||||
public static void deleteDirectory(File directory) throws IOException {
|
public static void deleteDirectory(File directory) throws IOException {
|
||||||
@@ -91,4 +95,20 @@ public class FileIOUtils {
|
|||||||
out.flush();
|
out.flush();
|
||||||
out.close();
|
out.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Closes {@code Closeable} quietly.
|
||||||
|
*
|
||||||
|
* @param closeable {@code Closeable} to close
|
||||||
|
*/
|
||||||
|
public static void closeQuietly(Closeable closeable) {
|
||||||
|
if (closeable == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
closeable.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.warn("IOException during close", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -29,4 +29,7 @@ public class HoodieRemoteException extends RuntimeException {
|
|||||||
super(t.getMessage(), t);
|
super(t.getMessage(), t);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public HoodieRemoteException(String message, IOException t) {
|
||||||
|
super(message + "\n" + t.getMessage(), t);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,7 +18,9 @@
|
|||||||
|
|
||||||
package org.apache.hudi.timeline.service;
|
package org.apache.hudi.timeline.service;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.metrics.Registry;
|
import org.apache.hudi.common.metrics.Registry;
|
||||||
|
import org.apache.hudi.common.table.marker.MarkerOperation;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.dto.BaseFileDTO;
|
import org.apache.hudi.common.table.timeline.dto.BaseFileDTO;
|
||||||
import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO;
|
import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO;
|
||||||
@@ -35,6 +37,7 @@ import org.apache.hudi.common.util.ValidationUtils;
|
|||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.timeline.service.handlers.BaseFileHandler;
|
import org.apache.hudi.timeline.service.handlers.BaseFileHandler;
|
||||||
import org.apache.hudi.timeline.service.handlers.FileSliceHandler;
|
import org.apache.hudi.timeline.service.handlers.FileSliceHandler;
|
||||||
|
import org.apache.hudi.timeline.service.handlers.MarkerHandler;
|
||||||
import org.apache.hudi.timeline.service.handlers.TimelineHandler;
|
import org.apache.hudi.timeline.service.handlers.TimelineHandler;
|
||||||
|
|
||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
@@ -43,6 +46,7 @@ import io.javalin.Context;
|
|||||||
import io.javalin.Handler;
|
import io.javalin.Handler;
|
||||||
import io.javalin.Javalin;
|
import io.javalin.Javalin;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.jetbrains.annotations.NotNull;
|
import org.jetbrains.annotations.NotNull;
|
||||||
@@ -50,6 +54,7 @@ import org.jetbrains.annotations.NotNull;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
@@ -63,35 +68,49 @@ public class RequestHandler {
|
|||||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
private static final Logger LOG = LogManager.getLogger(RequestHandler.class);
|
private static final Logger LOG = LogManager.getLogger(RequestHandler.class);
|
||||||
|
|
||||||
|
private final TimelineService.Config timelineServiceConfig;
|
||||||
private final FileSystemViewManager viewManager;
|
private final FileSystemViewManager viewManager;
|
||||||
private final Javalin app;
|
private final Javalin app;
|
||||||
private final TimelineHandler instantHandler;
|
private final TimelineHandler instantHandler;
|
||||||
private final FileSliceHandler sliceHandler;
|
private final FileSliceHandler sliceHandler;
|
||||||
private final BaseFileHandler dataFileHandler;
|
private final BaseFileHandler dataFileHandler;
|
||||||
|
private final MarkerHandler markerHandler;
|
||||||
private Registry metricsRegistry = Registry.getRegistry("TimelineService");
|
private Registry metricsRegistry = Registry.getRegistry("TimelineService");
|
||||||
private ScheduledExecutorService asyncResultService = Executors.newSingleThreadScheduledExecutor();
|
private ScheduledExecutorService asyncResultService = Executors.newSingleThreadScheduledExecutor();
|
||||||
private final boolean useAsync;
|
|
||||||
|
|
||||||
public RequestHandler(Javalin app, Configuration conf, FileSystemViewManager viewManager, boolean useAsync) throws IOException {
|
public RequestHandler(Javalin app, Configuration conf, TimelineService.Config timelineServiceConfig,
|
||||||
|
HoodieEngineContext hoodieEngineContext, FileSystem fileSystem,
|
||||||
|
FileSystemViewManager viewManager) throws IOException {
|
||||||
|
this.timelineServiceConfig = timelineServiceConfig;
|
||||||
this.viewManager = viewManager;
|
this.viewManager = viewManager;
|
||||||
this.app = app;
|
this.app = app;
|
||||||
this.instantHandler = new TimelineHandler(conf, viewManager);
|
this.instantHandler = new TimelineHandler(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
this.sliceHandler = new FileSliceHandler(conf, viewManager);
|
this.sliceHandler = new FileSliceHandler(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
this.dataFileHandler = new BaseFileHandler(conf, viewManager);
|
this.dataFileHandler = new BaseFileHandler(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
this.useAsync = useAsync;
|
if (timelineServiceConfig.enableMarkerRequests) {
|
||||||
if (useAsync) {
|
this.markerHandler = new MarkerHandler(
|
||||||
|
conf, timelineServiceConfig, hoodieEngineContext, fileSystem, viewManager, metricsRegistry);
|
||||||
|
} else {
|
||||||
|
this.markerHandler = null;
|
||||||
|
}
|
||||||
|
if (timelineServiceConfig.async) {
|
||||||
asyncResultService = Executors.newSingleThreadScheduledExecutor();
|
asyncResultService = Executors.newSingleThreadScheduledExecutor();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public RequestHandler(Javalin app, Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
|
||||||
this(app, conf, viewManager, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void register() {
|
public void register() {
|
||||||
registerDataFilesAPI();
|
registerDataFilesAPI();
|
||||||
registerFileSlicesAPI();
|
registerFileSlicesAPI();
|
||||||
registerTimelineAPI();
|
registerTimelineAPI();
|
||||||
|
if (markerHandler != null) {
|
||||||
|
registerMarkerAPI();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void stop() {
|
||||||
|
if (markerHandler != null) {
|
||||||
|
markerHandler.stop();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -147,40 +166,50 @@ public class RequestHandler {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void writeValueAsString(Context ctx, Object obj) throws JsonProcessingException {
|
private void writeValueAsString(Context ctx, Object obj) throws JsonProcessingException {
|
||||||
if (useAsync) {
|
if (timelineServiceConfig.async) {
|
||||||
writeValueAsStringAsync(ctx, obj);
|
writeValueAsStringAsync(ctx, obj);
|
||||||
} else {
|
} else {
|
||||||
writeValueAsStringSync(ctx, obj);
|
writeValueAsStringSync(ctx, obj);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void writeValueAsStringSync(Context ctx, Object obj) throws JsonProcessingException {
|
/**
|
||||||
|
* Serializes the result into JSON String.
|
||||||
|
*
|
||||||
|
* @param ctx Javalin context
|
||||||
|
* @param obj object to serialize
|
||||||
|
* @param metricsRegistry {@code Registry} instance for storing metrics
|
||||||
|
* @param objectMapper JSON object mapper
|
||||||
|
* @param logger {@code Logger} instance
|
||||||
|
* @return JSON String from the input object
|
||||||
|
* @throws JsonProcessingException
|
||||||
|
*/
|
||||||
|
public static String jsonifyResult(
|
||||||
|
Context ctx, Object obj, Registry metricsRegistry, ObjectMapper objectMapper, Logger logger)
|
||||||
|
throws JsonProcessingException {
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
boolean prettyPrint = ctx.queryParam("pretty") != null;
|
boolean prettyPrint = ctx.queryParam("pretty") != null;
|
||||||
String result =
|
String result =
|
||||||
prettyPrint ? OBJECT_MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(obj) : OBJECT_MAPPER.writeValueAsString(obj);
|
prettyPrint ? objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(obj)
|
||||||
final long jsonifyTime = timer.endTimer();
|
: objectMapper.writeValueAsString(obj);
|
||||||
ctx.result(result);
|
|
||||||
metricsRegistry.add("WRITE_VALUE_CNT", 1);
|
|
||||||
metricsRegistry.add("WRITE_VALUE_TIME", jsonifyTime);
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("Jsonify TimeTaken=" + jsonifyTime);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void writeValueAsStringAsync(Context ctx, Object obj) throws JsonProcessingException {
|
|
||||||
ctx.result(CompletableFuture.supplyAsync(() -> {
|
|
||||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
|
||||||
boolean prettyPrint = ctx.queryParam("pretty") != null;
|
|
||||||
try {
|
|
||||||
String result = prettyPrint ? OBJECT_MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(obj) : OBJECT_MAPPER.writeValueAsString(obj);
|
|
||||||
final long jsonifyTime = timer.endTimer();
|
final long jsonifyTime = timer.endTimer();
|
||||||
metricsRegistry.add("WRITE_VALUE_CNT", 1);
|
metricsRegistry.add("WRITE_VALUE_CNT", 1);
|
||||||
metricsRegistry.add("WRITE_VALUE_TIME", jsonifyTime);
|
metricsRegistry.add("WRITE_VALUE_TIME", jsonifyTime);
|
||||||
if (LOG.isDebugEnabled()) {
|
if (logger.isDebugEnabled()) {
|
||||||
LOG.debug("Jsonify TimeTaken=" + jsonifyTime);
|
logger.debug("Jsonify TimeTaken=" + jsonifyTime);
|
||||||
}
|
}
|
||||||
return result;
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void writeValueAsStringSync(Context ctx, Object obj) throws JsonProcessingException {
|
||||||
|
String result = jsonifyResult(ctx, obj, metricsRegistry, OBJECT_MAPPER, LOG);
|
||||||
|
ctx.result(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void writeValueAsStringAsync(Context ctx, Object obj) {
|
||||||
|
ctx.result(CompletableFuture.supplyAsync(() -> {
|
||||||
|
try {
|
||||||
|
return jsonifyResult(ctx, obj, metricsRegistry, OBJECT_MAPPER, LOG);
|
||||||
} catch (JsonProcessingException e) {
|
} catch (JsonProcessingException e) {
|
||||||
throw new HoodieException("Failed to JSON encode the value", e);
|
throw new HoodieException("Failed to JSON encode the value", e);
|
||||||
}
|
}
|
||||||
@@ -392,6 +421,44 @@ public class RequestHandler {
|
|||||||
}, true));
|
}, true));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void registerMarkerAPI() {
|
||||||
|
app.get(MarkerOperation.ALL_MARKERS_URL, new ViewHandler(ctx -> {
|
||||||
|
metricsRegistry.add("ALL_MARKERS", 1);
|
||||||
|
Set<String> markers = markerHandler.getAllMarkers(
|
||||||
|
ctx.queryParam(MarkerOperation.MARKER_DIR_PATH_PARAM, ""));
|
||||||
|
writeValueAsString(ctx, markers);
|
||||||
|
}, false));
|
||||||
|
|
||||||
|
app.get(MarkerOperation.CREATE_AND_MERGE_MARKERS_URL, new ViewHandler(ctx -> {
|
||||||
|
metricsRegistry.add("CREATE_AND_MERGE_MARKERS", 1);
|
||||||
|
Set<String> markers = markerHandler.getCreateAndMergeMarkers(
|
||||||
|
ctx.queryParam(MarkerOperation.MARKER_DIR_PATH_PARAM, ""));
|
||||||
|
writeValueAsString(ctx, markers);
|
||||||
|
}, false));
|
||||||
|
|
||||||
|
app.get(MarkerOperation.MARKERS_DIR_EXISTS_URL, new ViewHandler(ctx -> {
|
||||||
|
metricsRegistry.add("MARKERS_DIR_EXISTS", 1);
|
||||||
|
boolean exist = markerHandler.doesMarkerDirExist(
|
||||||
|
ctx.queryParam(MarkerOperation.MARKER_DIR_PATH_PARAM, ""));
|
||||||
|
writeValueAsString(ctx, exist);
|
||||||
|
}, false));
|
||||||
|
|
||||||
|
app.post(MarkerOperation.CREATE_MARKER_URL, new ViewHandler(ctx -> {
|
||||||
|
metricsRegistry.add("CREATE_MARKER", 1);
|
||||||
|
ctx.result(markerHandler.createMarker(
|
||||||
|
ctx,
|
||||||
|
ctx.queryParam(MarkerOperation.MARKER_DIR_PATH_PARAM, ""),
|
||||||
|
ctx.queryParam(MarkerOperation.MARKER_NAME_PARAM, "")));
|
||||||
|
}, false));
|
||||||
|
|
||||||
|
app.post(MarkerOperation.DELETE_MARKER_DIR_URL, new ViewHandler(ctx -> {
|
||||||
|
metricsRegistry.add("DELETE_MARKER_DIR", 1);
|
||||||
|
boolean success = markerHandler.deleteMarkers(
|
||||||
|
ctx.queryParam(MarkerOperation.MARKER_DIR_PATH_PARAM, ""));
|
||||||
|
writeValueAsString(ctx, success);
|
||||||
|
}, false));
|
||||||
|
}
|
||||||
|
|
||||||
private static boolean isRefreshCheckDisabledInQuery(Context ctxt) {
|
private static boolean isRefreshCheckDisabledInQuery(Context ctxt) {
|
||||||
return Boolean.parseBoolean(ctxt.queryParam(RemoteHoodieTableFileSystemView.REFRESH_OFF));
|
return Boolean.parseBoolean(ctxt.queryParam(RemoteHoodieTableFileSystemView.REFRESH_OFF));
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -21,6 +21,7 @@ package org.apache.hudi.timeline.service;
|
|||||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
|
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
@@ -31,7 +32,6 @@ import com.beust.jcommander.JCommander;
|
|||||||
import com.beust.jcommander.Parameter;
|
import com.beust.jcommander.Parameter;
|
||||||
import io.javalin.Javalin;
|
import io.javalin.Javalin;
|
||||||
import io.javalin.core.util.JettyServerUtil;
|
import io.javalin.core.util.JettyServerUtil;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
@@ -52,39 +52,31 @@ public class TimelineService {
|
|||||||
private static final int DEFAULT_NUM_THREADS = -1;
|
private static final int DEFAULT_NUM_THREADS = -1;
|
||||||
|
|
||||||
private int serverPort;
|
private int serverPort;
|
||||||
|
private Config timelineServerConf;
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
|
private transient HoodieEngineContext context;
|
||||||
private transient FileSystem fs;
|
private transient FileSystem fs;
|
||||||
private transient Javalin app = null;
|
private transient Javalin app = null;
|
||||||
private transient FileSystemViewManager fsViewsManager;
|
private transient FileSystemViewManager fsViewsManager;
|
||||||
private final int numThreads;
|
private transient RequestHandler requestHandler;
|
||||||
private final boolean shouldCompressOutput;
|
|
||||||
private final boolean useAsync;
|
|
||||||
|
|
||||||
public int getServerPort() {
|
public int getServerPort() {
|
||||||
return serverPort;
|
return serverPort;
|
||||||
}
|
}
|
||||||
|
|
||||||
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager, Configuration conf,
|
public TimelineService(HoodieEngineContext context, Configuration hadoopConf, Config timelineServerConf,
|
||||||
int numThreads, boolean compressOutput, boolean useAsync) throws IOException {
|
FileSystem fileSystem, FileSystemViewManager globalFileSystemViewManager) throws IOException {
|
||||||
this.conf = FSUtils.prepareHadoopConf(conf);
|
this.conf = FSUtils.prepareHadoopConf(hadoopConf);
|
||||||
this.fs = FileSystem.get(conf);
|
this.timelineServerConf = timelineServerConf;
|
||||||
this.serverPort = serverPort;
|
this.serverPort = timelineServerConf.serverPort;
|
||||||
|
this.context = context;
|
||||||
|
this.fs = fileSystem;
|
||||||
this.fsViewsManager = globalFileSystemViewManager;
|
this.fsViewsManager = globalFileSystemViewManager;
|
||||||
this.numThreads = numThreads;
|
|
||||||
this.shouldCompressOutput = compressOutput;
|
|
||||||
this.useAsync = useAsync;
|
|
||||||
}
|
|
||||||
|
|
||||||
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager) throws IOException {
|
|
||||||
this(serverPort, globalFileSystemViewManager, new Configuration(), DEFAULT_NUM_THREADS, true, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
public TimelineService(Config config) throws IOException {
|
|
||||||
this(config.serverPort, buildFileSystemViewManager(config,
|
|
||||||
new SerializableConfiguration(FSUtils.prepareHadoopConf(new Configuration()))), new Configuration(),
|
|
||||||
config.numThreads, config.compress, config.async);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Config for {@code TimelineService} class.
|
||||||
|
*/
|
||||||
public static class Config implements Serializable {
|
public static class Config implements Serializable {
|
||||||
|
|
||||||
@Parameter(names = {"--server-port", "-p"}, description = " Server Port")
|
@Parameter(names = {"--server-port", "-p"}, description = " Server Port")
|
||||||
@@ -119,8 +111,128 @@ public class TimelineService {
|
|||||||
@Parameter(names = {"--compress"}, description = "Compress output using gzip")
|
@Parameter(names = {"--compress"}, description = "Compress output using gzip")
|
||||||
public boolean compress = true;
|
public boolean compress = true;
|
||||||
|
|
||||||
|
@Parameter(names = {"--enable-marker-requests", "-em"}, description = "Enable handling of marker-related requests")
|
||||||
|
public boolean enableMarkerRequests = false;
|
||||||
|
|
||||||
|
@Parameter(names = {"--marker-batch-threads", "-mbt"}, description = "Number of threads to use for batch processing marker creation requests")
|
||||||
|
public int markerBatchNumThreads = 20;
|
||||||
|
|
||||||
|
@Parameter(names = {"--marker-batch-interval-ms", "-mbi"}, description = "The interval in milliseconds between two batch processing of marker creation requests")
|
||||||
|
public long markerBatchIntervalMs = 50;
|
||||||
|
|
||||||
|
@Parameter(names = {"--marker-parallelism", "-mdp"}, description = "Parallelism to use for reading and deleting marker files")
|
||||||
|
public int markerParallelism = 100;
|
||||||
|
|
||||||
@Parameter(names = {"--help", "-h"})
|
@Parameter(names = {"--help", "-h"})
|
||||||
public Boolean help = false;
|
public Boolean help = false;
|
||||||
|
|
||||||
|
public static Builder builder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Builder of Config class.
|
||||||
|
*/
|
||||||
|
public static class Builder {
|
||||||
|
private Integer serverPort = 26754;
|
||||||
|
private FileSystemViewStorageType viewStorageType = FileSystemViewStorageType.SPILLABLE_DISK;
|
||||||
|
private Integer maxViewMemPerTableInMB = 2048;
|
||||||
|
private Double memFractionForCompactionPerTable = 0.001;
|
||||||
|
private String baseStorePathForFileGroups = FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue();
|
||||||
|
private String rocksDBPath = FileSystemViewStorageConfig.ROCKSDB_BASE_PATH_PROP.defaultValue();
|
||||||
|
private int numThreads = DEFAULT_NUM_THREADS;
|
||||||
|
private boolean async = false;
|
||||||
|
private boolean compress = true;
|
||||||
|
private boolean enableMarkerRequests = false;
|
||||||
|
private int markerBatchNumThreads = 20;
|
||||||
|
private long markerBatchIntervalMs = 50L;
|
||||||
|
private int markerParallelism = 100;
|
||||||
|
|
||||||
|
public Builder() {}
|
||||||
|
|
||||||
|
public Builder serverPort(int serverPort) {
|
||||||
|
this.serverPort = serverPort;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder viewStorageType(FileSystemViewStorageType viewStorageType) {
|
||||||
|
this.viewStorageType = viewStorageType;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder maxViewMemPerTableInMB(int maxViewMemPerTableInMB) {
|
||||||
|
this.maxViewMemPerTableInMB = maxViewMemPerTableInMB;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder memFractionForCompactionPerTable(double memFractionForCompactionPerTable) {
|
||||||
|
this.memFractionForCompactionPerTable = memFractionForCompactionPerTable;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder baseStorePathForFileGroups(String baseStorePathForFileGroups) {
|
||||||
|
this.baseStorePathForFileGroups = baseStorePathForFileGroups;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder rocksDBPath(String rocksDBPath) {
|
||||||
|
this.rocksDBPath = rocksDBPath;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder numThreads(int numThreads) {
|
||||||
|
this.numThreads = numThreads;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder async(boolean async) {
|
||||||
|
this.async = async;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder compress(boolean compress) {
|
||||||
|
this.compress = compress;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder enableMarkerRequests(boolean enableMarkerRequests) {
|
||||||
|
this.enableMarkerRequests = enableMarkerRequests;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder markerBatchNumThreads(int markerBatchNumThreads) {
|
||||||
|
this.markerBatchNumThreads = markerBatchNumThreads;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder markerBatchIntervalMs(long markerBatchIntervalMs) {
|
||||||
|
this.markerBatchIntervalMs = markerBatchIntervalMs;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder markerParallelism(int markerParallelism) {
|
||||||
|
this.markerParallelism = markerParallelism;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Config build() {
|
||||||
|
Config config = new Config();
|
||||||
|
config.serverPort = this.serverPort;
|
||||||
|
config.viewStorageType = this.viewStorageType;
|
||||||
|
config.maxViewMemPerTableInMB = this.maxViewMemPerTableInMB;
|
||||||
|
config.memFractionForCompactionPerTable = this.memFractionForCompactionPerTable;
|
||||||
|
config.baseStorePathForFileGroups = this.baseStorePathForFileGroups;
|
||||||
|
config.rocksDBPath = this.rocksDBPath;
|
||||||
|
config.numThreads = this.numThreads;
|
||||||
|
config.async = this.async;
|
||||||
|
config.compress = this.compress;
|
||||||
|
config.enableMarkerRequests = this.enableMarkerRequests;
|
||||||
|
config.markerBatchNumThreads = this.markerBatchNumThreads;
|
||||||
|
config.markerBatchIntervalMs = this.markerBatchIntervalMs;
|
||||||
|
config.markerParallelism = this.markerParallelism;
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private int startServiceOnPort(int port) throws IOException {
|
private int startServiceOnPort(int port) throws IOException {
|
||||||
@@ -151,16 +263,17 @@ public class TimelineService {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public int startService() throws IOException {
|
public int startService() throws IOException {
|
||||||
final Server server = numThreads == DEFAULT_NUM_THREADS ? JettyServerUtil.defaultServer()
|
final Server server = timelineServerConf.numThreads == DEFAULT_NUM_THREADS ? JettyServerUtil.defaultServer()
|
||||||
: new Server(new QueuedThreadPool(numThreads));
|
: new Server(new QueuedThreadPool(timelineServerConf.numThreads));
|
||||||
|
|
||||||
app = Javalin.create().server(() -> server);
|
app = Javalin.create().server(() -> server);
|
||||||
if (!shouldCompressOutput) {
|
if (!timelineServerConf.compress) {
|
||||||
app.disableDynamicGzip();
|
app.disableDynamicGzip();
|
||||||
}
|
}
|
||||||
|
|
||||||
RequestHandler requestHandler = new RequestHandler(app, conf, fsViewsManager, useAsync);
|
requestHandler = new RequestHandler(
|
||||||
app.get("/", ctx -> ctx.result("Hello World"));
|
app, conf, timelineServerConf, context, fs, fsViewsManager);
|
||||||
|
app.get("/", ctx -> ctx.result("Hello Hudi"));
|
||||||
requestHandler.register();
|
requestHandler.register();
|
||||||
int realServerPort = startServiceOnPort(serverPort);
|
int realServerPort = startServiceOnPort(serverPort);
|
||||||
LOG.info("Starting Timeline server on port :" + realServerPort);
|
LOG.info("Starting Timeline server on port :" + realServerPort);
|
||||||
@@ -204,6 +317,9 @@ public class TimelineService {
|
|||||||
|
|
||||||
public void close() {
|
public void close() {
|
||||||
LOG.info("Closing Timeline Service");
|
LOG.info("Closing Timeline Service");
|
||||||
|
if (requestHandler != null) {
|
||||||
|
this.requestHandler.stop();
|
||||||
|
}
|
||||||
this.app.stop();
|
this.app.stop();
|
||||||
this.app = null;
|
this.app = null;
|
||||||
this.fsViewsManager.close();
|
this.fsViewsManager.close();
|
||||||
@@ -228,7 +344,9 @@ public class TimelineService {
|
|||||||
|
|
||||||
Configuration conf = FSUtils.prepareHadoopConf(new Configuration());
|
Configuration conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||||
FileSystemViewManager viewManager = buildFileSystemViewManager(cfg, new SerializableConfiguration(conf));
|
FileSystemViewManager viewManager = buildFileSystemViewManager(cfg, new SerializableConfiguration(conf));
|
||||||
TimelineService service = new TimelineService(cfg.serverPort, viewManager);
|
TimelineService service = new TimelineService(
|
||||||
|
new HoodieLocalEngineContext(FSUtils.prepareHadoopConf(new Configuration())),
|
||||||
|
new Configuration(), cfg, FileSystem.get(new Configuration()), viewManager);
|
||||||
service.run();
|
service.run();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -20,8 +20,10 @@ package org.apache.hudi.timeline.service.handlers;
|
|||||||
|
|
||||||
import org.apache.hudi.common.table.timeline.dto.BaseFileDTO;
|
import org.apache.hudi.common.table.timeline.dto.BaseFileDTO;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@@ -34,8 +36,9 @@ import java.util.stream.Collectors;
|
|||||||
*/
|
*/
|
||||||
public class BaseFileHandler extends Handler {
|
public class BaseFileHandler extends Handler {
|
||||||
|
|
||||||
public BaseFileHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
public BaseFileHandler(Configuration conf, TimelineService.Config timelineServiceConfig,
|
||||||
super(conf, viewManager);
|
FileSystem fileSystem, FileSystemViewManager viewManager) throws IOException {
|
||||||
|
super(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<BaseFileDTO> getLatestDataFiles(String basePath, String partitionPath) {
|
public List<BaseFileDTO> getLatestDataFiles(String basePath, String partitionPath) {
|
||||||
|
|||||||
@@ -19,11 +19,14 @@
|
|||||||
package org.apache.hudi.timeline.service.handlers;
|
package org.apache.hudi.timeline.service.handlers;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
|
||||||
import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO;
|
import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO;
|
||||||
import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO;
|
import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO;
|
||||||
import org.apache.hudi.common.table.timeline.dto.FileGroupDTO;
|
import org.apache.hudi.common.table.timeline.dto.FileGroupDTO;
|
||||||
import org.apache.hudi.common.table.timeline.dto.FileSliceDTO;
|
import org.apache.hudi.common.table.timeline.dto.FileSliceDTO;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@@ -36,8 +39,9 @@ import java.util.stream.Collectors;
|
|||||||
*/
|
*/
|
||||||
public class FileSliceHandler extends Handler {
|
public class FileSliceHandler extends Handler {
|
||||||
|
|
||||||
public FileSliceHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
public FileSliceHandler(Configuration conf, TimelineService.Config timelineServiceConfig,
|
||||||
super(conf, viewManager);
|
FileSystem fileSystem, FileSystemViewManager viewManager) throws IOException {
|
||||||
|
super(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<FileSliceDTO> getAllFileSlices(String basePath, String partitionPath) {
|
public List<FileSliceDTO> getAllFileSlices(String basePath, String partitionPath) {
|
||||||
|
|||||||
@@ -19,6 +19,7 @@
|
|||||||
package org.apache.hudi.timeline.service.handlers;
|
package org.apache.hudi.timeline.service.handlers;
|
||||||
|
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
@@ -28,12 +29,15 @@ import java.io.IOException;
|
|||||||
public abstract class Handler {
|
public abstract class Handler {
|
||||||
|
|
||||||
protected final Configuration conf;
|
protected final Configuration conf;
|
||||||
|
protected final TimelineService.Config timelineServiceConfig;
|
||||||
protected final FileSystem fileSystem;
|
protected final FileSystem fileSystem;
|
||||||
protected final FileSystemViewManager viewManager;
|
protected final FileSystemViewManager viewManager;
|
||||||
|
|
||||||
public Handler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
public Handler(Configuration conf, TimelineService.Config timelineServiceConfig,
|
||||||
|
FileSystem fileSystem, FileSystemViewManager viewManager) throws IOException {
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
this.fileSystem = FileSystem.get(conf);
|
this.timelineServiceConfig = timelineServiceConfig;
|
||||||
|
this.fileSystem = fileSystem;
|
||||||
this.viewManager = viewManager;
|
this.viewManager = viewManager;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -0,0 +1,198 @@
|
|||||||
|
/*
|
||||||
|
* 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.timeline.service.handlers;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.metrics.Registry;
|
||||||
|
import org.apache.hudi.common.model.IOType;
|
||||||
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationFuture;
|
||||||
|
import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationDispatchingRunnable;
|
||||||
|
import org.apache.hudi.timeline.service.handlers.marker.MarkerDirState;
|
||||||
|
|
||||||
|
import io.javalin.Context;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.concurrent.ScheduledFuture;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* REST Handler servicing marker requests.
|
||||||
|
*
|
||||||
|
* The marker creation requests are handled asynchronous, while other types of requests
|
||||||
|
* are handled synchronous.
|
||||||
|
*
|
||||||
|
* Marker creation requests are batch processed periodically by a thread. Each batch
|
||||||
|
* processing thread adds new markers to a marker file. Given that marker file operation
|
||||||
|
* can take time, multiple concurrent threads can run at the same, while they operate
|
||||||
|
* on different marker files storing mutually exclusive marker entries. At any given
|
||||||
|
* time, a marker file is touched by at most one thread to guarantee consistency.
|
||||||
|
* Below is an example of running batch processing threads.
|
||||||
|
*
|
||||||
|
* |-----| batch interval
|
||||||
|
* Worker Thread 1 |-------------------------->| writing to MARKERS0
|
||||||
|
* Worker Thread 2 |-------------------------->| writing to MARKERS1
|
||||||
|
* Worker Thread 3 |-------------------------->| writing to MARKERS2
|
||||||
|
*/
|
||||||
|
public class MarkerHandler extends Handler {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(MarkerHandler.class);
|
||||||
|
|
||||||
|
private final Registry metricsRegistry;
|
||||||
|
// a scheduled executor service to schedule dispatching of marker creation requests
|
||||||
|
private final ScheduledExecutorService dispatchingExecutorService;
|
||||||
|
// an executor service to schedule the worker threads of batch processing marker creation requests
|
||||||
|
private final ExecutorService batchingExecutorService;
|
||||||
|
// Parallelism for reading and deleting marker files
|
||||||
|
private final int parallelism;
|
||||||
|
// Marker directory states, {markerDirPath -> MarkerDirState instance}
|
||||||
|
private final Map<String, MarkerDirState> markerDirStateMap = new HashMap<>();
|
||||||
|
// A thread to dispatch marker creation requests to batch processing threads
|
||||||
|
private final MarkerCreationDispatchingRunnable markerCreationDispatchingRunnable;
|
||||||
|
private final Object firstCreationRequestSeenLock = new Object();
|
||||||
|
private transient HoodieEngineContext hoodieEngineContext;
|
||||||
|
private ScheduledFuture<?> dispatchingThreadFuture;
|
||||||
|
private boolean firstCreationRequestSeen;
|
||||||
|
|
||||||
|
public MarkerHandler(Configuration conf, TimelineService.Config timelineServiceConfig,
|
||||||
|
HoodieEngineContext hoodieEngineContext, FileSystem fileSystem,
|
||||||
|
FileSystemViewManager viewManager, Registry metricsRegistry) throws IOException {
|
||||||
|
super(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
|
LOG.debug("MarkerHandler FileSystem: " + this.fileSystem.getScheme());
|
||||||
|
LOG.debug("MarkerHandler batching params: batchNumThreads=" + timelineServiceConfig.markerBatchNumThreads
|
||||||
|
+ " batchIntervalMs=" + timelineServiceConfig.markerBatchIntervalMs + "ms");
|
||||||
|
this.hoodieEngineContext = hoodieEngineContext;
|
||||||
|
this.metricsRegistry = metricsRegistry;
|
||||||
|
this.parallelism = timelineServiceConfig.markerParallelism;
|
||||||
|
this.dispatchingExecutorService = Executors.newSingleThreadScheduledExecutor();
|
||||||
|
this.batchingExecutorService = Executors.newFixedThreadPool(timelineServiceConfig.markerBatchNumThreads);
|
||||||
|
this.markerCreationDispatchingRunnable =
|
||||||
|
new MarkerCreationDispatchingRunnable(markerDirStateMap, batchingExecutorService);
|
||||||
|
this.firstCreationRequestSeen = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stops the dispatching of marker creation requests.
|
||||||
|
*/
|
||||||
|
public void stop() {
|
||||||
|
if (dispatchingThreadFuture != null) {
|
||||||
|
dispatchingThreadFuture.cancel(true);
|
||||||
|
}
|
||||||
|
dispatchingExecutorService.shutdown();
|
||||||
|
batchingExecutorService.shutdown();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param markerDir marker directory path
|
||||||
|
* @return all marker paths in the marker directory
|
||||||
|
*/
|
||||||
|
public Set<String> getAllMarkers(String markerDir) {
|
||||||
|
MarkerDirState markerDirState = getMarkerDirState(markerDir);
|
||||||
|
return markerDirState.getAllMarkers();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param markerDir marker directory path
|
||||||
|
* @return all marker paths of write IO type "CREATE" and "MERGE"
|
||||||
|
*/
|
||||||
|
public Set<String> getCreateAndMergeMarkers(String markerDir) {
|
||||||
|
return getAllMarkers(markerDir).stream()
|
||||||
|
.filter(markerName -> !markerName.endsWith(IOType.APPEND.name()))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param markerDir marker directory path
|
||||||
|
* @return {@code true} if the marker directory exists; {@code false} otherwise.
|
||||||
|
*/
|
||||||
|
public boolean doesMarkerDirExist(String markerDir) {
|
||||||
|
MarkerDirState markerDirState = getMarkerDirState(markerDir);
|
||||||
|
return markerDirState.exists();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates a future for an async marker creation request
|
||||||
|
*
|
||||||
|
* The future is added to the marker creation future list and waits for the next batch processing
|
||||||
|
* of marker creation requests.
|
||||||
|
*
|
||||||
|
* @param context Javalin app context
|
||||||
|
* @param markerDir marker directory path
|
||||||
|
* @param markerName marker name
|
||||||
|
* @return the {@code CompletableFuture} instance for the request
|
||||||
|
*/
|
||||||
|
public CompletableFuture<String> createMarker(Context context, String markerDir, String markerName) {
|
||||||
|
LOG.info("Request: create marker " + markerDir + " " + markerName);
|
||||||
|
MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName);
|
||||||
|
// Add the future to the list
|
||||||
|
MarkerDirState markerDirState = getMarkerDirState(markerDir);
|
||||||
|
markerDirState.addMarkerCreationFuture(future);
|
||||||
|
if (!firstCreationRequestSeen) {
|
||||||
|
synchronized (firstCreationRequestSeenLock) {
|
||||||
|
if (!firstCreationRequestSeen) {
|
||||||
|
dispatchingThreadFuture = dispatchingExecutorService.scheduleAtFixedRate(markerCreationDispatchingRunnable,
|
||||||
|
timelineServiceConfig.markerBatchIntervalMs, timelineServiceConfig.markerBatchIntervalMs,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
firstCreationRequestSeen = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return future;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deletes markers in the directory.
|
||||||
|
*
|
||||||
|
* @param markerDir marker directory path
|
||||||
|
* @return {@code true} if successful; {@code false} otherwise.
|
||||||
|
*/
|
||||||
|
public Boolean deleteMarkers(String markerDir) {
|
||||||
|
boolean result = getMarkerDirState(markerDir).deleteAllMarkers();
|
||||||
|
markerDirStateMap.remove(markerDir);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private MarkerDirState getMarkerDirState(String markerDir) {
|
||||||
|
MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
|
||||||
|
if (markerDirState == null) {
|
||||||
|
synchronized (markerDirStateMap) {
|
||||||
|
if (markerDirStateMap.get(markerDir) == null) {
|
||||||
|
markerDirState = new MarkerDirState(markerDir, timelineServiceConfig.markerBatchNumThreads,
|
||||||
|
fileSystem, metricsRegistry, hoodieEngineContext, parallelism);
|
||||||
|
markerDirStateMap.put(markerDir, markerDirState);
|
||||||
|
} else {
|
||||||
|
markerDirState = markerDirStateMap.get(markerDir);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return markerDirState;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -21,8 +21,10 @@ package org.apache.hudi.timeline.service.handlers;
|
|||||||
import org.apache.hudi.common.table.timeline.dto.InstantDTO;
|
import org.apache.hudi.common.table.timeline.dto.InstantDTO;
|
||||||
import org.apache.hudi.common.table.timeline.dto.TimelineDTO;
|
import org.apache.hudi.common.table.timeline.dto.TimelineDTO;
|
||||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||||
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@@ -34,8 +36,9 @@ import java.util.List;
|
|||||||
*/
|
*/
|
||||||
public class TimelineHandler extends Handler {
|
public class TimelineHandler extends Handler {
|
||||||
|
|
||||||
public TimelineHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
public TimelineHandler(Configuration conf, TimelineService.Config timelineServiceConfig,
|
||||||
super(conf, viewManager);
|
FileSystem fileSystem, FileSystemViewManager viewManager) throws IOException {
|
||||||
|
super(conf, timelineServiceConfig, fileSystem, viewManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<InstantDTO> getLastInstant(String basePath) {
|
public List<InstantDTO> getLastInstant(String basePath) {
|
||||||
|
|||||||
@@ -0,0 +1,57 @@
|
|||||||
|
/*
|
||||||
|
* 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.timeline.service.handlers.marker;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Input of batch processing of marker creation requests for a single marker directory.
|
||||||
|
*/
|
||||||
|
public class BatchedMarkerCreationContext {
|
||||||
|
private final String markerDir;
|
||||||
|
private final MarkerDirState markerDirState;
|
||||||
|
// List of marker creation futures to process
|
||||||
|
private final List<MarkerCreationFuture> futures;
|
||||||
|
// File index to use to write markers
|
||||||
|
private final int fileIndex;
|
||||||
|
|
||||||
|
public BatchedMarkerCreationContext(String markerDir, MarkerDirState markerDirState,
|
||||||
|
List<MarkerCreationFuture> futures, int fileIndex) {
|
||||||
|
this.markerDir = markerDir;
|
||||||
|
this.markerDirState = markerDirState;
|
||||||
|
this.futures = futures;
|
||||||
|
this.fileIndex = fileIndex;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getMarkerDir() {
|
||||||
|
return markerDir;
|
||||||
|
}
|
||||||
|
|
||||||
|
public MarkerDirState getMarkerDirState() {
|
||||||
|
return markerDirState;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<MarkerCreationFuture> getFutures() {
|
||||||
|
return futures;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getFileIndex() {
|
||||||
|
return fileIndex;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,51 @@
|
|||||||
|
/*
|
||||||
|
* 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.timeline.service.handlers.marker;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A runnable for batch processing marker creation requests.
|
||||||
|
*/
|
||||||
|
public class BatchedMarkerCreationRunnable implements Runnable {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(BatchedMarkerCreationRunnable.class);
|
||||||
|
|
||||||
|
private final List<BatchedMarkerCreationContext> requestContextList;
|
||||||
|
|
||||||
|
public BatchedMarkerCreationRunnable(List<BatchedMarkerCreationContext> requestContextList) {
|
||||||
|
this.requestContextList = requestContextList;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
LOG.debug("Start processing create marker requests");
|
||||||
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
|
|
||||||
|
for (BatchedMarkerCreationContext requestContext : requestContextList) {
|
||||||
|
requestContext.getMarkerDirState().processMarkerCreationRequests(
|
||||||
|
requestContext.getFutures(), requestContext.getFileIndex());
|
||||||
|
}
|
||||||
|
LOG.debug("Finish batch processing of create marker requests in " + timer.endTimer() + " ms");
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,90 @@
|
|||||||
|
/*
|
||||||
|
* 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.timeline.service.handlers.marker;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A runnable that performs periodic, batched creation of markers for write operations.
|
||||||
|
*/
|
||||||
|
public class MarkerCreationDispatchingRunnable implements Runnable {
|
||||||
|
public static final Logger LOG = LogManager.getLogger(MarkerCreationDispatchingRunnable.class);
|
||||||
|
|
||||||
|
// Marker directory states, {markerDirPath -> MarkerDirState instance}
|
||||||
|
private final Map<String, MarkerDirState> markerDirStateMap;
|
||||||
|
private final ExecutorService executorService;
|
||||||
|
|
||||||
|
public MarkerCreationDispatchingRunnable(
|
||||||
|
Map<String, MarkerDirState> markerDirStateMap, ExecutorService executorService) {
|
||||||
|
this.markerDirStateMap = markerDirStateMap;
|
||||||
|
this.executorService = executorService;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Dispatches the marker creation requests that can be process to a worker thread of batch
|
||||||
|
* processing the requests.
|
||||||
|
*
|
||||||
|
* For each marker directory, goes through the following steps:
|
||||||
|
* (1) find the next available file index for writing. If no file index is available,
|
||||||
|
* skip the processing of this marker directory;
|
||||||
|
* (2) fetch the pending marker creation requests for this marker directory. If there is
|
||||||
|
* no request, skip this marker directory;
|
||||||
|
* (3) put the marker directory, marker dir state, list of requests futures, and the file index
|
||||||
|
* to a {@code MarkerDirRequestContext} instance and add the instance to the request context list.
|
||||||
|
*
|
||||||
|
* If the request context list is not empty, spins up a worker thread, {@code MarkerCreationBatchingRunnable},
|
||||||
|
* and pass all the request context to the thread for batch processing. The thread is responsible
|
||||||
|
* for responding to the request futures directly.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
List<BatchedMarkerCreationContext> requestContextList = new ArrayList<>();
|
||||||
|
|
||||||
|
// Only fetch pending marker creation requests that can be processed,
|
||||||
|
// i.e., that markers can be written to a underlying file
|
||||||
|
for (String markerDir : markerDirStateMap.keySet()) {
|
||||||
|
MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
|
||||||
|
Option<Integer> fileIndex = markerDirState.getNextFileIndexToUse();
|
||||||
|
if (!fileIndex.isPresent()) {
|
||||||
|
LOG.debug("All marker files are busy, skip batch processing of create marker requests in " + markerDir);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
List<MarkerCreationFuture> futures = markerDirState.fetchPendingMarkerCreationRequests();
|
||||||
|
if (futures.isEmpty()) {
|
||||||
|
markerDirState.markFileAsAvailable(fileIndex.get());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
requestContextList.add(
|
||||||
|
new BatchedMarkerCreationContext(markerDir, markerDirState, futures, fileIndex.get()));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (requestContextList.size() > 0) {
|
||||||
|
executorService.execute(
|
||||||
|
new BatchedMarkerCreationRunnable(requestContextList));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,69 @@
|
|||||||
|
/*
|
||||||
|
* 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.timeline.service.handlers.marker;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
|
|
||||||
|
import io.javalin.Context;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Future for async marker creation request.
|
||||||
|
*/
|
||||||
|
public class MarkerCreationFuture extends CompletableFuture<String> {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(MarkerCreationFuture.class);
|
||||||
|
private final Context context;
|
||||||
|
private final String markerDirPath;
|
||||||
|
private final String markerName;
|
||||||
|
private boolean result;
|
||||||
|
private final HoodieTimer timer;
|
||||||
|
|
||||||
|
public MarkerCreationFuture(Context context, String markerDirPath, String markerName) {
|
||||||
|
super();
|
||||||
|
this.timer = new HoodieTimer().startTimer();
|
||||||
|
this.context = context;
|
||||||
|
this.markerDirPath = markerDirPath;
|
||||||
|
this.markerName = markerName;
|
||||||
|
this.result = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Context getContext() {
|
||||||
|
return context;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getMarkerDirPath() {
|
||||||
|
return markerDirPath;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getMarkerName() {
|
||||||
|
return markerName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isSuccessful() {
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setResult(boolean result) {
|
||||||
|
LOG.debug("Request queued for " + timer.endTimer() + " ms");
|
||||||
|
this.result = result;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,375 @@
|
|||||||
|
/*
|
||||||
|
* 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.timeline.service.handlers.marker;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.metrics.Registry;
|
||||||
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.io.BufferedReader;
|
||||||
|
import java.io.BufferedWriter;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStreamReader;
|
||||||
|
import java.io.OutputStreamWriter;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
|
||||||
|
import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stores the state of a marker directory.
|
||||||
|
*
|
||||||
|
* The operations inside this class is designed to be thread-safe.
|
||||||
|
*/
|
||||||
|
public class MarkerDirState implements Serializable {
|
||||||
|
public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
|
||||||
|
private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
|
||||||
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
|
// Marker directory
|
||||||
|
private final String markerDirPath;
|
||||||
|
private final FileSystem fileSystem;
|
||||||
|
private final Registry metricsRegistry;
|
||||||
|
// A cached copy of all markers in memory
|
||||||
|
private final Set<String> allMarkers = new HashSet<>();
|
||||||
|
// A cached copy of marker entries in each marker file, stored in StringBuilder
|
||||||
|
// for efficient appending
|
||||||
|
// Mapping: {markerFileIndex -> markers}
|
||||||
|
private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
|
||||||
|
// A list of use status of underlying files storing markers by a thread.
|
||||||
|
// {@code true} means the file is in use by a {@code BatchCreateMarkerRunnable}.
|
||||||
|
// Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
|
||||||
|
private final List<Boolean> threadUseStatus;
|
||||||
|
// A list of pending futures from async marker creation requests
|
||||||
|
private final List<MarkerCreationFuture> markerCreationFutures = new ArrayList<>();
|
||||||
|
private final int parallelism;
|
||||||
|
private final Object markerCreationProcessingLock = new Object();
|
||||||
|
private transient HoodieEngineContext hoodieEngineContext;
|
||||||
|
// Last underlying file index used, for finding the next file index
|
||||||
|
// in a round-robin fashion
|
||||||
|
private int lastFileIndexUsed = -1;
|
||||||
|
|
||||||
|
public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
|
||||||
|
Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
|
||||||
|
this.markerDirPath = markerDirPath;
|
||||||
|
this.fileSystem = fileSystem;
|
||||||
|
this.metricsRegistry = metricsRegistry;
|
||||||
|
this.hoodieEngineContext = hoodieEngineContext;
|
||||||
|
this.parallelism = parallelism;
|
||||||
|
this.threadUseStatus =
|
||||||
|
Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
|
||||||
|
// Lazy initialization of markers by reading MARKERS* files on the file system
|
||||||
|
syncMarkersFromFileSystem();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return {@code true} if the marker directory exists in the system.
|
||||||
|
*/
|
||||||
|
public boolean exists() {
|
||||||
|
try {
|
||||||
|
return fileSystem.exists(new Path(markerDirPath));
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return all markers in the marker directory.
|
||||||
|
*/
|
||||||
|
public Set<String> getAllMarkers() {
|
||||||
|
return allMarkers;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds a {@code MarkerCreationCompletableFuture} instance from a marker
|
||||||
|
* creation request to the queue.
|
||||||
|
*
|
||||||
|
* @param future {@code MarkerCreationCompletableFuture} instance.
|
||||||
|
*/
|
||||||
|
public void addMarkerCreationFuture(MarkerCreationFuture future) {
|
||||||
|
synchronized (markerCreationFutures) {
|
||||||
|
markerCreationFutures.add(future);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the next file index to use in a round-robin fashion,
|
||||||
|
* or empty if no file is available.
|
||||||
|
*/
|
||||||
|
public Option<Integer> getNextFileIndexToUse() {
|
||||||
|
int fileIndex = -1;
|
||||||
|
synchronized (markerCreationProcessingLock) {
|
||||||
|
// Scans for the next free file index to use after {@code lastFileIndexUsed}
|
||||||
|
for (int i = 0; i < threadUseStatus.size(); i++) {
|
||||||
|
int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
|
||||||
|
if (!threadUseStatus.get(index)) {
|
||||||
|
fileIndex = index;
|
||||||
|
threadUseStatus.set(index, true);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (fileIndex >= 0) {
|
||||||
|
lastFileIndexUsed = fileIndex;
|
||||||
|
return Option.of(fileIndex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return Option.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marks the file as available to use again.
|
||||||
|
*
|
||||||
|
* @param fileIndex file index
|
||||||
|
*/
|
||||||
|
public void markFileAsAvailable(int fileIndex) {
|
||||||
|
synchronized (markerCreationProcessingLock) {
|
||||||
|
threadUseStatus.set(fileIndex, false);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return futures of pending marker creation requests and removes them from the list.
|
||||||
|
*/
|
||||||
|
public List<MarkerCreationFuture> fetchPendingMarkerCreationRequests() {
|
||||||
|
List<MarkerCreationFuture> pendingFutures;
|
||||||
|
synchronized (markerCreationFutures) {
|
||||||
|
if (markerCreationFutures.isEmpty()) {
|
||||||
|
return new ArrayList<>();
|
||||||
|
}
|
||||||
|
pendingFutures = new ArrayList<>(markerCreationFutures);
|
||||||
|
markerCreationFutures.clear();
|
||||||
|
}
|
||||||
|
return pendingFutures;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Processes pending marker creation requests.
|
||||||
|
*
|
||||||
|
* @param pendingMarkerCreationFutures futures of pending marker creation requests
|
||||||
|
* @param fileIndex file index to use to write markers
|
||||||
|
*/
|
||||||
|
public void processMarkerCreationRequests(
|
||||||
|
final List<MarkerCreationFuture> pendingMarkerCreationFutures, int fileIndex) {
|
||||||
|
if (pendingMarkerCreationFutures.isEmpty()) {
|
||||||
|
markFileAsAvailable(fileIndex);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath
|
||||||
|
+ " numRequests=" + pendingMarkerCreationFutures.size() + " fileIndex=" + fileIndex);
|
||||||
|
|
||||||
|
synchronized (markerCreationProcessingLock) {
|
||||||
|
for (MarkerCreationFuture future : pendingMarkerCreationFutures) {
|
||||||
|
String markerName = future.getMarkerName();
|
||||||
|
boolean exists = allMarkers.contains(markerName);
|
||||||
|
if (!exists) {
|
||||||
|
allMarkers.add(markerName);
|
||||||
|
StringBuilder stringBuilder = fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384));
|
||||||
|
stringBuilder.append(markerName);
|
||||||
|
stringBuilder.append('\n');
|
||||||
|
}
|
||||||
|
future.setResult(!exists);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
flushMarkersToFile(fileIndex);
|
||||||
|
markFileAsAvailable(fileIndex);
|
||||||
|
|
||||||
|
for (MarkerCreationFuture future : pendingMarkerCreationFutures) {
|
||||||
|
try {
|
||||||
|
future.complete(jsonifyResult(
|
||||||
|
future.getContext(), future.isSuccessful(), metricsRegistry, OBJECT_MAPPER, LOG));
|
||||||
|
} catch (JsonProcessingException e) {
|
||||||
|
throw new HoodieException("Failed to JSON encode the value", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deletes markers in the directory.
|
||||||
|
*
|
||||||
|
* @return {@code true} if successful; {@code false} otherwise.
|
||||||
|
*/
|
||||||
|
public boolean deleteAllMarkers() {
|
||||||
|
Path dirPath = new Path(markerDirPath);
|
||||||
|
boolean result = false;
|
||||||
|
try {
|
||||||
|
if (fileSystem.exists(dirPath)) {
|
||||||
|
FileStatus[] fileStatuses = fileSystem.listStatus(dirPath);
|
||||||
|
List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
|
||||||
|
.map(fileStatus -> fileStatus.getPath().toString())
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
if (markerDirSubPaths.size() > 0) {
|
||||||
|
SerializableConfiguration conf = new SerializableConfiguration(fileSystem.getConf());
|
||||||
|
int actualParallelism = Math.min(markerDirSubPaths.size(), parallelism);
|
||||||
|
hoodieEngineContext.foreach(markerDirSubPaths, subPathStr -> {
|
||||||
|
Path subPath = new Path(subPathStr);
|
||||||
|
FileSystem fileSystem = subPath.getFileSystem(conf.get());
|
||||||
|
fileSystem.delete(subPath, true);
|
||||||
|
}, actualParallelism);
|
||||||
|
}
|
||||||
|
|
||||||
|
result = fileSystem.delete(dirPath, false);
|
||||||
|
LOG.info("Removing marker directory at " + dirPath);
|
||||||
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||||
|
}
|
||||||
|
allMarkers.clear();
|
||||||
|
fileMarkersMap.clear();
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Syncs all markers maintained in the underlying files under the marker directory in the file system.
|
||||||
|
*/
|
||||||
|
private void syncMarkersFromFileSystem() {
|
||||||
|
Path dirPath = new Path(markerDirPath);
|
||||||
|
try {
|
||||||
|
if (fileSystem.exists(dirPath)) {
|
||||||
|
FileStatus[] fileStatuses = fileSystem.listStatus(dirPath);
|
||||||
|
List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
|
||||||
|
.map(fileStatus -> fileStatus.getPath().toString())
|
||||||
|
.filter(pathStr -> pathStr.contains(MARKERS_FILENAME_PREFIX))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
if (markerDirSubPaths.size() > 0) {
|
||||||
|
SerializableConfiguration conf = new SerializableConfiguration(fileSystem.getConf());
|
||||||
|
int actualParallelism = Math.min(markerDirSubPaths.size(), parallelism);
|
||||||
|
Map<String, Set<String>> fileMarkersSetMap =
|
||||||
|
hoodieEngineContext.mapToPair(markerDirSubPaths, markersFilePathStr -> {
|
||||||
|
Path markersFilePath = new Path(markersFilePathStr);
|
||||||
|
FileSystem fileSystem = markersFilePath.getFileSystem(conf.get());
|
||||||
|
FSDataInputStream fsDataInputStream = null;
|
||||||
|
BufferedReader bufferedReader = null;
|
||||||
|
Set<String> markers = new HashSet<>();
|
||||||
|
try {
|
||||||
|
LOG.debug("Read marker file: " + markersFilePathStr);
|
||||||
|
fsDataInputStream = fileSystem.open(markersFilePath);
|
||||||
|
bufferedReader = new BufferedReader(new InputStreamReader(fsDataInputStream, StandardCharsets.UTF_8));
|
||||||
|
markers = bufferedReader.lines().collect(Collectors.toSet());
|
||||||
|
bufferedReader.close();
|
||||||
|
fsDataInputStream.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to read MARKERS file " + markerDirPath, e);
|
||||||
|
} finally {
|
||||||
|
closeQuietly(bufferedReader);
|
||||||
|
closeQuietly(fsDataInputStream);
|
||||||
|
}
|
||||||
|
return new ImmutablePair<>(markersFilePathStr, markers);
|
||||||
|
}, actualParallelism);
|
||||||
|
|
||||||
|
for (String markersFilePathStr: fileMarkersSetMap.keySet()) {
|
||||||
|
Set<String> fileMarkers = fileMarkersSetMap.get(markersFilePathStr);
|
||||||
|
if (!fileMarkers.isEmpty()) {
|
||||||
|
int index = parseMarkerFileIndex(markersFilePathStr);
|
||||||
|
|
||||||
|
if (index >= 0) {
|
||||||
|
fileMarkersMap.put(index, new StringBuilder(StringUtils.join(",", fileMarkers)));
|
||||||
|
allMarkers.addAll(fileMarkers);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parses the marker file index from the marker file path.
|
||||||
|
*
|
||||||
|
* E.g., if the marker file path is /tmp/table/.hoodie/.temp/000/MARKERS3, the index returned is 3.
|
||||||
|
*
|
||||||
|
* @param markerFilePathStr full path of marker file
|
||||||
|
* @return the marker file index
|
||||||
|
*/
|
||||||
|
private int parseMarkerFileIndex(String markerFilePathStr) {
|
||||||
|
String markerFileName = new Path(markerFilePathStr).getName();
|
||||||
|
int prefixIndex = markerFileName.indexOf(MARKERS_FILENAME_PREFIX);
|
||||||
|
if (prefixIndex < 0) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
return Integer.parseInt(markerFileName.substring(prefixIndex + MARKERS_FILENAME_PREFIX.length()));
|
||||||
|
} catch (NumberFormatException nfe) {
|
||||||
|
LOG.error("Failed to parse marker file index from " + markerFilePathStr);
|
||||||
|
throw new HoodieException(nfe.getMessage(), nfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Flushes markers to the underlying file.
|
||||||
|
*
|
||||||
|
* @param markerFileIndex file index to use.
|
||||||
|
*/
|
||||||
|
private void flushMarkersToFile(int markerFileIndex) {
|
||||||
|
LOG.debug("Write to " + markerDirPath + "/" + MARKERS_FILENAME_PREFIX + markerFileIndex);
|
||||||
|
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||||
|
Path markersFilePath = new Path(markerDirPath, MARKERS_FILENAME_PREFIX + markerFileIndex);
|
||||||
|
Path dirPath = markersFilePath.getParent();
|
||||||
|
try {
|
||||||
|
if (!fileSystem.exists(dirPath)) {
|
||||||
|
fileSystem.mkdirs(dirPath);
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to make dir " + dirPath, e);
|
||||||
|
}
|
||||||
|
FSDataOutputStream fsDataOutputStream = null;
|
||||||
|
BufferedWriter bufferedWriter = null;
|
||||||
|
try {
|
||||||
|
fsDataOutputStream = fileSystem.create(markersFilePath);
|
||||||
|
bufferedWriter = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8));
|
||||||
|
bufferedWriter.write(fileMarkersMap.get(markerFileIndex).toString());
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to overwrite marker file " + markersFilePath, e);
|
||||||
|
} finally {
|
||||||
|
closeQuietly(bufferedWriter);
|
||||||
|
closeQuietly(fsDataOutputStream);
|
||||||
|
}
|
||||||
|
LOG.debug(markersFilePath.toString() + " written in " + timer.endTimer() + " ms");
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -30,6 +30,8 @@ import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
|||||||
import org.apache.hudi.common.table.view.TestHoodieTableFileSystemView;
|
import org.apache.hudi.common.table.view.TestHoodieTableFileSystemView;
|
||||||
import org.apache.hudi.timeline.service.TimelineService;
|
import org.apache.hudi.timeline.service.TimelineService;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -51,7 +53,8 @@ public class TestRemoteHoodieTableFileSystemView extends TestHoodieTableFileSyst
|
|||||||
HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
|
HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
|
||||||
|
|
||||||
try {
|
try {
|
||||||
server = new TimelineService(0,
|
server = new TimelineService(localEngineContext, new Configuration(),
|
||||||
|
TimelineService.Config.builder().serverPort(0).build(), FileSystem.get(new Configuration()),
|
||||||
FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, sConf, commonConfig));
|
FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, sConf, commonConfig));
|
||||||
server.startService();
|
server.startService();
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
|
|||||||
@@ -20,7 +20,9 @@ package org.apache.hudi.utilities.perf;
|
|||||||
|
|
||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||||
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.FileSlice;
|
import org.apache.hudi.common.model.FileSlice;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
@@ -37,6 +39,7 @@ import com.beust.jcommander.Parameter;
|
|||||||
import com.codahale.metrics.Histogram;
|
import com.codahale.metrics.Histogram;
|
||||||
import com.codahale.metrics.Snapshot;
|
import com.codahale.metrics.Snapshot;
|
||||||
import com.codahale.metrics.UniformReservoir;
|
import com.codahale.metrics.UniformReservoir;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -71,7 +74,12 @@ public class TimelineServerPerf implements Serializable {
|
|||||||
public TimelineServerPerf(Config cfg) throws IOException {
|
public TimelineServerPerf(Config cfg) throws IOException {
|
||||||
this.cfg = cfg;
|
this.cfg = cfg;
|
||||||
useExternalTimelineServer = (cfg.serverHost != null);
|
useExternalTimelineServer = (cfg.serverHost != null);
|
||||||
this.timelineServer = new TimelineService(cfg.getTimelinServerConfig());
|
TimelineService.Config timelineServiceConf = cfg.getTimelinServerConfig();
|
||||||
|
this.timelineServer = new TimelineService(
|
||||||
|
new HoodieLocalEngineContext(FSUtils.prepareHadoopConf(new Configuration())),
|
||||||
|
new Configuration(), timelineServiceConf, FileSystem.get(new Configuration()),
|
||||||
|
TimelineService.buildFileSystemViewManager(timelineServiceConf,
|
||||||
|
new SerializableConfiguration(FSUtils.prepareHadoopConf(new Configuration()))));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void setHostAddrFromSparkConf(SparkConf sparkConf) {
|
private void setHostAddrFromSparkConf(SparkConf sparkConf) {
|
||||||
|
|||||||
Reference in New Issue
Block a user