You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/05/29 19:08:12 UTC

[GitHub] [arrow] lidavidm commented on a change in pull request #7289: ARROW-8948: [Java][Integration] enable duplicate field names integration tests

lidavidm commented on a change in pull request #7289:
URL: https://github.com/apache/arrow/pull/7289#discussion_r432674577



##########
File path: java/vector/src/main/java/org/apache/arrow/vector/VectorSchemaRoot.java
##########
@@ -163,8 +163,22 @@ public void clear() {
     return fieldVectors.stream().collect(Collectors.toList());
   }
 
+  /**
+   * gets a vector by name.
+   *
+   * if name occurs multiple times this returns the first inserted entry for name
+   */
   public FieldVector getVector(String name) {

Review comment:
       This does change the complexity - though presumably it's not an issue unless someone has tens of thousands of columns or is repeatedly fetching vectors in a tight loop.

##########
File path: dev/archery/archery/integration/tester_java.py
##########
@@ -37,7 +37,8 @@ class JavaTester(Tester):
     FLIGHT_SERVER = True
     FLIGHT_CLIENT = True
 
-    JAVA_OPTS = ['-Dio.netty.tryReflectionSetAccessible=true']
+    JAVA_OPTS = ['-Dio.netty.tryReflectionSetAccessible=true',
+                 '-Darrow.struct.conflict.policy=CONFLICT_APPEND']

Review comment:
       If I'm not mistaken - this means the "correct" or "compatible" behavior is opt-in via a JVM flag? Are these flags clearly documented somewhere, I think we have a few others?

##########
File path: java/vector/src/main/java/org/apache/arrow/vector/util/MapWithOrdinal.java
##########
@@ -45,212 +32,36 @@
  * <p>For any instance with N items, this implementation guarantees that ordinals are in the range of [0, N). However,
  * the ordinal assignment is dynamic and may change after an insertion or deletion. Consumers of this class are
  * responsible for explicitly checking the ordinal corresponding to a key via
- * {@link org.apache.arrow.vector.util.MapWithOrdinal#getOrdinal(Object)} before attempting to execute a lookup
+ * {@link MultiMapWithOrdinal#getOrdinal(Object)} before attempting to execute a lookup
  * with an ordinal.
  *
  * @param <K> key type
  * @param <V> value type
  */
-public class MapWithOrdinal<K, V> implements Map<K, V> {
-  private static final Logger logger = LoggerFactory.getLogger(MapWithOrdinal.class);
-
-  private final Map<K, Entry<Integer, V>> primary = new HashMap<>();
-  private final IntObjectHashMap<V> secondary = new IntObjectHashMap<>();
-
-  private final Map<K, V> delegate = new Map<K, V>() {
-    @Override
-    public boolean isEmpty() {
-      return size() == 0;
-    }
-
-    @Override
-    public int size() {
-      return primary.size();
-    }
-
-    @Override
-    public boolean containsKey(Object key) {
-      return primary.containsKey(key);
-    }
-
-    @Override
-    public boolean containsValue(Object value) {
-      return primary.containsValue(value);
-    }
-
-    @Override
-    public V get(Object key) {
-      Entry<Integer, V> pair = primary.get(key);
-      if (pair != null) {
-        return pair.getValue();
-      }
-      return null;
-    }
-
-    @Override
-    public V put(K key, V value) {
-      final Entry<Integer, V> oldPair = primary.get(key);
-      // if key exists try replacing otherwise, assign a new ordinal identifier
-      final int ordinal = oldPair == null ? primary.size() : oldPair.getKey();
-      primary.put(key, new AbstractMap.SimpleImmutableEntry<>(ordinal, value));
-      secondary.put(ordinal, value);
-      return oldPair == null ? null : oldPair.getValue();
-    }
-
-    @Override
-    public V remove(Object key) {
-      final Entry<Integer, V> oldPair = primary.remove(key);
-      if (oldPair != null) {
-        final int lastOrdinal = secondary.size();
-        final V last = secondary.get(lastOrdinal);
-        // normalize mappings so that all numbers until primary.size() is assigned
-        // swap the last element with the deleted one
-        secondary.put(oldPair.getKey(), last);
-        primary.put((K) key, new AbstractMap.SimpleImmutableEntry<>(oldPair.getKey(), last));
-      }
-      return oldPair == null ? null : oldPair.getValue();
-    }
-
-    @Override
-    public void putAll(Map<? extends K, ? extends V> m) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void clear() {
-      primary.clear();
-      secondary.clear();
-    }
-
-    @Override
-    public Set<K> keySet() {
-      return primary.keySet();
-    }
-
-    @Override
-    public Collection<V> values() {
-      return StreamSupport.stream(secondary.entries().spliterator(), false)
-          .map((IntObjectMap.PrimitiveEntry<V> t) -> Preconditions.checkNotNull(t).value())
-          .collect(Collectors.toList());
-    }
-
-    @Override
-    public Set<Entry<K, V>> entrySet() {
-      return primary.entrySet().stream()
-          .map(entry -> new AbstractMap.SimpleImmutableEntry<>(entry.getKey(), entry.getValue().getValue()))
-          .collect(Collectors.toSet());
-    }
-  };
-
-  /**
-   * Returns the value corresponding to the given ordinal.
-   *
-   * @param id ordinal value for lookup
-   * @return an instance of V
-   */
-  public V getByOrdinal(int id) {
-    return secondary.get(id);
-  }
-
-  /**
-   * Returns the ordinal corresponding to the given key.
-   *
-   * @param key key for ordinal lookup
-   * @return ordinal value corresponding to key if it exists or -1
-   */
-  public int getOrdinal(K key) {
-    Entry<Integer, V> pair = primary.get(key);
-    if (pair != null) {
-      return pair.getKey();
-    }
-    return -1;
-  }
-
-  @Override
-  public int size() {
-    return delegate.size();
-  }
+public interface MapWithOrdinal<K, V> {

Review comment:
       (Presumably this was for internal use only.)

##########
File path: java/vector/src/main/java/org/apache/arrow/vector/util/MapWithOrdinal.java
##########
@@ -45,212 +32,36 @@
  * <p>For any instance with N items, this implementation guarantees that ordinals are in the range of [0, N). However,
  * the ordinal assignment is dynamic and may change after an insertion or deletion. Consumers of this class are
  * responsible for explicitly checking the ordinal corresponding to a key via
- * {@link org.apache.arrow.vector.util.MapWithOrdinal#getOrdinal(Object)} before attempting to execute a lookup
+ * {@link MultiMapWithOrdinal#getOrdinal(Object)} before attempting to execute a lookup
  * with an ordinal.
  *
  * @param <K> key type
  * @param <V> value type
  */
-public class MapWithOrdinal<K, V> implements Map<K, V> {
-  private static final Logger logger = LoggerFactory.getLogger(MapWithOrdinal.class);
-
-  private final Map<K, Entry<Integer, V>> primary = new HashMap<>();
-  private final IntObjectHashMap<V> secondary = new IntObjectHashMap<>();
-
-  private final Map<K, V> delegate = new Map<K, V>() {
-    @Override
-    public boolean isEmpty() {
-      return size() == 0;
-    }
-
-    @Override
-    public int size() {
-      return primary.size();
-    }
-
-    @Override
-    public boolean containsKey(Object key) {
-      return primary.containsKey(key);
-    }
-
-    @Override
-    public boolean containsValue(Object value) {
-      return primary.containsValue(value);
-    }
-
-    @Override
-    public V get(Object key) {
-      Entry<Integer, V> pair = primary.get(key);
-      if (pair != null) {
-        return pair.getValue();
-      }
-      return null;
-    }
-
-    @Override
-    public V put(K key, V value) {
-      final Entry<Integer, V> oldPair = primary.get(key);
-      // if key exists try replacing otherwise, assign a new ordinal identifier
-      final int ordinal = oldPair == null ? primary.size() : oldPair.getKey();
-      primary.put(key, new AbstractMap.SimpleImmutableEntry<>(ordinal, value));
-      secondary.put(ordinal, value);
-      return oldPair == null ? null : oldPair.getValue();
-    }
-
-    @Override
-    public V remove(Object key) {
-      final Entry<Integer, V> oldPair = primary.remove(key);
-      if (oldPair != null) {
-        final int lastOrdinal = secondary.size();
-        final V last = secondary.get(lastOrdinal);
-        // normalize mappings so that all numbers until primary.size() is assigned
-        // swap the last element with the deleted one
-        secondary.put(oldPair.getKey(), last);
-        primary.put((K) key, new AbstractMap.SimpleImmutableEntry<>(oldPair.getKey(), last));
-      }
-      return oldPair == null ? null : oldPair.getValue();
-    }
-
-    @Override
-    public void putAll(Map<? extends K, ? extends V> m) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void clear() {
-      primary.clear();
-      secondary.clear();
-    }
-
-    @Override
-    public Set<K> keySet() {
-      return primary.keySet();
-    }
-
-    @Override
-    public Collection<V> values() {
-      return StreamSupport.stream(secondary.entries().spliterator(), false)
-          .map((IntObjectMap.PrimitiveEntry<V> t) -> Preconditions.checkNotNull(t).value())
-          .collect(Collectors.toList());
-    }
-
-    @Override
-    public Set<Entry<K, V>> entrySet() {
-      return primary.entrySet().stream()
-          .map(entry -> new AbstractMap.SimpleImmutableEntry<>(entry.getKey(), entry.getValue().getValue()))
-          .collect(Collectors.toSet());
-    }
-  };
-
-  /**
-   * Returns the value corresponding to the given ordinal.
-   *
-   * @param id ordinal value for lookup
-   * @return an instance of V
-   */
-  public V getByOrdinal(int id) {
-    return secondary.get(id);
-  }
-
-  /**
-   * Returns the ordinal corresponding to the given key.
-   *
-   * @param key key for ordinal lookup
-   * @return ordinal value corresponding to key if it exists or -1
-   */
-  public int getOrdinal(K key) {
-    Entry<Integer, V> pair = primary.get(key);
-    if (pair != null) {
-      return pair.getKey();
-    }
-    return -1;
-  }
-
-  @Override
-  public int size() {
-    return delegate.size();
-  }
+public interface MapWithOrdinal<K, V> {

Review comment:
       This no longer extends the `Map<K,V>` interface - is that an acceptable break?

##########
File path: java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractStructVector.java
##########
@@ -196,21 +245,51 @@ protected void putChild(String name, FieldVector vector) {
     putVector(name, vector);
   }
 
+  private void put(String name, FieldVector vector, boolean overwrite) {
+    final boolean old = vectors.put(
+        Preconditions.checkNotNull(name, "field name cannot be null"),
+        Preconditions.checkNotNull(vector, "vector cannot be null"),
+        overwrite
+    );
+    if (old) {
+      logger.debug("Field [{}] mutated to [{}] ", name,
+          vector.getClass().getSimpleName());
+    }
+  }
+
   /**
    * Inserts the input vector into the map if it does not exist, replaces if it exists already.

Review comment:
       This docstring no longer matches the behavior.




----------------------------------------------------------------
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.

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