linliu-code commented on code in PR #9564:
URL: https://github.com/apache/hudi/pull/9564#discussion_r1323795649


##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetKeyedLookupReader.java:
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.hudi.io.storage;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.metadata.parquet.ParquetFileMetadataLoader;
+import org.apache.hudi.metadata.parquet.RowGroup;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.hadoop.util.CompressionConverter;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.SortedSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+
+import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
+import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
+import static org.apache.parquet.column.ValuesType.VALUES;
+
+/**
+ * Implements an efficient lookup for a key in a Parquet file, by using page 
level statistics, and bloom filters.
+ * Parquet file is expected to have two columns :
+ * 1. `key` binary column, which is the column to be used for lookup
+ * 2. `value` binary column, which is the column to be returned as a result of 
lookup
+ *
+ * as well as being sorted by `key` column.
+ *
+ * Known limitations:
+ * 1) Does not do bloom filter based skipping of row groups.
+ */
+public class HoodieParquetKeyedLookupReader {
+  private static Logger LOG = 
LoggerFactory.getLogger(HoodieParquetKeyedLookupReader.class);
+  private static String KEY = "key";
+  private static String VALUE = "value";
+  private final InputFile parquetFile;
+  private final Configuration conf;
+  private final ParquetFileMetadataLoader metadataLoader;
+  private final CompressionCodecFactory codecFactory;
+  private final ParquetMetadataConverter converter;
+
+  public HoodieParquetKeyedLookupReader(Configuration conf, InputFile 
parquetFile) throws Exception {
+    this.conf = conf;
+    this.parquetFile = parquetFile;
+    this.metadataLoader = new ParquetFileMetadataLoader(
+        parquetFile, 
ParquetFileMetadataLoader.Options.builder().enableLoadBloomFilters().build());
+    this.codecFactory = HadoopCodecs.newFactory(0);
+    this.converter = new ParquetMetadataConverter();
+
+    metadataLoader.load();
+  }
+
+  public Map<String, Option<String>> lookup(SortedSet<String> keys) throws 
Exception {
+    Map<String, Option<String>> keyToValue = new HashMap<>();
+    try (CompressionConverter.TransParquetFileReader reader = new 
CompressionConverter.TransParquetFileReader(
+        parquetFile, HadoopReadOptions.builder(conf).build())) {
+      Map<String, String> matchingRecords = getMatchingRecords(reader, new 
LinkedList<>(keys));
+      for (String key: keys) {
+        if (matchingRecords.containsKey(key)) {
+          keyToValue.put(key, Option.of(matchingRecords.get(key)));
+        } else {
+          keyToValue.put(key, Option.empty());
+        }
+      }
+    }
+    return keyToValue;
+  }
+
+  private Map<String, String> 
getMatchingRecords(CompressionConverter.TransParquetFileReader reader,
+                                                 Queue<String> keys) throws 
Exception {
+    Map<String, String> keyToValue = new HashMap<>();
+    for (RowGroup rowGroup : metadataLoader.getRowGroups()) {
+      final int keyColNumber = searchColumn(rowGroup, KEY);
+      if (keyColNumber < 0) {
+        throw new IllegalArgumentException("Cannot find key column in 
schema.");
+      }
+
+      // Skip row group using bloom filter if possible.
+      if (shouldSkip(rowGroup, keyColNumber, new LinkedList<>(keys))) {
+        continue;
+      }
+
+      ArrayList<Pair<String, Long>> keyPositions = 
lookupKeyColumnChunk(reader, rowGroup, keyColNumber, keys);
+      // Now fetch the respective values out.
+      final int valueColNumber = searchColumn(rowGroup, VALUE);
+      if (valueColNumber < 0) {
+        throw new IllegalArgumentException("Cannot find value column in 
schema.");
+      }
+      if (!keyPositions.isEmpty()) {
+        keyToValue.putAll(fetchFromValueColumnChunk(reader, rowGroup, 
valueColNumber, keyPositions));
+      }
+      LOG.info("Done with rowGroup");
+    }
+    return keyToValue;
+  }
+
+  public static boolean shouldSkip(RowGroup rowGroup, int colNumber, 
Queue<String> keys) {
+    if (rowGroup.getBloomFilters().isEmpty()) {
+      return false;
+    }
+
+    BloomFilter bloomFilter = null;
+    try {
+      bloomFilter = rowGroup.getBloomFilters().get(colNumber);
+    } catch (Exception e) {
+      LOG.warn("Can not load the bloom filter correctly.", e);
+      return false;
+    }
+
+    // Bloom filter is not found for this column; can not skip.
+    if (bloomFilter == null) {
+      return false;
+    }
+
+    while (!keys.isEmpty()) {
+      String key = keys.poll();
+      Binary binary = Binary.fromString(key);
+      long hash = bloomFilter.hash(binary);
+      // At least one key is found; can not skip.
+      if (bloomFilter.findHash(hash)) {
+        return false;
+      }
+    }
+
+    // No keys are found in the filter; skip.
+    return true;
+  }
+
+  private int searchColumn(RowGroup rowGroup, String columnDotPath) {
+    final int numColumns = rowGroup.getBlockMetaData().getColumns().size();
+    for (int i = 0; i < numColumns; i++) {
+      if 
(rowGroup.getBlockMetaData().getColumns().get(i).getPath().toDotString().equals(columnDotPath))
 {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  private ArrayList<Pair<String, Long>> 
lookupKeyColumnChunk(CompressionConverter.TransParquetFileReader reader,
+                                                             RowGroup rowGroup,
+                                                             int colNumber,
+                                                             Queue<String> 
keys) throws IOException {
+    ColumnChunkMetaData chunkMetaData = 
rowGroup.getBlockMetaData().getColumns().get(colNumber);
+    ColumnIndex columnIndex = rowGroup.getColumnIndices().get(colNumber);
+    OffsetIndex pageLocation = rowGroup.getPageLocations().get(colNumber);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = 
codecFactory.getDecompressor(chunkMetaData.getCodec());
+    ColumnPath columnPath = 
rowGroup.getBlockMetaData().getColumns().get(colNumber).getPath();
+    ColumnDescriptor columnDescriptor = 
metadataLoader.getFileMetaData().getSchema().getColumnDescription(columnPath.toArray());
+    PrimitiveType type = 
metadataLoader.getFileMetaData().getSchema().getType(columnDescriptor.getPath()).asPrimitiveType();
+
+    AtomicInteger pageCounter = new AtomicInteger(0);
+    int totalPages = pageLocation.getPageCount();
+    ArrayList<Pair<String, Long>> keyAndPositions = new ArrayList<>();
+    for (int pageIndex = 0; pageIndex < totalPages; pageIndex++) {
+      String pageMinKey = new 
String(columnIndex.getMinValues().get(pageIndex).array(), 
StandardCharsets.UTF_8);
+      String pageMaxKey = new 
String(columnIndex.getMaxValues().get(pageIndex).array(), 
StandardCharsets.UTF_8);
+
+      while (keys.peek() != null && keys.peek().compareTo(pageMinKey) < 0) {

Review Comment:
   Done!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to