You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mb...@apache.org on 2016/09/24 14:48:22 UTC

asterixdb git commit: Add upsert option for feed

Repository: asterixdb
Updated Branches:
  refs/heads/master a1eb7533f -> 0608bf58f


Add upsert option for feed

For ASTERIXDB-1567. Provide "upsert-feed"="true" for
feed configuration, which changes the default record
insert to upsert. Added one test case for upsert feed.

Change-Id: Ic5133e7c6941fea4110cc9983f99502f364dc810
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1068
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>


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

Branch: refs/heads/master
Commit: 0608bf58feff796865833f098038e6fbef3e5166
Parents: a1eb753
Author: Michael Blow <mb...@apache.org>
Authored: Sat Sep 24 00:29:23 2016 -0400
Committer: Michael Blow <mb...@apache.org>
Committed: Sat Sep 24 07:47:41 2016 -0700

----------------------------------------------------------------------
 .../LangExpressionToPlanTranslator.java         | 17 ++++++--
 .../asterix-app/data/tinysocial/twu_update.adm  |  8 ++++
 .../feeds/upsert-feed/upsert-feed.1.ddl.aql     | 46 ++++++++++++++++++++
 .../feeds/upsert-feed/upsert-feed.2.update.aql  | 29 ++++++++++++
 .../feeds/upsert-feed/upsert-feed.3.server.aql  | 26 +++++++++++
 .../feeds/upsert-feed/upsert-feed.4.sleep.aql   | 26 +++++++++++
 .../feeds/upsert-feed/upsert-feed.5.update.aql  | 28 ++++++++++++
 .../feeds/upsert-feed/upsert-feed.6.query.aql   | 31 +++++++++++++
 .../feeds/upsert-feed/upsert-feed.7.server.aql  | 27 ++++++++++++
 .../feeds/upsert-feed/upsert-feed.8.ddl.aql     | 27 ++++++++++++
 .../queries/records/RecordsQueries.xml          |  5 +++
 .../results/feeds/upsert-feed/upsert-feed.1.adm |  6 +++
 .../src/test/resources/runtimets/testsuite.xml  | 12 +++--
 .../external/util/ExternalDataConstants.java    |  2 +
 .../external/util/ExternalDataUtils.java        |  4 ++
 .../metadata/feeds/FeedMetadataUtil.java        |  8 ----
 16 files changed, 283 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 3a0629d..09a0476 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -37,6 +37,7 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -81,9 +82,9 @@ import org.apache.asterix.metadata.declared.ResultSetDataSink;
 import org.apache.asterix.metadata.declared.ResultSetSinkId;
 import org.apache.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
 import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.asterix.metadata.functions.ExternalFunctionCompilerUtil;
 import org.apache.asterix.metadata.utils.DatasetUtils;
 import org.apache.asterix.om.base.AInt64;
@@ -451,8 +452,10 @@ class LangExpressionToPlanTranslator
         List<LogicalVariable> metaAndKeysVars = null;
         List<Mutable<ILogicalExpression>> metaAndKeysExprs = null;
         List<Mutable<ILogicalExpression>> metaExpSingletonList = null;
-        boolean isChangeFeed = FeedMetadataUtil.isChangeFeed(metadataProvider, sfs.getDataverseName(),
-                sfs.getFeedName());
+        Feed feed = metadataProvider.findFeed(sfs.getDataverseName(), sfs.getFeedName());
+        boolean isChangeFeed = ExternalDataUtils.isChangeFeed(feed.getAdapterConfiguration());
+        boolean isUpsertFeed = ExternalDataUtils.isUpsertFeed(feed.getAdapterConfiguration());
+
         if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
             metaAndKeysVars = new ArrayList<>();
             metaAndKeysExprs = new ArrayList<>();
@@ -509,8 +512,14 @@ class LangExpressionToPlanTranslator
                 feedModificationOp.getInputs().add(assign.getInputs().get(0));
             }
         } else {
+            final InsertDeleteUpsertOperator.Kind opKind = isUpsertFeed ? InsertDeleteUpsertOperator.Kind.UPSERT
+                    : InsertDeleteUpsertOperator.Kind.INSERT;
             feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
-                    metaExpSingletonList, InsertDeleteUpsertOperator.Kind.INSERT, false);
+                    metaExpSingletonList, opKind, false);
+            if (isUpsertFeed) {
+                feedModificationOp.setPrevRecordVar(context.newVar());
+                feedModificationOp.setPrevRecordType(targetDatasource.getItemType());
+            }
             feedModificationOp.getInputs().add(new MutableObject<>(assign));
         }
         if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/data/tinysocial/twu_update.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/data/tinysocial/twu_update.adm b/asterixdb/asterix-app/data/tinysocial/twu_update.adm
