You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by xi...@apache.org on 2017/10/23 17:28:47 UTC

asterixdb git commit: [ISSUE PENDING][FUN] Prevent blind result write in external UDF

Repository: asterixdb
Updated Branches:
  refs/heads/master 8de799db3 -> 0e5990369


[ISSUE PENDING][FUN] Prevent blind result write in external UDF

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
Current the result from UDF doesn't have type check. It's possible to
write data into result frame with a different datatype. This will cause
unpredicted behavior when reading the data. Also, to keep things
simpler, when return data type is not valid, we throw exception instead
of return null.

Change-Id: Ifce4a9e882febe4529f5c53e54db78e0cfa326fe
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2082
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Steven Jacobs <sj...@ucr.edu>


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

Branch: refs/heads/master
Commit: 0e59903690d264e018d71f895b0c60db34cbbdfa
Parents: 8de799d
Author: Xikui Wang <xk...@gmail.com>
Authored: Sun Oct 22 23:18:47 2017 -0700
Committer: Xikui Wang <xk...@gmail.com>
Committed: Mon Oct 23 10:28:29 2017 -0700

----------------------------------------------------------------------
 .../return_invalid_type.1.ddl.sqlpp             | 63 ++++++++++++++++++++
 .../return_invalid_type.2.lib.sqlpp             | 19 ++++++
 .../return_invalid_type.3.update.sqlpp          | 26 ++++++++
 .../return_invalid_type.4.query.sqlpp           | 23 +++++++
 .../resources/runtimets/testsuite_it_sqlpp.xml  |  6 ++
 .../asterix/common/exceptions/ErrorCode.java    |  2 +
 .../main/resources/asx_errormsg/en.properties   |  2 +
 .../external/generator/DataGenerator.java       |  2 +-
 .../library/ExternalFunctionProvider.java       | 11 +---
 .../external/library/JavaFunctionHelper.java    | 15 ++++-
 .../external/library/AddHashTagsFunction.java   |  1 -
 11 files changed, 157 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.ddl.sqlpp
