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 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) { Review Comment: Let's take in a `FileStatus` here instead of simply the path. That way we can get the modification time directly from the file status instead of refetching it. With cloud storage like AWS S3 or GCP's GCS, every call is sending a request to their service which adds latency and you also get billed for it. ########## xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java: ########## @@ -149,87 +148,114 @@ public CommitsBacklog<Long> getCommitsBacklog(InstantsForIncrementalSync syncIns @Override public TableChange getTableChangeForCommit(Long modificationTime) { - Stream<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, basePath); Set<InternalDataFile> addedInternalDataFiles = new HashSet<>(); - List<FileStatus> tableChangesAfter = - parquetFiles - .filter(fileStatus -> fileStatus.getModificationTime() > modificationTime) - .collect(Collectors.toList()); - InternalTable internalTable = getMostRecentTable(parquetFiles); - for (FileStatus tableStatus : tableChangesAfter) { - InternalDataFile currentDataFile = createInternalDataFileFromParquetFile(tableStatus); + List<ParquetFileConfig> filesMetadata = + parquetDataManagerExtractor.getParquetFilesMetadataAfterTime( + hadoopConf, + parquetDataManagerExtractor.getParquetFiles(hadoopConf, basePath), Review Comment: Let's try to fetch the list of parquet files once from the file system for better performance. ########## 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: Reading the metadata for all files is going to be quite expensive. Let's lazily evaluate this when it is needed? -- 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]