new file mode 100644
index 0000000..477d08d
--- /dev/null
+++ b/asterixdb/asterix-app/data/tinysocial/twu_update.adm
@@ -0,0 +1,8 @@
+{"screen-name":"NathanGiesen@211","lang":"en","friends_count":18,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416}
+{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159}
+{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649}
+{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136}
+{"screen-name":"NathanGiesen@211","lang":"en","friends_count":18,"statuses_count":473,"name":"Nathan Giesen","followers_count":4}
+{"screen-name":"Farrel@64","lang":"en","friends_count":42,"statuses_count":13,"name":"Will Farrel","followers_count":1422}
+{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":455,"statuses_count":164,"name":"Nila Milliron","followers_count":1}
+{"screen-name":"Zeed","lang":"en","friends_count":32,"statuses_count":13,"name":"Steven Zeed","followers_count":331}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql
new file mode 100644
index 0000000..14b857f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use dataverse experiments;
+
+create type TwitterUser if not exists as open{
+    screen-name: string,
+    friends_count: int32,
+    name: string,
+    followers_count: int32
+};
+
+create dataset TwitterUsers(TwitterUser) primary key screen-name;
+
+create feed UserFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TwitterUser"),
+    ("format"="adm"),
+    ("upsert-feed"="true")
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
new file mode 100644
index 0000000..051f266
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+use dataverse experiments;
+set wait-for-completion-feed "false";
+
+connect feed UserFeed to dataset TwitterUsers;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
new file mode 100644
index 0000000..3da77f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
@@ -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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+start client 10001 file-client localhost ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql
new file mode 100644
index 0000000..99ad0d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql
@@ -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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+10000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
new file mode 100644
index 0000000..8ad18ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+use dataverse experiments;
+disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql
new file mode 100644
index 0000000..05d0cee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+use dataverse experiments;
+
+for $x in dataset TwitterUsers
+order by $x.screen-name
+return $x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql
new file mode 100644
index 0000000..d331474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql
new file mode 100644
index 0000000..d556f2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+use dataverse experiments;
+drop dataverse experiments;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/queries/records/RecordsQueries.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/records/RecordsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries/records/RecordsQueries.xml
index bfe8fa7..60ee26c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/records/RecordsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/records/RecordsQueries.xml
@@ -176,4 +176,9 @@
       <output-dir compare="Text">highly-nested-open</output-dir>
     </compilation-unit>
   </test-case>
+  <test-case FilePath="records">
+    <compilation-unit name="closed-nullable-fields_issue1616">
+      <output-dir compare="Text">closed-nullable-fields_issue1616</output-dir>
+    </compilation-unit>
+  </test-case>
 </test-group>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm
new file mode 100644
index 0000000..2d0a865
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm
@@ -0,0 +1,6 @@
+{ "screen-name": "ChangEwing_573", "friends_count": 182, "name": "Chang Ewing", "followers_count": 32136, "lang": "en", "statuses_count": 394 }
+{ "screen-name": "ColineGeyer@63", "friends_count": 121, "name": "Coline Geyer", "followers_count": 17159, "lang": "en", "statuses_count": 362 }
+{ "screen-name": "Farrel@64", "friends_count": 42, "name": "Will Farrel", "followers_count": 1422, "lang": "en", "statuses_count": 13 }
+{ "screen-name": "NathanGiesen@211", "friends_count": 18, "name": "Nathan Giesen", "followers_count": 4, "lang": "en", "statuses_count": 473 }
+{ "screen-name": "NilaMilliron_tw", "friends_count": 455, "name": "Nila Milliron", "followers_count": 1, "lang": "en", "statuses_count": 164 }
+{ "screen-name": "Zeed", "friends_count": 32, "name": "Steven Zeed", "followers_count": 331, "lang": "en", "statuses_count": 13 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 2f277cc..a244a74 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -214,6 +214,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
+      <compilation-unit name="upsert-feed">
+        <output-dir compare="Text">upsert-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
       <compilation-unit name="feeds_01">
         <output-dir compare="Text">feeds_01</output-dir>
       </compilation-unit>
@@ -4822,13 +4827,6 @@
   </test-group>
   &RecordsQueries;
   &DeepEqualQueries;
-  <test-group name="records">
-      <test-case FilePath="records">
-      <compilation-unit name="closed-nullable-fields_issue1616">
-        <output-dir compare="Text">closed-nullable-fields_issue1616</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
   <test-group name="scan">
     <test-case FilePath="scan">
       <compilation-unit name="10">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index b666487..c5167c1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -93,6 +93,8 @@ public class ExternalDataConstants {
     public static final String KEY_VALUE_FORMAT = "value-format";
     // a boolean indicating whether the feed is a change feed
     public static final String KEY_IS_CHANGE_FEED = "change-feed";
+    // a boolean indicating whether the feed use upsert
+    public static final String KEY_IS_UPSERT_FEED = "upsert-feed";
     // an integer representing the number of keys in a change feed
     public static final String KEY_KEY_SIZE = "key-size";
     // a boolean indicating whether the feed produces records with metadata

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 23cd39c..8eb8815 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -296,6 +296,10 @@ public class ExternalDataUtils {
         return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_IS_CHANGE_FEED));
     }
 
+    public static boolean isUpsertFeed(Map<String, String> configuration) {
+        return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_IS_UPSERT_FEED));
+    }
+
     public static int getNumberOfKeys(Map<String, String> configuration) throws AsterixException {
         String keyIndexes = configuration.get(ExternalDataConstants.KEY_KEY_INDEXES);
         if (keyIndexes == null) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0608bf58/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index f91ab8e..a6d5c48 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -50,7 +50,6 @@ import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.Datatype;
@@ -564,11 +563,4 @@ public class FeedMetadataUtil {
         }
         return outputType;
     }
-
-    public static boolean isChangeFeed(AqlMetadataProvider mdProvider, String dataverse, String feedName)
-            throws AlgebricksException {
-        Feed feed = mdProvider.findFeed(dataverse, feedName);
-        return ExternalDataUtils.isChangeFeed(feed.getAdapterConfiguration());
-    }
-
 }