You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2023/04/01 13:05:00 UTC

[jira] [Commented] (PARQUET-1211) Column indexes: read/write API

    [ https://issues.apache.org/jira/browse/PARQUET-1211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17707554#comment-17707554 ] 

ASF GitHub Bot commented on PARQUET-1211:
-----------------------------------------

ConeyLiu commented on code in PR #456:
URL: https://github.com/apache/parquet-mr/pull/456#discussion_r1155108674


##########
parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.util.Objects.requireNonNull;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.Formatter;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveStringifier;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+
+import it.unimi.dsi.fastutil.booleans.BooleanArrayList;
+import it.unimi.dsi.fastutil.booleans.BooleanList;
+import it.unimi.dsi.fastutil.booleans.BooleanLists;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+import it.unimi.dsi.fastutil.longs.LongLists;
+
+/**
+ * Builder implementation to create {@link ColumnIndex} objects during writing a parquet file.
+ */
+public abstract class ColumnIndexBuilder {
+
+  static abstract class ColumnIndexBase implements ColumnIndex {
+    private static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.allocate(0);
+    private static final int MAX_VALUE_LENGTH_FOR_TOSTRING = 40;
+    private static final String TOSTRING_TRUNCATION_MARKER = "(...)";
+    private static final int TOSTRING_TRUNCATION_START_POS =
+        (MAX_VALUE_LENGTH_FOR_TOSTRING - TOSTRING_TRUNCATION_MARKER.length()) / 2;
+    private static final int TOSTRING_TRUNCATION_END_POS =
+        MAX_VALUE_LENGTH_FOR_TOSTRING - TOSTRING_TRUNCATION_MARKER.length() - TOSTRING_TRUNCATION_START_POS;
+    private static final String TOSTRING_MISSING_VALUE_MARKER = "<none>";
+
+    final PrimitiveStringifier stringifier;
+    final PrimitiveComparator<Binary> comparator;
+    private boolean[] nullPages;
+    private BoundaryOrder boundaryOrder;
+    // might be null
+    private long[] nullCounts;
+
+    static String truncate(String str) {
+      if (str.length() <= MAX_VALUE_LENGTH_FOR_TOSTRING) {
+        return str;
+      }
+      return str.substring(0, TOSTRING_TRUNCATION_START_POS) + TOSTRING_TRUNCATION_MARKER
+          + str.substring(str.length() - TOSTRING_TRUNCATION_END_POS);
+    }
+
+    ColumnIndexBase(PrimitiveType type) {
+      comparator = type.comparator();
+      stringifier = type.stringifier();
+    }
+
+    @Override
+    public BoundaryOrder getBoundaryOrder() {
+      return boundaryOrder;
+    }
+
+    @Override
+    public List<Long> getNullCounts() {
+      if (nullCounts == null) {
+        return null;
+      }
+      return LongLists.unmodifiable(LongArrayList.wrap(nullCounts));
+    }
+
+    @Override
+    public List<Boolean> getNullPages() {
+      return BooleanLists.unmodifiable(BooleanArrayList.wrap(nullPages));
+    }
+
+    @Override
+    public List<ByteBuffer> getMinValues() {
+      List<ByteBuffer> list = new ArrayList<>(getPageCount());
+      for (int i = 0, n = getPageCount(); i < n; ++i) {
+        if (isNullPage(i)) {
+          list.add(EMPTY_BYTE_BUFFER);
+        } else {
+          list.add(getMinValueAsBytes(i));
+        }
+      }
+      return list;
+    }
+
+    @Override
+    public List<ByteBuffer> getMaxValues() {
+      List<ByteBuffer> list = new ArrayList<>(getPageCount());
+      for (int i = 0, n = getPageCount(); i < n; ++i) {
+        if (isNullPage(i)) {
+          list.add(EMPTY_BYTE_BUFFER);
+        } else {
+          list.add(getMaxValueAsBytes(i));
+        }
+      }
+      return list;
+    }
+
+    @Override
+    public String toString() {
+      try (Formatter formatter = new Formatter()) {
+        formatter.format("Boudary order: %s\n", boundaryOrder);
+        String minMaxPart = "  %-" + MAX_VALUE_LENGTH_FOR_TOSTRING + "s  %-" + MAX_VALUE_LENGTH_FOR_TOSTRING + "s\n";
+        formatter.format("%-10s  %20s" + minMaxPart, "", "null count", "min", "max");
+        String format = "page-%-5d  %20s" + minMaxPart;
+        for (int i = 0, n = nullPages.length; i < n; ++i) {
+          String nullCount = nullCounts == null ? TOSTRING_MISSING_VALUE_MARKER : Long.toString(nullCounts[i]);
+          String min, max;
+          if (nullPages[i]) {
+            min = max = TOSTRING_MISSING_VALUE_MARKER;
+          } else {
+            min = truncate(getMinValueAsString(i));
+            max = truncate(getMaxValueAsString(i));
+          }
+          formatter.format(format, i, nullCount, min, max);
+        }
+        return formatter.toString();
+      }
+    }
+
+    int getPageCount() {
+      return nullPages.length;
+    }
+
+    boolean isNullPage(int pageIndex) {
+      return nullPages[pageIndex];
+    }
+
+    abstract ByteBuffer getMinValueAsBytes(int pageIndex);
+
+    abstract ByteBuffer getMaxValueAsBytes(int pageIndex);
+
+    abstract String getMinValueAsString(int pageIndex);
+
+    abstract String getMaxValueAsString(int pageIndex);
+  }
+
+  private static final ColumnIndexBuilder NO_OP_BUILDER = new ColumnIndexBuilder() {
+    @Override
+    public ColumnIndex build() {
+      return null;
+    }
+
+    @Override
+    public void add(Statistics<?> stats) {
+    }
+
+    @Override
+    void addMinMax(Object min, Object max) {
+    }
+
+    @Override
+    ColumnIndexBase createColumnIndex(PrimitiveType type) {
+      return null;
+    }
+
+    @Override
+    void clearMinMax() {
+    }
+
+    @Override
+    void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    }
+
+    @Override
+    int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {

Review Comment:
   @gszadovszky Thanks for the response. It is a little stranger for `IntColumnIndexBuilder` uses a comparator typed  `PrimitiveComparator<Binary>`.
   https://github.com/apache/parquet-mr/blob/5608695f5777de1eb0899d9075ec9411cfdf31d3/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IntColumnIndexBuilder.java#L123 





> Column indexes: read/write API
> ------------------------------
>
>                 Key: PARQUET-1211
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1211
>             Project: Parquet
>          Issue Type: Sub-task
>            Reporter: Gabor Szadovszky
>            Assignee: Gabor Szadovszky
>            Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)