>From Dmitry Lychagin <[email protected]>: Dmitry Lychagin has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187 )
Change subject: [ASTERIXDB-2838][RT][FUN] Batched PyUDF calls ...................................................................... Patch Set 16: (16 comments) https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.0.ddl.sqlpp File asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.0.ddl.sqlpp: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.0.ddl.sqlpp@29 PS16, Line 29: load dataset Tweet using localfs(("path"="asterix_nc1://data/twitter/real.adm"),("format"="adm")); we usually put load statement into a separate, .update.sqlpp file. https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.10.ddl.sqlpp File asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.10.ddl.sqlpp: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.10.ddl.sqlpp@23 PS16, Line 23: as "sentiment", "TweetSent.sentiment" at testlib with {"null-call": "true"}; minor. "null-call" value could also be a boolean. Do we have a testcase for it? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.5.query.sqlpp File asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.5.query.sqlpp: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.5.query.sqlpp@23 PS16, Line 23: limit 100; need ORDER BY clause to establish stable order of the results. https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.6.query.sqlpp File asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.6.query.sqlpp: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.6.query.sqlpp@25 PS16, Line 25: limit 100; need ORDER BY clause. https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.7.update.sqlpp File asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.7.update.sqlpp: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.7.update.sqlpp@29 PS16, Line 29: { "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 31, "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 31, "name": "Asia✨02/03❤️‼️", "screen_name": "AsiaCarlton", "lang": "en", "location": "Houston, TX", "create_at": date("2013-02-17"), "description": "I'm A Mf'Kn Queen ❣✊ #freenick", "followers_count": 1083, "friends_count": 924, "statues_count": 6067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }, We can't have this kind of language in our tests. Why can't we reuse some of the existing twitter data we already have in asterix-app/data/twitter? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java File asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java@141 PS16, Line 141: throw new HyracksDataException(e.getMessage()); use HyrackDataException.create() https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java File asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@103 PS16, Line 103: argHolders = Arrays.stream(fnArgColumns).map(i -> ByteBuffer.wrap(new byte[ctx.getInitialFrameSize()])) why do you not use ctx.allocateFrame()? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@105 PS16, Line 105: outputWrapper = ByteBuffer.wrap(new byte[ctx.getInitialFrameSize()]); why do you not use ctx.allocateFrame()? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@113 PS16, Line 113: nullCalls[func] = new ATypeTag[numTuples]; can we avoid array creation here (array per frame) and reuse/extend existing array? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@139 PS16, Line 139: try (MessageUnpacker errorUnpacker = MessagePack.newDefaultUnpacker(result.getFirst())) { MessageUnpacker has reset() method. Can we have one instance and reset it, instead of allocating new instance each time here? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@167 PS16, Line 167: for (int colIdx = 0; colIdx < cols.length; colIdx++) { If fnDescs[func].getFunctionInfo().getNullCall()==true then you don't even need to bother with this loop, right? Can you move this check out of handleNullMatrix() and put it here? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@191 PS16, Line 191: //TODO: maybe this could be done in parallel for each unique library evaluator? we shouldn't start any threads here, but may be we could somehow introduce an async API for callPythonMulti(), so we would submit all requests first, then wait for the results from each library evaluator. https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@192 PS16, Line 192: List<Pair<ByteBuffer, Counter>> batchResults = new ArrayList<>(argHolders.size()); can we avoid object creation here? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@200 PS16, Line 200: batchResults.add(new Pair<>(columnResult, new Counter(numResults))); avoid object creation? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java@202 PS16, Line 202: batchResults.add(new Pair<>(null, new Counter(-1))); avoid object creation? https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java File asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187/16/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java@33 PS16, Line 33: private static final long serialVersionUID = 4L; increase serialVersionUID? -- To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10187 To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings Gerrit-Project: asterixdb Gerrit-Branch: cheshire-cat Gerrit-Change-Id: I5af4da999985afcc33cdfacea79576f1d6109173 Gerrit-Change-Number: 10187 Gerrit-PatchSet: 16 Gerrit-Owner: Ian Maxon <[email protected]> Gerrit-Reviewer: Anon. E. Moose #1000171 Gerrit-Reviewer: Dmitry Lychagin <[email protected]> Gerrit-Reviewer: Ian Maxon <[email protected]> Gerrit-Reviewer: Jenkins <[email protected]> Gerrit-Reviewer: Michael Blow <[email protected]> Gerrit-Comment-Date: Wed, 10 Mar 2021 01:50:34 +0000 Gerrit-HasComments: Yes Gerrit-Has-Labels: No Gerrit-MessageType: comment
