sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2681696577


##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.xtable.parquet;
+
+import java.io.IOException;
+import java.util.*;
+
+import lombok.Builder;
+import lombok.extern.log4j.Log4j2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Manages Parquet file operations including reading, writing, and partition 
discovery and path
+ * construction.
+ *
+ * <p>This class provides functions to handle Parquet metadata, validate 
schemas during appends, and
+ * calculate target partition directories based on file modification times and 
defined partition
+ * fields.
+ */
+@Log4j2
+@Builder
+public class ParquetDataManager {
+  private ParquetMetadataExtractor metadataExtractor = 
ParquetMetadataExtractor.getInstance();
+  private static final long DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024; // 128MB
+  private static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024; // 1MB
+
+  /* Use Parquet API to append to a file */
+
+  // after appending check required before appending the file
+  private static boolean checkIfSchemaIsSame(
+      Configuration conf, Path fileToAppend, Path fileFromTable) {
+    ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf, 
fileToAppend);
+    ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf, 
fileFromTable);
+    return 
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+  }
+
+  private static ParquetFileConfig getParquetFileConfig(Configuration conf, 
Path fileToAppend) {
+    ParquetFileConfig parquetFileConfig = new ParquetFileConfig(conf, 
fileToAppend);
+    return parquetFileConfig;
+  }
+  // TODO add safe guards for possible empty parquet files
+  // append a file (merges two files into one .parquet under a partition 
folder)

Review Comment:
   no, but we are keeping one file per partition so that appended files will 
not scattered, to understand the benefits of doing this take this example: If 
you sync 100 rows every 5 minutes: over 24 hours, we’ll have 288 files, also 
for query and storage/compression performances.



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