You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/01/05 16:00:39 UTC

arrow git commit: ARROW-411: [Java] Move compactor functions in Integration to a separate Validator module

Repository: arrow
Updated Branches:
  refs/heads/master fdbc57941 -> 9513ca774


ARROW-411: [Java] Move compactor functions in Integration to a separate Validator module

Author: Li Jin <ic...@gmail.com>

Closes #267 from icexelloss/validator and squashes the following commits:

b4e86c5 [Li Jin] ARROW-411: Move compator functions in Integration to a separate Validator moduleO


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/9513ca77
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/9513ca77
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/9513ca77

Branch: refs/heads/master
Commit: 9513ca7741bc036ff369cbbd3b3ee3f4bcc06722
Parents: fdbc579
Author: Li Jin <ic...@gmail.com>
Authored: Thu Jan 5 11:00:32 2017 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Thu Jan 5 11:00:32 2017 -0500

----------------------------------------------------------------------
 .../org/apache/arrow/tools/Integration.java     |  94 +-------------
 .../org/apache/arrow/tools/TestIntegration.java |  32 -----
 .../org/apache/arrow/vector/util/Validator.java | 125 +++++++++++++++++++
 .../apache/arrow/vector/util/TestValidator.java |  57 +++++++++
 4 files changed, 185 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/9513ca77/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
----------------------------------------------------------------------
diff --git a/java/tools/src/main/java/org/apache/arrow/tools/Integration.java b/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
index fd835a6..36d4ee5 100644
--- a/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
+++ b/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
@@ -28,7 +28,6 @@ import java.util.List;
 
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
-import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.VectorLoader;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.VectorUnloader;
@@ -39,10 +38,8 @@ import org.apache.arrow.vector.file.ArrowWriter;
 import org.apache.arrow.vector.file.json.JsonFileReader;
 import org.apache.arrow.vector.file.json.JsonFileWriter;
 import org.apache.arrow.vector.schema.ArrowRecordBatch;
