Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2765966732
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -108,49 +87,15 @@ public List
getParquetFiles(Configuration hadoopConf, String
}
}
- public Stream getConfigsFromStream(
- List fileStream, Configuration conf) {
-
-return fileStream.stream()
-.map(
-fileStatus -> {
- Path path = fileStatus.getPath();
-
- ParquetMetadata metadata =
-
ParquetMetadataExtractor.getInstance().readParquetMetadata(conf, path);
-
- return ParquetFileConfig.builder()
- .schema(metadata.getFileMetaData().getSchema())
- .metadata(metadata)
- .path(path)
- .size(fileStatus.getLen())
- .modificationTime(fileStatus.getModificationTime())
- .rowGroupIndex(metadata.getBlocks().size())
- .codec(
- metadata.getBlocks().isEmpty()
- ? null
- :
metadata.getBlocks().get(0).getColumns().get(0).getCodec())
- .build();
-});
+ Stream getCurrentFileInfo() {
Review Comment:
@the-other-tim-brown You would want to rename this to the plural
(getCurrentFilesInfo())?
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
the-other-tim-brown commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2752256993
##
xtable-core/src/test/java/org/apache/xtable/parquet/ITParquetDataManager.java:
##
@@ -0,0 +1,208 @@
+/*
+ * 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 static org.apache.spark.sql.functions.expr;
+import static org.junit.jupiter.api.Assertions.*;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import org.apache.xtable.conversion.SourceTable;
+import org.apache.xtable.model.InternalSnapshot;
+import org.apache.xtable.model.InternalTable;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.storage.TableFormat;
+
+public class ITParquetDataManager {
Review Comment:
This test is not directly testing the ParquetDataManager. It is testing the
ParquetConversionSource and therefore the tests can be moved in with the
ITParquetConversionSource
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
the-other-tim-brown commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2752226514
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetFileConfig.java:
##
@@ -0,0 +1,70 @@
+/*
+ * 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 lombok.AccessLevel;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.experimental.FieldDefaults;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+@Getter
+@FieldDefaults(makeFinal = true, level = AccessLevel.PRIVATE)
+@Builder
+@AllArgsConstructor
+class ParquetFileConfig {
+ MessageType schema;
+ ParquetMetadata metadata;
+ long rowGroupIndex;
+ long modifTime;
+ long size;
+ CompressionCodecName codec;
+ Path path;
+
+ public ParquetFileConfig(Configuration conf, Path file) {
+long modifTime = -1L;
+ParquetMetadata metadata =
Review Comment:
@sapienza88 I see this is not addressed but it is marked as resolved. I will
push an update here to resolve the issue.
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3806661236 @sapienza88 I will add comments/next steps today. -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3806598112 @vinishjail97 @the-other-tim-brown anybody to explain the remaining work to push this PR? -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2706002961
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##
@@ -149,87 +148,114 @@ public CommitsBacklog
getCommitsBacklog(InstantsForIncrementalSync syncIns
@Override
public TableChange getTableChangeForCommit(Long modificationTime) {
-Stream parquetFiles = getParquetFiles(hadoopConf,
basePath);
Set addedInternalDataFiles = new HashSet<>();
-List tableChangesAfter =
-parquetFiles
-.filter(fileStatus -> fileStatus.getModificationTime() >
modificationTime)
-.collect(Collectors.toList());
-InternalTable internalTable = getMostRecentTable(parquetFiles);
-for (FileStatus tableStatus : tableChangesAfter) {
- InternalDataFile currentDataFile =
createInternalDataFileFromParquetFile(tableStatus);
+List filesMetadata =
+parquetDataManagerExtractor.getParquetFilesMetadataAfterTime(
+hadoopConf,
+parquetDataManagerExtractor.getParquetFiles(hadoopConf, basePath),
Review Comment:
Would you define it as a class attribute?
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2706002961
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##
@@ -149,87 +148,114 @@ public CommitsBacklog
getCommitsBacklog(InstantsForIncrementalSync syncIns
@Override
public TableChange getTableChangeForCommit(Long modificationTime) {
-Stream parquetFiles = getParquetFiles(hadoopConf,
basePath);
Set addedInternalDataFiles = new HashSet<>();
-List tableChangesAfter =
-parquetFiles
-.filter(fileStatus -> fileStatus.getModificationTime() >
modificationTime)
-.collect(Collectors.toList());
-InternalTable internalTable = getMostRecentTable(parquetFiles);
-for (FileStatus tableStatus : tableChangesAfter) {
- InternalDataFile currentDataFile =
createInternalDataFileFromParquetFile(tableStatus);
+List filesMetadata =
+parquetDataManagerExtractor.getParquetFilesMetadataAfterTime(
+hadoopConf,
+parquetDataManagerExtractor.getParquetFiles(hadoopConf, basePath),
Review Comment:
Would you define it as a class attribute?
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2705871589
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##
@@ -149,87 +148,114 @@ public CommitsBacklog
getCommitsBacklog(InstantsForIncrementalSync syncIns
@Override
public TableChange getTableChangeForCommit(Long modificationTime) {
-Stream parquetFiles = getParquetFiles(hadoopConf,
basePath);
Set addedInternalDataFiles = new HashSet<>();
-List tableChangesAfter =
-parquetFiles
-.filter(fileStatus -> fileStatus.getModificationTime() >
modificationTime)
-.collect(Collectors.toList());
-InternalTable internalTable = getMostRecentTable(parquetFiles);
-for (FileStatus tableStatus : tableChangesAfter) {
- InternalDataFile currentDataFile =
createInternalDataFileFromParquetFile(tableStatus);
+List filesMetadata =
+parquetDataManagerExtractor.getParquetFilesMetadataAfterTime(
+hadoopConf,
+parquetDataManagerExtractor.getParquetFiles(hadoopConf, basePath),
Review Comment:
unless the listing function returns a List instead of a
Stream, those many calls are required or else the stream
"closes"
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2705871589
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##
@@ -149,87 +148,114 @@ public CommitsBacklog
getCommitsBacklog(InstantsForIncrementalSync syncIns
@Override
public TableChange getTableChangeForCommit(Long modificationTime) {
-Stream parquetFiles = getParquetFiles(hadoopConf,
basePath);
Set addedInternalDataFiles = new HashSet<>();
-List tableChangesAfter =
-parquetFiles
-.filter(fileStatus -> fileStatus.getModificationTime() >
modificationTime)
-.collect(Collectors.toList());
-InternalTable internalTable = getMostRecentTable(parquetFiles);
-for (FileStatus tableStatus : tableChangesAfter) {
- InternalDataFile currentDataFile =
createInternalDataFileFromParquetFile(tableStatus);
+List filesMetadata =
+parquetDataManagerExtractor.getParquetFilesMetadataAfterTime(
+hadoopConf,
+parquetDataManagerExtractor.getParquetFiles(hadoopConf, basePath),
Review Comment:
unless the listing function returns a List instead of a
Stream, those many calls are required or else the stream
"closes"
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2705871589
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##
@@ -149,87 +148,114 @@ public CommitsBacklog
getCommitsBacklog(InstantsForIncrementalSync syncIns
@Override
public TableChange getTableChangeForCommit(Long modificationTime) {
-Stream parquetFiles = getParquetFiles(hadoopConf,
basePath);
Set addedInternalDataFiles = new HashSet<>();
-List tableChangesAfter =
-parquetFiles
-.filter(fileStatus -> fileStatus.getModificationTime() >
modificationTime)
-.collect(Collectors.toList());
-InternalTable internalTable = getMostRecentTable(parquetFiles);
-for (FileStatus tableStatus : tableChangesAfter) {
- InternalDataFile currentDataFile =
createInternalDataFileFromParquetFile(tableStatus);
+List filesMetadata =
+parquetDataManagerExtractor.getParquetFilesMetadataAfterTime(
+hadoopConf,
+parquetDataManagerExtractor.getParquetFiles(hadoopConf, basePath),
Review Comment:
unless the listing function returns a List of a
Stream, those many calls are required or else the stream
"closes"
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
the-other-tim-brown commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2705823819
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetFileConfig.java:
##
@@ -0,0 +1,70 @@
+/*
+ * 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 lombok.AccessLevel;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.experimental.FieldDefaults;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+@Getter
+@FieldDefaults(makeFinal = true, level = AccessLevel.PRIVATE)
+@Builder
+@AllArgsConstructor
+class ParquetFileConfig {
+ MessageType schema;
+ ParquetMetadata metadata;
+ long rowGroupIndex;
+ long modifTime;
+ long size;
+ CompressionCodecName codec;
+ Path path;
+
+ public ParquetFileConfig(Configuration conf, Path file) {
+long modifTime = -1L;
+ParquetMetadata metadata =
+ParquetMetadataExtractor.getInstance().readParquetMetadata(conf, file);
+
+if (metadata.getBlocks().isEmpty()) {
+ throw new IllegalStateException("Parquet file contains no row groups.");
+}
+try {
+ modifTime =
file.getFileSystem(conf).getFileStatus(file).getModificationTime();
+} catch (IOException e) {
+ e.printStackTrace();
Review Comment:
Let's wrap this in a `ReadException` instead of printing.
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetFileConfig.java:
##
@@ -0,0 +1,70 @@
+/*
+ * 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 lombok.AccessLevel;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.experimental.FieldDefaults;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+@Getter
+@FieldDefaults(makeFinal = true, level = AccessLevel.PRIVATE)
+@Builder
+@AllArgsConstructor
+class ParquetFileConfig {
+ MessageType schema;
+ ParquetMetadata metadata;
+ long rowGroupIndex;
+ long modifTime;
Review Comment:
Let's call this `modificationTime`
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetFileConfig.java:
##
@@ -0,0 +1,70 @@
+/*
+ * 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 WARRAN
Re: [PR] Parquet Incremental Sync [incubator-xtable]
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.
+ *
+ * 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 become 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.
##
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.Bl
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2628373321
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3740566313 @vinishjail97 @the-other-tim-brown pls check the latest changes for a review. thanks. -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3740310914 @vinishjail97 you mentioned this earlier "Filters files by modification time", how that time is set when the files are ingested?in the footers? -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
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.
+ *
+ * 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 become 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]
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2681698461
##
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.
+ *
+ * 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)
+ public static Path appendNewParquetFiles(Path filePath, Path fileToAppend,
MessageType schema)
+ throws IOException {
+Configuration conf = new Configuration();
+long firstBlockIndex = getParquetFileConfig(conf,
filePath).getRowGroupIndex();
+ParquetMetadata existingFooter = ParquetFileReader.readFooter(conf,
filePath);
+Map existingMeta =
existingFooter.getFileMetaData().getKeyValueMetaData();
+Path tempPath = new Path(filePath.getParent(), "." + filePath.getName() +
".tmp");
+ParquetFileWriter writer =
+new ParquetFileWriter(
+HadoopOutputFile.fromPath(tempPath, conf),
+schema,
+ParquetFileWriter.Mode.OVERWRITE,
+DEFAULT_BLOCK_SIZE,
+0);
+// write the initial table with the appended file to add into the
outputPath
+writer.start();
Review Comment:
yes writing/copying data to append it
--
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: [PR] Parquet Incremental Sync [incubator-xtable]
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.
+ *
+ * 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]
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3737875584 > @sapienza88 Should I push a scaffolding PR for basic functionality with incremental sync and you can take it forward from there? no, I'd rather do it from scratch -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2680733571
##
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.
+ *
+ * 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:
Why are we merging two parquet files to one? We are not building a
compaction service right?
##
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;
+imp
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3695997614 > @sapienza88 I'm adding a more detailed design and a class level structure to unblock this PR. > > **Design Principle** XTable operates at a metadata level only. The current PR approach of writing new Parquet files with filtered data is incorrect. XTable should: > > * Discover existing Parquet files from storage > * Generate table format metadata (Hudi, Iceberg, Delta) for those files > * NEVER write new Parquet files or transform data. > > **Architecture** > > ``` > ┌┐ > │ ParquetConversionSource │ > │ - Uses ParquetFileDiscovery to find files │ > │ - Converts file metadata to InternalDataFile │ > │ - Returns snapshots and table changes │ > └┘ > │ > ▼ > ┌┐ > │ ParquetFileDiscovery (new class) │ > │ - Lists all .parquet files from filesystem│ > │ - Filters files by modification time │ > │ - Returns lightweight file metadata │ > └┘ > │ > ▼ > ┌┐ > │FileSystem (HDFS/S3/GCS/Azure) │ > │ - fs.listFiles(basePath, recursive=true) │ > └┘ > ``` > > Use file modification time as commit identifier, you will be able to identify which files have been synced and which haven't been synced. The files not synced need to have metadata generated. The future functionality like making it optimized, handling deleted parquet files in storage can be handled incrementally, hoping to scope low for this PR. - @vinishjail97 thanks. We are already implementing most of the suggested logic, pls look at: - https://github.com/apache/incubator-xtable/blob/22f4026f00b05069e952d7bfbefee7dda10d79c3/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java#L72 **for** Converts file metadata to InternalDataFile, - https://github.com/apache/incubator-xtable/blob/22f4026f00b05069e952d7bfbefee7dda10d79c3/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java#L217 **for** Lists all .parquet files from filesystem - https://github.com/apache/incubator-xtable/blob/22f4026f00b05069e952d7bfbefee7dda10d79c3/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java#L209 **for** Filters files by modification time - https://github.com/apache/incubator-xtable/blob/22f4026f00b05069e952d7bfbefee7dda10d79c3/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java#L151C3-L151C68 **for** returning snapshot and table changes. Let me know if the highlighted implementation can be used in the current PR. -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3695751441 @sapienza88 I'm adding a more detailed design and a class level structure to unblock this PR. **Design Principle** XTable operates at a metadata level only. The current PR approach of writing new Parquet files with filtered data is incorrect. XTable should: - Discover existing Parquet files from storage - Generate table format metadata (Hudi, Iceberg, Delta) for those files - NEVER write new Parquet files or transform data. **Architecture** ``` ┌┐ │ ParquetConversionSource │ │ - Uses ParquetFileDiscovery to find files │ │ - Converts file metadata to InternalDataFile │ │ - Returns snapshots and table changes │ └┘ │ ▼ ┌┐ │ ParquetFileDiscovery (new class) │ │ - Lists all .parquet files from filesystem│ │ - Filters files by modification time │ │ - Returns lightweight file metadata │ └┘ │ ▼ ┌┐ │FileSystem (HDFS/S3/GCS/Azure) │ │ - fs.listFiles(basePath, recursive=true) │ └┘ ``` Use file modification time as commit identifier, you will be able to identify which files have been synced and which haven't been synced. The files not synced need to have metadata generated. The future functionality like making it optimized, handling deleted parquet files in storage can be handled incrementally, hoping to scope low for this PR. -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2641045923
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory dat
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2643979521
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2643979521
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2643979521
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3683921594 XTable shouldn't be writing any new data or parquet files it operators at a metadata level. Can you see this comment for reference? I had written few approaches on how to do incremental parquet sync. https://github.com/apache/incubator-xtable/issues/550#issuecomment-2386526797 -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2641045923
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory dat
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2628373321
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3666930021 @vinishjail97 I added some comments on the functions so that the approach is clearer. All above suggestions were also taken into account in my last commit. -- 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: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2628373321
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
sapienza88 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2628373321
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog) when
reading the full table
+ // after appending
+ // check required before appending the file
+ private boolean checkSchemaIsSame(Configuration conf, Path fileToAppend,
Path fileFromTable) {
+ParquetFileConfig schemaFileAppend = getParquetFileConfig(conf,
fileToAppend);
+ParquetFileConfig schemaFileFromTable = getParquetFileConfig(conf,
fileFromTable);
+return
schemaFileAppend.getSchema().equals(schemaFileFromTable.getSchema());
+ }
+
+ Instant parsePartitionDirToStartTime(
+ String partitionDir, List partitionFields) {
+Map partitionValues = new HashMap<>();
+String[] parts = partitionDir.split("/");
+for (String part : parts) {
+ String[] keyValue = part.split("=");
+ if (keyValue.length == 2) {
+partitionValues.put(keyValue[0].toLowerCase(), keyValue[1]);
+ }
+}
+int year = 1970;
+int month = 1;
+int day = 1;
+int hour = 0;
+int minute = 0;
+int second = 0;
+
+for (InternalPartitionField field : partitionFields) {
+ String fieldName = field.getSourceField().getName().toLowerCase();
+ String value = partitionValues.get(fieldName);
+
+ if (value != null) {
+try {
+ int intValue = Integer.parseInt(value);
+
+ switch (fieldName) {
+case "year":
+ year = intValue;
+ break;
+case "month":
+ month = intValue;
+ break;
+case "day":
+ day = intValue;
+ break;
+case "hour":
+ hour = intValue;
+ break;
+case "minute":
+ minute = intValue;
+ break;
+case "second":
+ second = intValue;
+ break;
+default:
+ break;
+ }
+} catch (NumberFormatException e) {
+ System.err.println(
+ "Warning: Invalid number format for partition field '" +
fieldName + "': " + value);
+}
+ }
+}
+
+try {
+ LocalDateTime localDateTime = LocalDateTime.of(year, month, day, hour,
minute, second);
+ return localDateTime.toInstant(ZoneOffset.UTC);
+} catch (java.time.DateTimeException e) {
+ throw new IllegalArgumentException(
+ "Invalid partition directory date
Re: [PR] Parquet Incremental Sync [incubator-xtable]
vinishjail97 commented on code in PR #768:
URL: https://github.com/apache/incubator-xtable/pull/768#discussion_r2628314654
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
Review Comment:
This is just returning the ParquetReader and not reading any data? Can we
rename to getParquetReader?
##
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java:
##
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import lombok.Builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+
+import org.apache.xtable.model.schema.InternalPartitionField;
+
+@Builder
+public class ParquetDataManager {
+ private ParquetMetadataExtractor metadataExtractor =
ParquetMetadataExtractor.getInstance();
+
+ public ParquetReader readParquetDataAsReader(String filePath, Configuration
conf) {
+ParquetReader reader = null;
+Path file = new Path(filePath);
+try {
+ reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
+} catch (IOException e) {
+ e.printStackTrace();
+}
+return reader;
+ }
+
+ // TODO check if footer of added file can cause problems (catalog)
Re: [PR] Parquet Incremental Sync [incubator-xtable]
rahil-c commented on PR #768: URL: https://github.com/apache/incubator-xtable/pull/768#issuecomment-3656502344 I can do first review for this @the-other-tim-brown @vinishjail97 -- 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]
