[
https://issues.apache.org/jira/browse/HUDI-1138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17383837#comment-17383837
]
ASF GitHub Bot commented on HUDI-1138:
--------------------------------------
yihua commented on a change in pull request #3233:
URL: https://github.com/apache/hudi/pull/3233#discussion_r672872648
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
##########
@@ -40,51 +41,49 @@
private static final Logger LOG =
LogManager.getLogger(EmbeddedTimelineService.class);
private int serverPort;
- private int preferredPort;
private String hostAddr;
private HoodieEngineContext context;
private final SerializableConfiguration hadoopConf;
- private final FileSystemViewStorageConfig config;
- private final HoodieMetadataConfig metadataConfig;
+ private final HoodieWriteConfig writeConfig;
private final String basePath;
- private final int numThreads;
- private final boolean shouldCompressOutput;
- private final boolean useAsync;
private transient FileSystemViewManager viewManager;
private transient TimelineService server;
- public EmbeddedTimelineService(HoodieEngineContext context, String
embeddedTimelineServiceHostAddr, int embeddedTimelineServerPort,
- HoodieMetadataConfig metadataConfig,
FileSystemViewStorageConfig config, String basePath,
- int numThreads, boolean compressOutput,
boolean useAsync) {
+ public EmbeddedTimelineService(HoodieEngineContext context, String
embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig) {
setHostAddr(embeddedTimelineServiceHostAddr);
this.context = context;
- this.config = config;
- this.basePath = basePath;
- this.metadataConfig = metadataConfig;
+ this.basePath = writeConfig.getBasePath();
this.hadoopConf = context.getHadoopConf();
this.viewManager = createViewManager();
- this.preferredPort = embeddedTimelineServerPort;
- this.numThreads = numThreads;
- this.shouldCompressOutput = compressOutput;
- this.useAsync = useAsync;
+ this.writeConfig = writeConfig;
}
private FileSystemViewManager createViewManager() {
// Using passed-in configs to build view storage configs
FileSystemViewStorageConfig.Builder builder =
-
FileSystemViewStorageConfig.newBuilder().fromProperties(config.getProps());
+
FileSystemViewStorageConfig.newBuilder().fromProperties(writeConfig.getClientSpecifiedViewStorageConfig().getProps());
FileSystemViewStorageType storageType = builder.build().getStorageType();
if (storageType.equals(FileSystemViewStorageType.REMOTE_ONLY)
|| storageType.equals(FileSystemViewStorageType.REMOTE_FIRST)) {
// Reset to default if set to Remote
builder.withStorageType(FileSystemViewStorageType.MEMORY);
}
- return FileSystemViewManager.createViewManager(context, metadataConfig,
builder.build(), basePath);
+ return FileSystemViewManager.createViewManager(context,
writeConfig.getMetadataConfig(), builder.build(), basePath);
}
public void startServer() throws IOException {
- server = new TimelineService(preferredPort, viewManager,
hadoopConf.newCopy(), numThreads, shouldCompressOutput, useAsync);
+ TimelineService.Config timelineServiceConf = new TimelineService.Config();
+ timelineServiceConf.serverPort =
writeConfig.getEmbeddedTimelineServerPort();
+ timelineServiceConf.numThreads =
writeConfig.getEmbeddedTimelineServerThreads();
+ timelineServiceConf.compress =
writeConfig.getEmbeddedTimelineServerCompressOutput();;
+ timelineServiceConf.async =
writeConfig.getEmbeddedTimelineServerUseAsync();
+ timelineServiceConf.markerBatchNumThreads =
writeConfig.getMarkersTimelineBasedBatchNumThreads();
Review comment:
Yes. I added a `Builder` class for the `TimelineService.Config`.
Member variable scope of TimelineService.Config, i.e., public, is not touched
given it's used in parsing the CLI arguments.
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectMarkerFiles.java
##########
@@ -175,70 +156,29 @@ public static String stripMarkerSuffix(String path) {
return markerFiles;
}
- private 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);
- }
-
- /**
- * 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) {
+ protected Path create(String partitionPath, String dataFileName, IOType
type, boolean checkIfExists) {
+ LOG.info("^^^ [direct] Create marker file : " + partitionPath + " " +
dataFileName);
+ long startTimeMs = System.currentTimeMillis();
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
+ Path dirPath = markerPath.getParent();
try {
- LOG.info("Creating Marker Path=" + markerPath);
- fs.create(markerPath, false).close();
+ if (!fs.exists(dirPath)) {
+ fs.mkdirs(dirPath); // create a new partition as needed.
+ }
} 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 {
- if (fs.exists(markerPath)) {
+ if (checkIfExists && fs.exists(markerPath)) {
LOG.warn("Marker Path=" + markerPath + " already exists, cancel
creation");
- return false;
+ return null;
Review comment:
Resolved.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
> Re-implement marker files via timeline server
> ---------------------------------------------
>
> Key: HUDI-1138
> URL: https://issues.apache.org/jira/browse/HUDI-1138
> Project: Apache Hudi
> Issue Type: Improvement
> Components: Writer Core
> Affects Versions: 0.9.0
> Reporter: Vinoth Chandar
> Assignee: Ethan Guo
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 0.9.0
>
>
> Even as you can argue that RFC-15/consolidated metadata, removes the need for
> deleting partial files written due to spark task failures/stage retries. It
> will still leave extra files inside the table (and users will pay for it
> every month) and we need the marker mechanism to be able to delete these
> partial files.
> Here we explore if we can improve the current marker file mechanism, that
> creates one marker file per data file written, by
> Delegating the createMarker() call to the driver/timeline server, and have it
> create marker metadata into a single file handle, that is flushed for
> durability guarantees
>
> P.S: I was tempted to think Spark listener mechanism can help us deal with
> failed tasks, but it has no guarantees. the writer job could die without
> deleting a partial file. i.e it can improve things, but cant provide
> guarantees
--
This message was sent by Atlassian Jira
(v8.3.4#803005)