Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[GOBBLIN-1929] add dataset root in some common type of datasets #3802

Merged
merged 1 commit into from
Oct 18, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -237,4 +237,9 @@ private static boolean sameFile(FileStatus fileInSource, FileStatus fileInTarget
return fileInTarget.getLen() == fileInSource.getLen() && fileInSource.getModificationTime() <= fileInTarget
.getModificationTime();
}

@Override
public String getDatasetPath() {
return Path.getPathWithoutSchemeAndAuthority(this.rootPath).toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,15 @@ public String datasetURN() {
return this.getFileSetId();
}

@Override
public String getDatasetPath() {
try {
return this.destIcebergTable.accessTableMetadata().location();
} catch (IcebergTable.TableNotFoundException e) {
throw new RuntimeException(e);
}
}

/**
* Finds all files read by the table and generates CopyableFiles.
* For the specific semantics see {@link #createFileSets}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -178,11 +178,8 @@ protected static IcebergSnapshotInfo.ManifestFileInfo calcManifestFileInfo(Manif
}

protected static List<String> discoverDataFilePaths(ManifestFile manifest, FileIO io) throws IOException {
CloseableIterable<String> manifestPathsIterable = ManifestFiles.readPaths(manifest, io);
try {
try (CloseableIterable<String> manifestPathsIterable = ManifestFiles.readPaths(manifest, io)) {
return Lists.newArrayList(manifestPathsIterable);
} finally {
manifestPathsIterable.close();
}
}
protected DatasetDescriptor getDatasetDescriptor(FileSystem fs) {
Expand All @@ -194,6 +191,7 @@ protected DatasetDescriptor getDatasetDescriptor(FileSystem fs) {
descriptor.addMetadata(DatasetConstants.FS_URI, fs.getUri().toString());
return descriptor;
}

/** Registers {@link IcebergTable} after publishing data.
* @param dstMetadata is null if destination {@link IcebergTable} is absent, in which case registration is skipped */
protected void registerIcebergTable(TableMetadata srcMetadata, TableMetadata dstMetadata) {
Expand Down
Loading