prashantwason commented on a change in pull request #5179:
URL: https://github.com/apache/hudi/pull/5179#discussion_r839370478



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java
##########
@@ -118,20 +135,86 @@ public void trySave(int taskPartitionId) {
     }
   }
 
+  private String getMetafileExtension() {
+    // To be backwards compatible, there is no extension to the properties 
file base partition metafile
+    return format.isPresent() ? format.get().getFileExtension() : "";
+  }
+
+  /**
+   * Write the partition metadata in the correct format in the given file path.
+   *
+   * @param filePath Path of the file to write
+   * @throws IOException
+   */
+  private void writeMetafile(Path filePath) throws IOException {
+    if (format.isPresent()) {
+      Schema schema = HoodieAvroUtils.getRecordKeySchema();
+
+      switch (format.get()) {
+        case PARQUET:
+          // Since we are only interested in saving metadata to the footer, 
the schema, blocksizes and other
+          // parameters are not important.
+          MessageType type = 
Types.buildMessage().optional(PrimitiveTypeName.INT64).named("dummyint").named("dummy");
+          HoodieAvroWriteSupport writeSupport = new 
HoodieAvroWriteSupport(type, schema, Option.empty());
+          try (ParquetWriter writer = new ParquetWriter(filePath, 
writeSupport, CompressionCodecName.UNCOMPRESSED, 1024, 1024)) {
+            for (String key : props.stringPropertyNames()) {
+              writeSupport.addFooterMetadata(key, props.getProperty(key));
+            }
+          }
+          break;
+        case ORC:
+          // Since we are only interested in saving metadata to the footer, 
the schema, blocksizes and other
+          // parameters are not important.
+          OrcFile.WriterOptions writerOptions = 
OrcFile.writerOptions(fs.getConf()).fileSystem(fs)
+              .setSchema(AvroOrcUtils.createOrcSchema(schema));
+          try (Writer writer = OrcFile.createWriter(filePath, writerOptions)) {
+            for (String key : props.stringPropertyNames()) {
+              writer.addUserMetadata(key, 
ByteBuffer.wrap(props.getProperty(key).getBytes()));
+            }
+          }
+          break;
+        default:
+          throw new HoodieException("Unsupported format for partition 
metafiles: " + format.get());
+      }
+    } else {
+      // Backwards compatible properties file format
+      FSDataOutputStream os = fs.create(filePath, true);
+      props.store(os, "partition metadata");
+      os.hsync();
+      os.hflush();
+      os.close();
+    }
+  }
+
   /**
    * Read out the metadata for this partition.
    */
   public void readFromFS() throws IOException {
-    FSDataInputStream is = null;
+    Option<Path> metafilePathOption = getPartitionMetafilePath(fs, 
partitionPath);
+    if (!metafilePathOption.isPresent()) {
+      throw new HoodieException("Partition metafile not found in path " + 
partitionPath);
+    }
+
+    final Path metafilePath = metafilePathOption.get();
     try {
-      Path metaFile = new Path(partitionPath, HOODIE_PARTITION_METAFILE);
-      is = fs.open(metaFile);
-      props.load(is);
-    } catch (IOException ioe) {
-      throw new HoodieException("Error reading Hoodie partition metadata for " 
+ partitionPath, ioe);
-    } finally {
-      if (is != null) {
-        is.close();
+      BaseFileUtils reader = 
BaseFileUtils.getInstance(metafilePath.toString());
+      Map<String, String> metadata = reader.readFooter(fs.getConf(), true, 
metafilePath, PARTITION_DEPTH_KEY, COMMIT_TIME_KEY);
+      props.clear();
+      metadata.forEach((k, v) -> props.put(k, v));
+    } catch (UnsupportedOperationException e) {
+      
ValidationUtils.checkArgument(metafilePath.toString().endsWith(HOODIE_PARTITION_METAFILE_PREFIX),

Review comment:
       Done




-- 
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]


Reply via email to