This is an automated email from the ASF dual-hosted git repository.

bchapuis pushed a commit to branch geoparquet-filtering-and-simplification
in repository https://gitbox.apache.org/repos/asf/incubator-baremaps.git

commit f1c749a2664948fe88d3d80abbe835366e3232dc
Author: Bertil Chapuis <[email protected]>
AuthorDate: Tue Oct 8 22:18:46 2024 +0200

    Distinguish types with single and repeated values
---
 .../geoparquet/GeoParquetTypeConversion.java       |   4 +-
 .../baremaps/geoparquet/GeoParquetGroup.java       | 135 ++++++++++++---------
 .../geoparquet/GeoParquetGroupConverter.java       |   1 -
 .../baremaps/geoparquet/GeoParquetReaderTest.java  |   2 +-
 4 files changed, 81 insertions(+), 61 deletions(-)

diff --git 
a/baremaps-core/src/main/java/org/apache/baremaps/storage/geoparquet/GeoParquetTypeConversion.java
 
b/baremaps-core/src/main/java/org/apache/baremaps/storage/geoparquet/GeoParquetTypeConversion.java
index 3e618e9a..b5de3a3d 100644
--- 
a/baremaps-core/src/main/java/org/apache/baremaps/storage/geoparquet/GeoParquetTypeConversion.java
+++ 
b/baremaps-core/src/main/java/org/apache/baremaps/storage/geoparquet/GeoParquetTypeConversion.java
@@ -76,7 +76,7 @@ public class GeoParquetTypeConversion {
       }
       Field field = fields.get(i);
       switch (field.type()) {
-        case BINARY -> values.add(group.getBinaryValue(i).getBytes());
+        case BINARY -> values.add(group.getBinary(i).getBytes());
         case BOOLEAN -> values.add(group.getBooleanValue(i));
         case INTEGER -> values.add(group.getIntegerValue(i));
         case INT96, LONG -> values.add(group.getLongValue(i));
@@ -101,7 +101,7 @@ public class GeoParquetTypeConversion {
       }
       Field field = fields.get(i);
       nested.put(field.name(), switch (field.type()) {
-        case BINARY -> group.getBinaryValue(i).getBytes();
+        case BINARY -> group.getBinary(i).getBytes();
         case BOOLEAN -> group.getBooleanValue(i);
         case INTEGER -> group.getIntegerValue(i);
         case INT96, LONG -> group.getLongValue(i);
diff --git 
a/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroup.java
 
b/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroup.java
index 7675d1ee..8108b11c 100644
--- 
a/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroup.java
+++ 
b/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroup.java
@@ -20,7 +20,6 @@ package org.apache.baremaps.geoparquet;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.parquet.io.api.Binary;
-import org.apache.parquet.io.api.RecordConsumer;
 import org.apache.parquet.schema.GroupType;
 import org.locationtech.jts.geom.Envelope;
 import org.locationtech.jts.geom.Geometry;
@@ -36,7 +35,7 @@ public class GeoParquetGroup {
 
   private final Schema geoParquetSchema;
 
-  private final List<?>[] data;
+  private final Object[] data;
 
   public GeoParquetGroup(
       GroupType schema,
@@ -45,9 +44,14 @@ public class GeoParquetGroup {
     this.schema = schema;
     this.metadata = metadata;
     this.geoParquetSchema = geoParquetSchema;
-    this.data = new List[schema.getFields().size()];
+    this.data = new Object[schema.getFields().size()];
     for (int i = 0; i < schema.getFieldCount(); i++) {
-      this.data[i] = new ArrayList<>();
+      Field field = geoParquetSchema.fields().get(i);
+      if (field.cardinality() == Cardinality.REPEATED) {
+        this.data[i] = new ArrayList<>();
+      } else {
+        this.data[i] = null; // For REQUIRED or OPTIONAL fields
+      }
     }
   }
 
@@ -70,8 +74,13 @@ public class GeoParquetGroup {
   }
 
   public int getFieldRepetitionCount(int fieldIndex) {
-    List<?> list = data[fieldIndex];
-    return list == null ? 0 : list.size();
+    Field field = geoParquetSchema.fields().get(fieldIndex);
+    if (field.cardinality() == Cardinality.REPEATED) {
+      List<?> list = (List<?>) data[fieldIndex];
+      return list == null ? 0 : list.size();
+    } else {
+      return data[fieldIndex] == null ? 0 : 1;
+    }
   }
 
   public String getValueToString(int fieldIndex, int index) {
@@ -120,64 +129,54 @@ public class GeoParquetGroup {
   }
 
   private Object getValue(int fieldIndex, int index) {
-    List<?> list = getObjects(fieldIndex);
-    try {
+    Object value = data[fieldIndex];
+    if (value instanceof List<?>list) {
       return list.get(index);
-    } catch (IndexOutOfBoundsException e) {
+    } else if (index == 0) {
+      return value;
+    } else {
       String elementText = String.format(" element number %d ", index);
       throw createGeoParquetException(fieldIndex, elementText);
     }
   }
 
-  private List<?> getObjects(int fieldIndex) {
-    List<?> list;
-    if (fieldIndex < 0 || fieldIndex >= data.length) {
-      throw createGeoParquetException(fieldIndex, "");
-    }
-    list = data[fieldIndex];
-    return list;
-  }
-
   private GeoParquetException createGeoParquetException(int fieldIndex, String 
elementText) {
     String msg = String.format("Not found %d (%s)%s in group%n%s", fieldIndex,
         schema.getFieldName(fieldIndex), elementText, this);
     return new GeoParquetException(msg);
   }
 
-  private void add(int fieldIndex, Primitive value) {
-    org.apache.parquet.schema.Type type = schema.getType(fieldIndex);
-    List list = getObjects(fieldIndex);
-    if (!type.isRepetition(org.apache.parquet.schema.Type.Repetition.REPEATED)
-        && !list.isEmpty()) {
-      throw new IllegalStateException("field " + fieldIndex + " (" + 
type.getName()
-          + ") can not have more than one value: " + list);
+  private void addValue(int fieldIndex, Object value) {
+    if (data[fieldIndex] instanceof List list) {
+      list.add(value);
+    } else {
+      data[fieldIndex] = value;
     }
-    list.add(value);
   }
 
   public void add(int fieldIndex, int value) {
-    add(fieldIndex, new IntegerValue(value));
+    addValue(fieldIndex, new IntegerValue(value));
   }
 
   public void add(int fieldIndex, long value) {
-    add(fieldIndex, new LongValue(value));
+    addValue(fieldIndex, new LongValue(value));
   }
 
   public void add(int fieldIndex, String value) {
-    add(fieldIndex, new BinaryValue(Binary.fromString(value)));
+    addValue(fieldIndex, new BinaryValue(Binary.fromString(value)));
   }
 
   public void add(int fieldIndex, boolean value) {
-    add(fieldIndex, new BooleanValue(value));
+    addValue(fieldIndex, new BooleanValue(value));
   }
 
   public void add(int fieldIndex, Binary value) {
     switch 
(getParquetSchema().getType(fieldIndex).asPrimitiveType().getPrimitiveTypeName())
 {
       case BINARY, FIXED_LEN_BYTE_ARRAY:
-        add(fieldIndex, new BinaryValue(value));
+        addValue(fieldIndex, new BinaryValue(value));
         break;
       case INT96:
-        add(fieldIndex, new Int96Value(value));
+        addValue(fieldIndex, new Int96Value(value));
         break;
       default:
         throw new UnsupportedOperationException(
@@ -186,16 +185,15 @@ public class GeoParquetGroup {
   }
 
   public void add(int fieldIndex, float value) {
-    add(fieldIndex, new FloatValue(value));
+    addValue(fieldIndex, new FloatValue(value));
   }
 
   public void add(int fieldIndex, double value) {
-    add(fieldIndex, new DoubleValue(value));
+    addValue(fieldIndex, new DoubleValue(value));
   }
 
   public void add(int fieldIndex, GeoParquetGroup value) {
-    List list = data[fieldIndex];
-    list.add(value);
+    addValue(fieldIndex, value);
   }
 
   public void add(int fieldIndex, Geometry geometry) {
@@ -240,10 +238,6 @@ public class GeoParquetGroup {
     add(getParquetSchema().getFieldIndex(field), 
Binary.fromConstantByteArray(bytes));
   }
 
-  public void writeValue(int field, int index, RecordConsumer recordConsumer) {
-    ((Primitive) getValue(field, index)).writeValue(recordConsumer);
-  }
-
   public String toString() {
     return toString("");
   }
@@ -258,28 +252,55 @@ public class GeoParquetGroup {
     int i = 0;
     for (org.apache.parquet.schema.Type field : schema.getFields()) {
       String name = field.getName();
-      List<?> values = data[i];
+      Object object = data[i];
       ++i;
-      if (values != null && !values.isEmpty()) {
-        for (Object value : values) {
+      if (object != null) {
+        if (object instanceof List values) {
+          for (Object value : values) {
+            builder.append(indent).append(name);
+            if (value == null) {
+              builder.append(": NULL\n");
+            } else if (value instanceof GeoParquetGroup geoParquetGroup) {
+              builder.append('\n');
+              geoParquetGroup.appendToString(builder, indent + "  ");
+            } else {
+              builder.append(": ").append(value).append('\n');
+            }
+          }
+        } else {
           builder.append(indent).append(name);
-          if (value == null) {
+          if (object == null) {
             builder.append(": NULL\n");
-          } else if (value instanceof GeoParquetGroup geoParquetGroup) {
+          } else if (object instanceof GeoParquetGroup geoParquetGroup) {
             builder.append('\n');
             geoParquetGroup.appendToString(builder, indent + "  ");
           } else {
-            builder.append(": ").append(value).append('\n');
+            builder.append(": ").append(object).append('\n');
           }
         }
       }
     }
   }
 
+  private Primitive getValue(int fieldIndex) {
+    Field field = geoParquetSchema.fields().get(fieldIndex);
+    if (field.cardinality() == Cardinality.REPEATED) {
+      throw new IllegalStateException("Field " + fieldIndex + " (" + 
field.name()
+          + ") is repeated. Use getValues() instead.");
+    }
+    return (Primitive) data[fieldIndex];
+  }
+
   public List<Primitive> getValues(int fieldIndex) {
+    Field field = geoParquetSchema.fields().get(fieldIndex);
+    if (field.cardinality() != Cardinality.REPEATED) {
+      throw new IllegalStateException("Field " + fieldIndex + " (" + 
field.name()
+          + ") is not repeated. Use getValue() instead.");
+    }
     return (List<Primitive>) data[fieldIndex];
   }
 
+
   private List<GeoParquetGroup> getGroups(int fieldIndex) {
     return (List<GeoParquetGroup>) data[fieldIndex];
   }
@@ -314,8 +335,8 @@ public class GeoParquetGroup {
         field.schema());
   }
 
-  public Binary getBinaryValue(int fieldIndex) {
-    return getBinaryValues(fieldIndex).get(0);
+  public Binary getBinary(int fieldIndex) {
+    return getBinary(fieldIndex, 0);
   }
 
   public List<Binary> getBinaryValues(int fieldIndex) {
@@ -323,7 +344,7 @@ public class GeoParquetGroup {
   }
 
   public Boolean getBooleanValue(int fieldIndex) {
-    return getBooleanValues(fieldIndex).get(0);
+    return getBoolean(fieldIndex, 0);
   }
 
   public List<Boolean> getBooleanValues(int fieldIndex) {
@@ -331,7 +352,7 @@ public class GeoParquetGroup {
   }
 
   public Double getDoubleValue(int fieldIndex) {
-    return getDoubleValues(fieldIndex).get(0);
+    return getDouble(fieldIndex, 0);
   }
 
   public List<Double> getDoubleValues(int fieldIndex) {
@@ -339,7 +360,7 @@ public class GeoParquetGroup {
   }
 
   public Float getFloatValue(int fieldIndex) {
-    return getFloatValues(fieldIndex).get(0);
+    return getFloat(fieldIndex, 0);
   }
 
   public List<Float> getFloatValues(int fieldIndex) {
@@ -347,7 +368,7 @@ public class GeoParquetGroup {
   }
 
   public Integer getIntegerValue(int fieldIndex) {
-    return getIntegerValues(fieldIndex).get(0);
+    return getInteger(fieldIndex, 0);
   }
 
   public List<Integer> getIntegerValues(int fieldIndex) {
@@ -355,7 +376,7 @@ public class GeoParquetGroup {
   }
 
   public Binary getInt96Value(int fieldIndex) {
-    return getBinaryValues(fieldIndex).get(0);
+    return getBinary(fieldIndex, 0);
   }
 
   public List<Binary> getInt96Values(int fieldIndex) {
@@ -363,7 +384,7 @@ public class GeoParquetGroup {
   }
 
   public Long getLongValue(int fieldIndex) {
-    return getLongValues(fieldIndex).get(0);
+    return getLong(fieldIndex, 0);
   }
 
   public List<Long> getLongValues(int fieldIndex) {
@@ -371,7 +392,7 @@ public class GeoParquetGroup {
   }
 
   public String getStringValue(int fieldIndex) {
-    return getStringValues(fieldIndex).get(0);
+    return getString(fieldIndex, 0);
   }
 
   public List<String> getStringValues(int fieldIndex) {
@@ -379,7 +400,7 @@ public class GeoParquetGroup {
   }
 
   public Geometry getGeometryValue(int fieldIndex) {
-    return getGeometryValues(fieldIndex).get(0);
+    return getGeometry(fieldIndex, 0);
   }
 
   public List<Geometry> getGeometryValues(int fieldIndex) {
@@ -424,7 +445,7 @@ public class GeoParquetGroup {
     return getGroups(fieldIndex);
   }
 
-  public Binary getBinaryValue(String fieldName) {
+  public Binary getBinary(String fieldName) {
     return getBinaryValues(fieldName).get(0);
   }
 
diff --git 
a/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroupConverter.java
 
b/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroupConverter.java
index d5c06724..529e0266 100644
--- 
a/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroupConverter.java
+++ 
b/baremaps-geoparquet/src/main/java/org/apache/baremaps/geoparquet/GeoParquetGroupConverter.java
@@ -45,7 +45,6 @@ class GeoParquetGroupConverter extends GroupConverter {
       } else {
         converters[i] = new GeoParquetGroupConverter(this, i, 
type.asGroupType());
       }
-
     }
   }
 
diff --git 
a/baremaps-geoparquet/src/test/java/org/apache/baremaps/geoparquet/GeoParquetReaderTest.java
 
b/baremaps-geoparquet/src/test/java/org/apache/baremaps/geoparquet/GeoParquetReaderTest.java
index 52813474..96628722 100644
--- 
a/baremaps-geoparquet/src/test/java/org/apache/baremaps/geoparquet/GeoParquetReaderTest.java
+++ 
b/baremaps-geoparquet/src/test/java/org/apache/baremaps/geoparquet/GeoParquetReaderTest.java
@@ -39,7 +39,7 @@ class GeoParquetReaderTest {
     } else {
       geoParquetGroupStream = geoParquetReader.read();
     }
-    geoParquetGroupStream.forEach(group -> groupCount.getAndIncrement());
+    geoParquetGroupStream.peek(System.out::println).forEach(group -> 
groupCount.getAndIncrement());
     assertEquals(expectedGroupCount, groupCount.get());
   }
 

Reply via email to