You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by ti...@apache.org on 2016/07/15 04:42:17 UTC

vxquery git commit: some array access cleanup

Repository: vxquery
Updated Branches:
  refs/heads/master 4670e80f7 -> 247300977


some array access cleanup

- extract common code in SequencePointable and ArrayPointable to
  AbstractSequencePointable
- add code to extract sequences from arrays to ArrayPointable and use it
  in JnMembersScalarEvaluator and KeysOrMembersScalarEvaluator
- reduce complexity of KeysOrMembersScalarEvaluator.evaluate by introducing
  a switch and pulling exception wrapping further out


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

Branch: refs/heads/master
Commit: 2473009772dfec2a1dc920f14785afffe015851a
Parents: 4670e80
Author: Till Westmann <ti...@apache.org>
Authored: Wed Jul 13 21:49:21 2016 -0700
Committer: Till Westmann <ti...@apache.org>
Committed: Thu Jul 14 21:36:22 2016 -0700

----------------------------------------------------------------------
 .../accessors/AbstractSequencePointable.java    | 59 ++++++++++++++++++++
 .../datamodel/accessors/SequencePointable.java  | 46 ++-------------
 .../accessors/jsonitem/ArrayPointable.java      | 48 +++++-----------
 .../json/JnMembersScalarEvaluator.java          | 56 ++++++-------------
 .../json/KeysOrMembersScalarEvaluator.java      | 58 +++++++------------
 5 files changed, 115 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/24730097/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/AbstractSequencePointable.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/AbstractSequencePointable.java b/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/AbstractSequencePointable.java
new file mode 100644
index 0000000..2fdfcef
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/AbstractSequencePointable.java
@@ -0,0 +1,59 @@
+/*
+ * 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.vxquery.datamodel.accessors;
+
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+
+public class AbstractSequencePointable extends AbstractPointable {
+    private static final int ENTRY_COUNT_SIZE = IntegerPointable.TYPE_TRAITS.getFixedLength();
+    private static final int SLOT_SIZE = IntegerPointable.TYPE_TRAITS.getFixedLength();
+
+    public int getEntryCount() {
+        return getEntryCount(bytes, start);
+    }
+
+    protected static int getEntryCount(byte[] bytes, int start) {
+        return IntegerPointable.getInteger(bytes, start);
+    }
+
+    public void getEntry(int idx, IPointable pointer) {
+        int dataAreaOffset = getDataAreaOffset(bytes, start);
+        pointer.set(bytes, dataAreaOffset + getRelativeEntryStartOffset(idx), getEntryLength(idx));
+    }
+
+    static int getSlotValue(byte[] bytes, int start, int idx) {
+        return IntegerPointable.getInteger(bytes, getSlotArrayOffset(start) + idx * SLOT_SIZE);
+    }
+
+    private int getRelativeEntryStartOffset(int idx) {
+        return idx == 0 ? 0 : getSlotValue(bytes, start, idx - 1);
+    }
+
+    private int getEntryLength(int idx) {
+        return getSlotValue(bytes, start, idx) - getRelativeEntryStartOffset(idx);
+    }
+
+    private static int getSlotArrayOffset(int start) {
+        return start + ENTRY_COUNT_SIZE;
+    }
+
+    static int getDataAreaOffset(byte[] bytes, int start) {
+        return getSlotArrayOffset(start) + getEntryCount(bytes, start) * SLOT_SIZE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/24730097/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/SequencePointable.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/SequencePointable.java b/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/SequencePointable.java
index 9ccac0b..401f606 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/SequencePointable.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/SequencePointable.java
@@ -17,15 +17,11 @@
 package org.apache.vxquery.datamodel.accessors;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 
-public class SequencePointable extends AbstractPointable {
-    private static final int ENTRY_COUNT_SIZE = IntegerPointable.TYPE_TRAITS.getFixedLength();
-    private static final int SLOT_SIZE = IntegerPointable.TYPE_TRAITS.getFixedLength();
+public class SequencePointable extends AbstractSequencePointable {
     public static final IPointableFactory FACTORY = new IPointableFactory() {
         private static final long serialVersionUID = 1L;
 
@@ -41,40 +37,8 @@ public class SequencePointable extends AbstractPointable {
     };
 
     public static int getSequenceLength(byte[] bytes, int start) {
-        int entryCount = getEntryCount(bytes, start);
-        return getSlotValue(bytes, start, entryCount - 1) + (getDataAreaOffset(bytes, start) - start);
+        int entryCount = AbstractSequencePointable.getEntryCount(bytes, start);
+        return AbstractSequencePointable.getSlotValue(bytes, start, entryCount - 1)
+                + (AbstractSequencePointable.getDataAreaOffset(bytes, start) - start);
     }
-
-    public int getEntryCount() {
-        return getEntryCount(bytes, start);
-    }
-
-    private static int getEntryCount(byte[] bytes, int start) {
-        return IntegerPointable.getInteger(bytes, start);
-    }
-
-    public void getEntry(int idx, IPointable pointer) {
-        int dataAreaOffset = getDataAreaOffset(bytes, start);
-        pointer.set(bytes, dataAreaOffset + getRelativeEntryStartOffset(idx), getEntryLength(idx));
-    }
-
-    private static int getSlotValue(byte[] bytes, int start, int idx) {
-        return IntegerPointable.getInteger(bytes, getSlotArrayOffset(start) + idx * SLOT_SIZE);
-    }
-
-    private int getRelativeEntryStartOffset(int idx) {
-        return idx == 0 ? 0 : getSlotValue(bytes, start, idx - 1);
-    }
-
-    private int getEntryLength(int idx) {
-        return getSlotValue(bytes, start, idx) - getRelativeEntryStartOffset(idx);
-    }
-
-    private static int getSlotArrayOffset(int start) {
-        return start + ENTRY_COUNT_SIZE;
-    }
-
-    private static int getDataAreaOffset(byte[] bytes, int start) {
-        return getSlotArrayOffset(start) + getEntryCount(bytes, start) * SLOT_SIZE;
-    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/24730097/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/jsonitem/ArrayPointable.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/jsonitem/ArrayPointable.java b/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/jsonitem/ArrayPointable.java
index 462b4d6..306c8d5 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/jsonitem/ArrayPointable.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/jsonitem/ArrayPointable.java
@@ -16,16 +16,17 @@
  */
 package org.apache.vxquery.datamodel.accessors.jsonitem;
 