new file mode 100644
index 0000000..e375558
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.ddl.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+drop dataverse externallibtest if exists;
+create dataverse externallibtest;
+use externallibtest;
+
+create type TwitterUserType as closed {
+ screen_name: string,
+ language: string,
+ friends_count: int32,
+ status_count: int32,
+ name: string,
+ followers_count: int32
+};
+
+create type Tweet as closed {
+ id: int64,
+ user: TwitterUserType,
+ latitude: double,
+ longitude: double,
+ message_text: string,
+ created_at: string,
+ country: string
+};
+
+create type ProcessedTweet if not exists as open {
+    id: int64,
+    user_name:string,
+    location:point,
+    created_at:string,
+    message_text:string,
+    country: string,
+    topics: {{string}}
+};
+
+create dataset SyntheticTweets(Tweet) primary key id;
+
+create feed  SyntheticTweetFeed using stream
+(("duration"="1"),
+("stream-source"="org.apache.asterix.external.input.stream.factory.TwitterFirehoseStreamFactory"),
+("tps"="50"),
+("type-name"="Tweet"),
+("tput-duration"="1"),
+("dataverse-dataset"="feeds:SyntheticTweets"),
+("format"="adm"),
+("mode"="controlled"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.2.lib.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.2.lib.sqlpp
new file mode 100644
index 0000000..d1e0e87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.2.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install externallibtest testlib target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.3.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.3.update.sqlpp
new file mode 100644
index 0000000..be057f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.3.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use externallibtest;
+
+set `wait-for-completion-feed` `true`;
+
+connect feed SyntheticTweetFeed to dataset SyntheticTweets;
+
+start feed SyntheticTweetFeed;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.4.query.sqlpp
new file mode 100644
index 0000000..59ea2c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.4.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use externallibtest;
+
+SELECT testlib#addHashTagsInPlace(t) FROM SyntheticTweets t;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
index 8f4455f..edba869 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
@@ -48,5 +48,11 @@
         <output-dir compare="Text">feed-with-external-function</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="return_invalid_type">
+        <output-dir compare="Text">getCapital</output-dir>
+        <expected-error>External UDF cannot produce expected result.</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 279624d..895cd55 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -74,6 +74,8 @@ public class ErrorCode {
 
     public static final int INSTANTIATION_ERROR = 100;
 
+    public static final int EXTERNAL_UDF_RESULT_TYPE_ERROR = 200;
+
     // Compilation errors
     public static final int PARSE_ERROR = 1001;
     public static final int COMPILATION_TYPE_MISMATCH = 1002;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 6ce78f0..98b8e2f 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -67,6 +67,8 @@
 
 100 = Unable to instantiate class %1$s
 
+200 = External UDF cannot produce expected result. Please check the UDF configuration
+
 # Compile-time check errors
 1007 = Invalid expression: function %1$s expects its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s
 1008 = Invalid parameter number: function %1$s cannot take %2$s parameters

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/generator/DataGenerator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/generator/DataGenerator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/generator/DataGenerator.java
index c2eca84..9bc3037 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/generator/DataGenerator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/generator/DataGenerator.java
@@ -421,7 +421,7 @@ public class DataGenerator {
             buffer.append(" ");
             String jargonTerm = jargon.get(random.nextInt(jargon.size()));
             referredTopics.add(jargonTerm);
-            buffer.append(jargonTerm);
+            buffer.append("#" + jargonTerm);
             buffer.append(" is ");
             buffer.append(adjective);
             if (random.nextBoolean()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
index 7602b9a..9e8b9b7 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
@@ -25,7 +25,6 @@ import org.apache.asterix.external.api.IExternalFunction;
 import org.apache.asterix.external.api.IExternalScalarFunction;
 import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -79,14 +78,8 @@ class ExternalScalarFunction extends ExternalFunction implements IExternalScalar
         try {
             resultBuffer.reset();
             ((IExternalScalarFunction) externalFunction).evaluate(argumentProvider);
-            /*
-             * Make sure that if "setResult" is not called,
-             * or the result object is missing we let Hyracks storage manager know
-             * we want to discard a missing object
-             */
-            byte byteOutput = resultBuffer.getByteArray()[0];
-            if (!argumentProvider.isValidResult() || byteOutput == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
-                resultBuffer.getDataOutput().writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+            if (!argumentProvider.isValidResult()) {
+                throw new RuntimeDataException(ErrorCode.EXTERNAL_UDF_RESULT_TYPE_ERROR);
             }
         } catch (Exception e) {
             throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
index 3006454..a369888 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
@@ -28,6 +28,7 @@ import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.api.IFunctionHelper;
 import org.apache.asterix.external.api.IJObject;
 import org.apache.asterix.external.library.java.JObjectPointableVisitor;
+import org.apache.asterix.external.library.java.JObjects;
 import org.apache.asterix.external.library.java.JObjects.JNull;
 import org.apache.asterix.external.library.java.JTypeTag;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
@@ -36,6 +37,7 @@ import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.container.IObjectPool;
@@ -81,8 +83,7 @@ public class JavaFunctionHelper implements IFunctionHelper {
 
     @Override
     public void setResult(IJObject result) throws HyracksDataException {
-        if (result == null) {
-            JNull.INSTANCE.serialize(outputProvider.getDataOutput(), true);
+        if (result == null || checkInvalidReturnValueType(result, finfo.getReturnType())) {
             isValidResult = false;
         } else {
             isValidResult = true;
@@ -91,6 +92,16 @@ public class JavaFunctionHelper implements IFunctionHelper {
         }
     }
 
+    private boolean checkInvalidReturnValueType(IJObject result, IAType expectedType) {
+        if (expectedType.getTypeTag() != result.getTypeTag()) {
+            return true;
+        }
+        if (expectedType.getTypeTag() == ATypeTag.OBJECT) {
+            return !expectedType.getTypeName().equals(((JObjects.JRecord) result).getRecordType().getTypeName());
+        }
+        return false;
+    }
+
     /**
      * Gets the value of the result flag
      *

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0e599036/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
index 96a40fb..3c6881a 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/AddHashTagsFunction.java
@@ -78,7 +78,6 @@ public class AddHashTagsFunction implements IExternalScalarFunction {
         outputRecord.setField(Datatypes.Tweet.MESSAGE, text);
         outputRecord.setField(Datatypes.ProcessedTweet.TOPICS, list);
 
-        inputRecord.addField(Datatypes.ProcessedTweet.TOPICS, list);
         functionHelper.setResult(outputRecord);
     }