-import org.apache.arrow.vector.types.pojo.ArrowType;
-import org.apache.arrow.vector.types.pojo.ArrowType.FloatingPoint;
-import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.arrow.vector.util.Validator;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.Options;
@@ -51,8 +48,6 @@ import org.apache.commons.cli.PosixParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Objects;
-
 public class Integration {
   private static final Logger LOGGER = LoggerFactory.getLogger(Integration.class);
 
@@ -143,7 +138,7 @@ public class Integration {
           LOGGER.debug("ARROW schema: " + arrowSchema);
           LOGGER.debug("JSON Input file size: " + jsonFile.length());
           LOGGER.debug("JSON schema: " + jsonSchema);
-          compareSchemas(jsonSchema, arrowSchema);
+          Validator.compareSchemas(jsonSchema, arrowSchema);
 
           List<ArrowBlock> recordBatches = footer.getRecordBatches();
           Iterator<ArrowBlock> iterator = recordBatches.iterator();
@@ -154,8 +149,7 @@ public class Integration {
                 VectorSchemaRoot arrowRoot = new VectorSchemaRoot(arrowSchema, allocator);) {
               VectorLoader vectorLoader = new VectorLoader(arrowRoot);
               vectorLoader.load(inRecordBatch);
-              // TODO: compare
-              compare(arrowRoot, jsonRoot);
+              Validator.compareVectorSchemaRoot(arrowRoot, jsonRoot);
             }
             jsonRoot.close();
           }
@@ -227,86 +221,4 @@ public class Integration {
     System.exit(1);
   }
 
-
-  private static void compare(VectorSchemaRoot arrowRoot, VectorSchemaRoot jsonRoot) {
-    compareSchemas(jsonRoot.getSchema(), arrowRoot.getSchema());
-    if (arrowRoot.getRowCount() != jsonRoot.getRowCount()) {
-      throw new IllegalArgumentException("Different row count:\n" + arrowRoot.getRowCount() + "\n" + jsonRoot.getRowCount());
-    }
-    List<FieldVector> arrowVectors = arrowRoot.getFieldVectors();
-    List<FieldVector> jsonVectors = jsonRoot.getFieldVectors();
-    if (arrowVectors.size() != jsonVectors.size()) {
-      throw new IllegalArgumentException("Different column count:\n" + arrowVectors.size() + "\n" + jsonVectors.size());
-    }
-    for (int i = 0; i < arrowVectors.size(); i++) {
-      Field field = arrowRoot.getSchema().getFields().get(i);
-      FieldVector arrowVector = arrowVectors.get(i);
-      FieldVector jsonVector = jsonVectors.get(i);
-      int valueCount = arrowVector.getAccessor().getValueCount();
-      if (valueCount != jsonVector.getAccessor().getValueCount()) {
-        throw new IllegalArgumentException("Different value count for field " + field + " : " + valueCount + " != " + jsonVector.getAccessor().getValueCount());
-      }
-      for (int j = 0; j < valueCount; j++) {
-        Object arrow = arrowVector.getAccessor().getObject(j);
-        Object json = jsonVector.getAccessor().getObject(j);
-        if (!equals(field.getType(), arrow, json)) {
-          throw new IllegalArgumentException(
-              "Different values in column:\n" + field + " at index " + j + ": " + arrow + " != " + json);
-        }
-      }
-    }
-  }
-
-  private static boolean equals(ArrowType type, final Object arrow, final Object json) {
-    if (type instanceof ArrowType.FloatingPoint) {
-      FloatingPoint fpType = (FloatingPoint) type;
-      switch (fpType.getPrecision()) {
-      case DOUBLE:
-        return equalEnough((Double)arrow, (Double)json);
-      case SINGLE:
-        return equalEnough((Float)arrow, (Float)json);
-      case HALF:
-      default:
-        throw new UnsupportedOperationException("unsupported precision: " + fpType);
-      }
-    }
-    return Objects.equal(arrow, json);
-  }
-
-  static boolean equalEnough(Float f1, Float f2) {
-    if (f1 == null || f2 == null) {
-      return f1 == null && f2 == null;
-    }
-    if (f1.isNaN()) {
-      return f2.isNaN();
-    }
-    if (f1.isInfinite()) {
-      return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
-    }
-    float average = Math.abs((f1 + f2) / 2);
-    float differenceScaled = Math.abs(f1 - f2) / (average == 0.0f ? 1f : average);
-    return differenceScaled < 1.0E-6f;
-  }
-
-  static boolean equalEnough(Double f1, Double f2) {
-    if (f1 == null || f2 == null) {
-      return f1 == null && f2 == null;
-    }
-    if (f1.isNaN()) {
-      return f2.isNaN();
-    }
-    if (f1.isInfinite()) {
-      return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
-    }
-    double average = Math.abs((f1 + f2) / 2);
-    double differenceScaled = Math.abs(f1 - f2) / (average == 0.0d ? 1d : average);
-    return differenceScaled < 1.0E-12d;
-  }
-
-
-  private static void compareSchemas(Schema jsonSchema, Schema arrowSchema) {
-    if (!arrowSchema.equals(jsonSchema)) {
-      throw new IllegalArgumentException("Different schemas:\n" + arrowSchema + "\n" + jsonSchema);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/9513ca77/java/tools/src/test/java/org/apache/arrow/tools/TestIntegration.java
----------------------------------------------------------------------
diff --git a/java/tools/src/test/java/org/apache/arrow/tools/TestIntegration.java b/java/tools/src/test/java/org/apache/arrow/tools/TestIntegration.java
index ee6196b..0ae32be 100644
--- a/java/tools/src/test/java/org/apache/arrow/tools/TestIntegration.java
+++ b/java/tools/src/test/java/org/apache/arrow/tools/TestIntegration.java
@@ -22,9 +22,7 @@ import static org.apache.arrow.tools.ArrowFileTestFixtures.validateOutput;
 import static org.apache.arrow.tools.ArrowFileTestFixtures.write;
 import static org.apache.arrow.tools.ArrowFileTestFixtures.writeData;
 import static org.apache.arrow.tools.ArrowFileTestFixtures.writeInput;
-import static org.apache.arrow.tools.Integration.equalEnough;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -238,34 +236,4 @@ public class TestIntegration {
       write(parent.getChild("root"), testInFile);
     }
   }
-
-  @Test
-  public void testFloatComp() {
-    assertTrue(equalEnough(912.4140000000002F, 912.414F));
-    assertTrue(equalEnough(912.4140000000002D, 912.414D));
-    assertTrue(equalEnough(912.414F, 912.4140000000002F));
-    assertTrue(equalEnough(912.414D, 912.4140000000002D));
-    assertFalse(equalEnough(912.414D, 912.4140001D));
-    assertFalse(equalEnough(null, 912.414D));
-    assertTrue(equalEnough((Float)null, null));
-    assertTrue(equalEnough((Double)null, null));
-    assertFalse(equalEnough(912.414D, null));
-    assertFalse(equalEnough(Double.MAX_VALUE, Double.MIN_VALUE));
-    assertFalse(equalEnough(Double.MIN_VALUE, Double.MAX_VALUE));
-    assertTrue(equalEnough(Double.MAX_VALUE, Double.MAX_VALUE));
-    assertTrue(equalEnough(Double.MIN_VALUE, Double.MIN_VALUE));
-    assertTrue(equalEnough(Double.NEGATIVE_INFINITY, Double.NEGATIVE_INFINITY));
-    assertFalse(equalEnough(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY));
-    assertTrue(equalEnough(Double.NaN, Double.NaN));
-    assertFalse(equalEnough(1.0, Double.NaN));
-    assertFalse(equalEnough(Float.MAX_VALUE, Float.MIN_VALUE));
-    assertFalse(equalEnough(Float.MIN_VALUE, Float.MAX_VALUE));
-    assertTrue(equalEnough(Float.MAX_VALUE, Float.MAX_VALUE));
-    assertTrue(equalEnough(Float.MIN_VALUE, Float.MIN_VALUE));
-    assertTrue(equalEnough(Float.NEGATIVE_INFINITY, Float.NEGATIVE_INFINITY));
-    assertFalse(equalEnough(Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY));
-    assertTrue(equalEnough(Float.NaN, Float.NaN));
-    assertFalse(equalEnough(1.0F, Float.NaN));
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/9513ca77/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java b/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
new file mode 100644
index 0000000..a974582
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
@@ -0,0 +1,125 @@
+/**
+ * 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.arrow.vector.util;
+
+import java.util.List;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.Schema;
+
+import com.google.common.base.Objects;
+
+/**
+ * Utility class for validating arrow data structures
+ */
+public class Validator {
+
+    /**
+     * Validate two arrow schemas are equal.
+     *
+     * @throws IllegalArgumentException if they are different.
+     */
+    public static void compareSchemas(Schema schema1, Schema schema2) {
+        if (!schema2.equals(schema1)) {
+            throw new IllegalArgumentException("Different schemas:\n" + schema2 + "\n" + schema1);
+        }
+    }
+
+    /**
+     * Validate two arrow vectorSchemaRoot are equal.
+     *
+     * @throws IllegalArgumentException if they are different.
+     */
+    public static void compareVectorSchemaRoot(VectorSchemaRoot root1, VectorSchemaRoot root2) {
+        compareSchemas(root2.getSchema(), root1.getSchema());
+        if (root1.getRowCount() != root2.getRowCount()) {
+            throw new IllegalArgumentException("Different row count:\n" + root1.getRowCount() + "\n" + root2.getRowCount());
+        }
+        List<FieldVector> arrowVectors = root1.getFieldVectors();
+        List<FieldVector> jsonVectors = root2.getFieldVectors();
+        if (arrowVectors.size() != jsonVectors.size()) {
+            throw new IllegalArgumentException("Different column count:\n" + arrowVectors.size() + "\n" + jsonVectors.size());
+        }
+        for (int i = 0; i < arrowVectors.size(); i++) {
+            Field field = root1.getSchema().getFields().get(i);
+            FieldVector arrowVector = arrowVectors.get(i);
+            FieldVector jsonVector = jsonVectors.get(i);
+            int valueCount = arrowVector.getAccessor().getValueCount();
+            if (valueCount != jsonVector.getAccessor().getValueCount()) {
+                throw new IllegalArgumentException("Different value count for field " + field + " : " + valueCount + " != " + jsonVector.getAccessor().getValueCount());
+            }
+            for (int j = 0; j < valueCount; j++) {
+                Object arrow = arrowVector.getAccessor().getObject(j);
+                Object json = jsonVector.getAccessor().getObject(j);
+                if (!equals(field.getType(), arrow, json)) {
+                    throw new IllegalArgumentException(
+                            "Different values in column:\n" + field + " at index " + j + ": " + arrow + " != " + json);
+                }
+            }
+        }
+    }
+
+    static boolean equals(ArrowType type, final Object o1, final Object o2) {
+        if (type instanceof ArrowType.FloatingPoint) {
+            ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) type;
+            switch (fpType.getPrecision()) {
+                case DOUBLE:
+                    return equalEnough((Double)o1, (Double)o2);
+                case SINGLE:
+                    return equalEnough((Float)o1, (Float)o2);
+                case HALF:
+                default:
+                    throw new UnsupportedOperationException("unsupported precision: " + fpType);
+            }
+        }
+        return Objects.equal(o1, o2);
+    }
+
+    static boolean equalEnough(Float f1, Float f2) {
+        if (f1 == null || f2 == null) {
+            return f1 == null && f2 == null;
+        }
+        if (f1.isNaN()) {
+            return f2.isNaN();
+        }
+        if (f1.isInfinite()) {
+            return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
+        }
+        float average = Math.abs((f1 + f2) / 2);
+        float differenceScaled = Math.abs(f1 - f2) / (average == 0.0f ? 1f : average);
+        return differenceScaled < 1.0E-6f;
+    }
+
+    static boolean equalEnough(Double f1, Double f2) {
+        if (f1 == null || f2 == null) {
+            return f1 == null && f2 == null;
+        }
+        if (f1.isNaN()) {
+            return f2.isNaN();
+        }
+        if (f1.isInfinite()) {
+            return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
+        }
+        double average = Math.abs((f1 + f2) / 2);
+        double differenceScaled = Math.abs(f1 - f2) / (average == 0.0d ? 1d : average);
+        return differenceScaled < 1.0E-12d;
+    }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/9513ca77/java/vector/src/test/java/org/apache/arrow/vector/util/TestValidator.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/util/TestValidator.java b/java/vector/src/test/java/org/apache/arrow/vector/util/TestValidator.java
new file mode 100644
index 0000000..7cf638e
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/util/TestValidator.java
@@ -0,0 +1,57 @@
+/*
+ * 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.arrow.vector.util;
+
+import static org.apache.arrow.vector.util.Validator.equalEnough;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class TestValidator {
+
+    @Test
+    public void testFloatComp() {
+        assertTrue(equalEnough(912.4140000000002F, 912.414F));
+        assertTrue(equalEnough(912.4140000000002D, 912.414D));
+        assertTrue(equalEnough(912.414F, 912.4140000000002F));
+        assertTrue(equalEnough(912.414D, 912.4140000000002D));
+        assertFalse(equalEnough(912.414D, 912.4140001D));
+        assertFalse(equalEnough(null, 912.414D));
+        assertTrue(equalEnough((Float)null, null));
+        assertTrue(equalEnough((Double)null, null));
+        assertFalse(equalEnough(912.414D, null));
+        assertFalse(equalEnough(Double.MAX_VALUE, Double.MIN_VALUE));
+        assertFalse(equalEnough(Double.MIN_VALUE, Double.MAX_VALUE));
+        assertTrue(equalEnough(Double.MAX_VALUE, Double.MAX_VALUE));
+        assertTrue(equalEnough(Double.MIN_VALUE, Double.MIN_VALUE));
+        assertTrue(equalEnough(Double.NEGATIVE_INFINITY, Double.NEGATIVE_INFINITY));
+        assertFalse(equalEnough(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY));
+        assertTrue(equalEnough(Double.NaN, Double.NaN));
+        assertFalse(equalEnough(1.0, Double.NaN));
+        assertFalse(equalEnough(Float.MAX_VALUE, Float.MIN_VALUE));
+        assertFalse(equalEnough(Float.MIN_VALUE, Float.MAX_VALUE));
+        assertTrue(equalEnough(Float.MAX_VALUE, Float.MAX_VALUE));
+        assertTrue(equalEnough(Float.MIN_VALUE, Float.MIN_VALUE));
+        assertTrue(equalEnough(Float.NEGATIVE_INFINITY, Float.NEGATIVE_INFINITY));
+        assertFalse(equalEnough(Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY));
+        assertTrue(equalEnough(Float.NaN, Float.NaN));
+        assertFalse(equalEnough(1.0F, Float.NaN));
+    }
+}