You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2019/01/11 04:20:09 UTC

[GitHub] mccheah commented on a change in pull request #73: Allow data output streams to be generated via custom mechanisms when given partitioning and file name

mccheah commented on a change in pull request #73: Allow data output streams to be generated via custom mechanisms when given partitioning and file name
URL: https://github.com/apache/incubator-iceberg/pull/73#discussion_r246999840
 
 

 ##########
 File path: core/src/main/java/com/netflix/iceberg/hadoop/HadoopFileIO.java
 ##########
 @@ -38,4 +55,98 @@ public void deleteFile(String path) {
       throw new RuntimeIOException(e, "Failed to delete file: %s", path);
     }
   }
+
+  @Override
+  public InputFile readMetadataFile(String fileName) {
+    return newInputFile(metadataPath(fileName).toString());
+  }
+
+  @Override
+  public OutputFile newMetadataFile(String fileName) {
+    return newOutputFile(metadataPath(fileName).toString());
+  }
+
+  @Override
+  public OutputFile newPartitionedDataFile(
+      PartitionSpec partitionSpec, StructLike filePartition, String fileName) {
+    String location;
+    if (useObjectStorage) {
+      // try to get db and table portions of the path for context in the object store
+      String context = pathContext(new Path(newDataFileLocation));
+      String partitionAndFilename = String.format(
+          "%s/%s", partitionSpec.partitionToPath(filePartition), fileName);
+      int hash = HASH_FUNC.apply(partitionAndFilename);
+      location = String.format(
+          "%s/%08x/%s/%s/%s",
+          objectStorePath,
+          hash,
+          context,
+          partitionSpec.partitionToPath(filePartition),
+          fileName);
+    } else {
+      location = String.format(
+          "%s/%s/%s",
+          newDataFileLocation,
+          partitionSpec.partitionToPath(filePartition),
+          fileName);
+    }
+    return newOutputFile(location);
+  }
+
+  @Override
+  public OutputFile newUnpartitionedDataFile(String fileName) {
+    return newOutputFile(String.format("%s/%s", newDataFileLocation, fileName));
+  }
+
+  private Path metadataPath(String filename) {
+    return new Path(new Path(tableLocation, "metadata"), filename);
+  }
+
+  public void updateProperties(Map<String, String> newTableProperties) {
 
 Review comment:
   This is a concession to the fact that when table properties are updated, it's possible that the data path could change. It's not immediately clear if we can do better than this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org