+import java.io.IOException;
+
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.vxquery.datamodel.accessors.AbstractSequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
 
-public class ArrayPointable extends AbstractPointable {
-    private static final int ENTRY_COUNT_SIZE = IntegerPointable.TYPE_TRAITS.getFixedLength();
-    private static final int SLOT_SIZE = IntegerPointable.TYPE_TRAITS.getFixedLength();
+public class ArrayPointable extends AbstractSequencePointable {
     public static final IPointableFactory FACTORY = new IPointableFactory() {
         private static final long serialVersionUID = 1L;
 
@@ -40,36 +41,13 @@ public class ArrayPointable extends AbstractPointable {
         }
     };
 
-    public int getEntryCount() {
-        return getEntryCount(bytes, start);
-    }
-
-    private static int getEntryCount(byte[] bytes, int start) {
-        return IntegerPointable.getInteger(bytes, start);
-    }
-
-    public void getEntry(int idx, IPointable pointer) {
-        int dataStart = getDataStart(bytes, start);
-        pointer.set(bytes, dataStart + getRelativeEntryStartOffset(idx), getEntryLength(idx));
-    }
-
-    private static int getEntryOffsetValue(byte[] bytes, int start, int idx) {
-        return IntegerPointable.getInteger(bytes, getOffsetsStart(start) + idx * SLOT_SIZE);
-    }
+    private TaggedValuePointable tvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
 
-    private int getRelativeEntryStartOffset(int idx) {
-        return idx == 0 ? 0 : getEntryOffsetValue(bytes, start, idx - 1);
-    }
-
-    private int getEntryLength(int idx) {
-        return getEntryOffsetValue(bytes, start, idx) - getRelativeEntryStartOffset(idx);
-    }
-
-    private static int getOffsetsStart(int start) {
-        return start + ENTRY_COUNT_SIZE;
-    }
-
-    private static int getDataStart(byte[] bytes, int start) {
-        return getOffsetsStart(start) + getEntryCount(bytes, start) * SLOT_SIZE;
+    public void appendItems(SequenceBuilder sb) throws IOException {
+        final int size = getEntryCount();
+        for (int j = 0; j < size; j++) {
+            getEntry(j, tvp);
+            sb.addItem(tvp);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/24730097/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnMembersScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnMembersScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnMembersScalarEvaluator.java
index 5777cbc..7b588c0 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnMembersScalarEvaluator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnMembersScalarEvaluator.java
@@ -34,68 +34,46 @@ import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScal
 
 public class JnMembersScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
     protected final IHyracksTaskContext ctx;
-    private final SequencePointable sp1, sp2;
+    private final SequencePointable sp;
     private final ArrayBackedValueStorage abvs;
     private final SequenceBuilder sb;
     private ArrayPointable ap;
+    private TaggedValuePointable tempTvp;
 
     public JnMembersScalarEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) {
         super(args);
         this.ctx = ctx;
-        sp1 = (SequencePointable) SequencePointable.FACTORY.createPointable();
-        sp2 = (SequencePointable) SequencePointable.FACTORY.createPointable();
+        sp = (SequencePointable) SequencePointable.FACTORY.createPointable();
         abvs = new ArrayBackedValueStorage();
         sb = new SequenceBuilder();
         ap = (ArrayPointable) ArrayPointable.FACTORY.createPointable();
+        tempTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
     }
 
     @Override
     protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
         TaggedValuePointable tvp = args[0];
-        TaggedValuePointable tvp1 = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-        abvs.reset();
-        sb.reset(abvs);
-        if (tvp.getTag() == ValueTag.SEQUENCE_TAG) {
-            TaggedValuePointable tempTvp = ppool.takeOne(TaggedValuePointable.class);
-            try {
-                tvp.getValue(sp1);
-                int size1 = sp1.getEntryCount();
+        try {
+            abvs.reset();
+            sb.reset(abvs);
+            if (tvp.getTag() == ValueTag.SEQUENCE_TAG) {
+                tvp.getValue(sp);
+                int size1 = sp.getEntryCount();
                 for (int i = 0; i < size1; i++) {
-                    sp1.getEntry(i, tempTvp);
+                    sp.getEntry(i, tempTvp);
                     if (tempTvp.getTag() == ValueTag.ARRAY_TAG) {
-                        membersSequence(tempTvp, result, tvp1);
-                    } else {
-                        XDMConstants.setEmptySequence(result);
+                        tempTvp.getValue(ap);
+                        ap.appendItems(sb);
                     }
                 }
-            } finally {
-                ppool.giveBack(tempTvp);
+            } else if (tvp.getTag() == ValueTag.ARRAY_TAG) {
+                tvp.getValue(ap);
+                ap.appendItems(sb);
             }
-        } else if (tvp.getTag() == ValueTag.ARRAY_TAG) {
-            membersSequence(tvp, result, tvp1);
-        } else {
-            XDMConstants.setEmptySequence(result);
-        }
-        try {
             sb.finish();
             result.set(abvs);
         } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
-    public void membersSequence(TaggedValuePointable tvp, IPointable result, TaggedValuePointable tvp1)
-            throws SystemException {
-        tvp.getValue(ap);
-        tvp.getValue(sp2);
-        int size = sp2.getEntryCount();
-        for (int j = 0; j < size; j++) {
-            sp2.getEntry(j, tvp1);
-            try {
-                sb.addItem(tvp1);
-            } catch (IOException e) {
-                throw new SystemException(ErrorCode.SYSE0001, e);
-            }
+            throw new SystemException(ErrorCode.SYSE0001, e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/24730097/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/KeysOrMembersScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/KeysOrMembersScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/KeysOrMembersScalarEvaluator.java
index 81afd94..b19985a 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/KeysOrMembersScalarEvaluator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/KeysOrMembersScalarEvaluator.java
@@ -16,11 +16,12 @@
 */
 package org.apache.vxquery.runtime.functions.json;
 
+import java.io.IOException;
+
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.vxquery.datamodel.accessors.SequencePointable;
 import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
 import org.apache.vxquery.datamodel.accessors.jsonitem.ArrayPointable;
 import org.apache.vxquery.datamodel.accessors.jsonitem.ObjectPointable;
@@ -30,13 +31,10 @@ import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
 
-import java.io.IOException;
-
 public class KeysOrMembersScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
     protected final IHyracksTaskContext ctx;
     private final ObjectPointable op;
     private final ArrayPointable ap;
-    private final SequencePointable sp;
     private final ArrayBackedValueStorage abvs;
     private final SequenceBuilder sb;
     private final TaggedValuePointable tempTvp;
@@ -48,45 +46,31 @@ public class KeysOrMembersScalarEvaluator extends AbstractTaggedValueArgumentSca
         ap = (ArrayPointable) ArrayPointable.FACTORY.createPointable();
         abvs = new ArrayBackedValueStorage();
         sb = new SequenceBuilder();
-        sp = (SequencePointable) SequencePointable.FACTORY.createPointable();
         tempTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
     }
 
     @Override
     protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
-        TaggedValuePointable tvp1 = args[0];
-        if (!((tvp1.getTag() == ValueTag.OBJECT_TAG) || (tvp1.getTag() == ValueTag.ARRAY_TAG))) {
-            throw new SystemException(ErrorCode.FORG0006);
-        }
-        if (tvp1.getTag() == ValueTag.OBJECT_TAG) {
-            try {
-                tvp1.getValue(op);
-                op.getKeys(result);
-            } catch (IOException e) {
-                throw new SystemException(ErrorCode.SYSE0001, e);
-
-            }
-        } else if (tvp1.getTag() == ValueTag.ARRAY_TAG) {
-            abvs.reset();
-            sb.reset(abvs);
-            tvp1.getValue(ap);
-            tvp1.getValue(sp);
-            int size = sp.getEntryCount();
-            for (int i = 0; i < size; i++) {
-                sp.getEntry(i, tempTvp);
-                try {
-                    sb.addItem(tempTvp);
-                } catch (IOException e) {
-                    throw new SystemException(ErrorCode.SYSE0001, e);
-                }
+        final TaggedValuePointable tvp = args[0];
+        try {
+            switch (tvp.getTag()) {
+                case ValueTag.OBJECT_TAG:
+                    tvp.getValue(op);
+                    op.getKeys(result);
+                    break;
+                case ValueTag.ARRAY_TAG:
+                    abvs.reset();
+                    sb.reset(abvs);
+                    tvp.getValue(ap);
+                    ap.appendItems(sb);
+                    sb.finish();
+                    result.set(abvs);
+                    break;
+                default:
+                    throw new SystemException(ErrorCode.FORG0006);
             }
-            try {
-                sb.finish();
-            } catch (IOException e) {
-                throw new SystemException(ErrorCode.SYSE0001, e);
-            }
-            result.set(abvs);
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
         }
     }
-
 }