You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2020/11/02 19:26:04 UTC

[asterixdb] 01/02: [NO ISSUE][COMP] Prohibit empty names of database objects

This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit 95fa33cbf348bce64534d1dcb5a1089d11ecc515
Author: Dmitry Lychagin <dm...@couchbase.com>
AuthorDate: Tue Oct 20 14:23:35 2020 -0700

    [NO ISSUE][COMP] Prohibit empty names of database objects
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - Disallow creation of database objects with empty names
      or names that start with a space character
    
    Change-Id: I6b5f99eec54b0882d05a0afdf925f9f6e56f0f00
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/8484
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../asterix/app/translator/QueryTranslator.java    | 18 +++++++
 .../invalid-dataset-name.1.ddl.sqlpp               | 29 +++++++++++
 .../invalid-dataset-name.2.ddl.sqlpp               | 29 +++++++++++
 .../invalid-dataset-name.3.ddl.sqlpp               | 20 ++++++++
 .../invalid-dataverse-name.1.ddl.sqlpp             | 24 ++++++++++
 .../invalid-dataverse-name.2.ddl.sqlpp             | 24 ++++++++++
 .../invalid-feed-name.1.ddl.sqlpp                  | 42 ++++++++++++++++
 .../invalid-feed-name.2.ddl.sqlpp                  | 42 ++++++++++++++++
 .../invalid-feed-name.3.ddl.sqlpp                  | 20 ++++++++
 .../invalid-feed-policy-name.1.ddl.sqlpp           | 30 ++++++++++++
 .../invalid-feed-policy-name.2.ddl.sqlpp           | 30 ++++++++++++
 .../invalid-feed-policy-name.3.ddl.sqlpp           | 20 ++++++++
 .../invalid-index-name.1.ddl.sqlpp                 | 31 ++++++++++++
 .../invalid-index-name.2.ddl.sqlpp                 | 31 ++++++++++++
 .../invalid-index-name.3.ddl.sqlpp                 | 20 ++++++++
 .../invalid-nodegroup-name.1.ddl.sqlpp             | 24 ++++++++++
 .../invalid-nodegroup-name.2.ddl.sqlpp             | 24 ++++++++++
 .../invalid-type-name.1.ddl.sqlpp                  | 31 ++++++++++++
 .../invalid-type-name.2.ddl.sqlpp                  | 31 ++++++++++++
 .../invalid-type-name.3.ddl.sqlpp                  | 20 ++++++++
 .../invalid-udf-name/invalid-udf-name.1.ddl.sqlpp  | 31 ++++++++++++
 .../invalid-udf-name/invalid-udf-name.2.ddl.sqlpp  | 31 ++++++++++++
 .../invalid-udf-name/invalid-udf-name.3.ddl.sqlpp  | 20 ++++++++
 .../test/resources/runtimets/testsuite_sqlpp.xml   | 56 ++++++++++++++++++++++
 .../asterix/common/exceptions/ErrorCode.java       |  1 +
 .../src/main/resources/asx_errormsg/en.properties  |  1 +
 26 files changed, 680 insertions(+)

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index d49996a..d29a5f5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -491,7 +491,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt,
             IRequestParameters requestParameters) throws Exception {
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
+        SourceLocation sourceLoc = stmtCreateDataverse.getSourceLocation();
         String dvName = stmtCreateDataverse.getDataverseName().getValue();
+        validateDatabaseObjectName(dvName, sourceLoc);
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
@@ -569,6 +571,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         SourceLocation sourceLoc = dd.getSourceLocation();
         String dataverseName = getActiveDataverse(dd.getDataverse());
         String datasetName = dd.getName().getValue();
+        validateDatabaseObjectName(datasetName, sourceLoc);
         DatasetType dsType = dd.getDatasetType();
         TypeExpression itemTypeExpr = dd.getItemType();
         String itemTypeDataverseName, itemTypeName;
@@ -911,6 +914,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
         String indexName = stmtCreateIndex.getIndexName().getValue();
+        validateDatabaseObjectName(indexName, sourceLoc);
         IndexType indexType = stmtCreateIndex.getIndexType();
         List<Integer> keySourceIndicators = stmtCreateIndex.getFieldSourceIndicators();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1344,6 +1348,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         SourceLocation sourceLoc = stmtCreateType.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
         String typeName = stmtCreateType.getIdent().getValue();
+        validateDatabaseObjectName(typeName, sourceLoc);
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         MetadataLockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -1932,6 +1937,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
         SourceLocation sourceLoc = cfs.getSourceLocation();
         FunctionSignature signature = cfs.getFunctionSignature();
+        validateDatabaseObjectName(signature.getName(), sourceLoc);
         String dataverse = getActiveDataverseName(signature.getNamespace());
         signature.setNamespace(dataverse);
 
@@ -2235,6 +2241,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String feedName = cfs.getFeedName().getValue();
+        validateDatabaseObjectName(feedName, sourceLoc);
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         MetadataLockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -2276,6 +2283,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         SourceLocation sourceLoc = cfps.getSourceLocation();
         dataverse = getActiveDataverse(null);
         policy = cfps.getPolicyName();
+        validateDatabaseObjectName(policy, sourceLoc);
         MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverse,
                 dataverse + "." + policy);
         try {
@@ -2872,6 +2880,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
         SourceLocation sourceLoc = stmtCreateNodegroup.getSourceLocation();
         String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
+        validateDatabaseObjectName(ngName, sourceLoc);
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -3281,4 +3290,13 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             throws CompilationException {
         ExternalDataUtils.validateAdapterSpecificProperties(configuration, srcLoc, warningCollector);
     }
+
+    public static void validateDatabaseObjectName(String name, SourceLocation sourceLoc) throws CompilationException {
+        if (name == null || name.isEmpty()) {
+            throw new CompilationException(ErrorCode.INVALID_DATABASE_OBJECT_NAME, sourceLoc, "<empty>");
+        }
+        if (Character.isWhitespace(name.codePointAt(0))) {
+            throw new CompilationException(ErrorCode.INVALID_DATABASE_OBJECT_NAME, sourceLoc, name);
+        }
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.1.ddl.sqlpp
new file mode 100644
index 0000000..f5c4588
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Empty dataset name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ``(id int not unknown) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.2.ddl.sqlpp
new file mode 100644
index 0000000..1bc445a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.2.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Dataset name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ` a`(id int not unknown) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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 test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.1.ddl.sqlpp
new file mode 100644
index 0000000..7322b6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.1.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/*
+ * Empty dataverse name -> Error
+ */
+
+create dataverse ``;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.2.ddl.sqlpp
new file mode 100644
index 0000000..623e40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.2.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/*
+ * Dataverse name starts with a space character -> Error
+ */
+
+create dataverse ` a`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.1.ddl.sqlpp
new file mode 100644
index 0000000..783a940
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+/*
+ * Empty feed name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type Tweet as closed
+{
+  id: int64
+};
+
+create dataset Tweets0(Tweet) primary key id;
+
+create feed `` with {
+ "adapter-name" : "socket_adapter",
+ "sockets" : "127.0.0.1:10001",
+ "address-type" : "IP",
+ "type-name" : "Tweet",
+ "format" : "adm"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.2.ddl.sqlpp
new file mode 100644
index 0000000..6d59d03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.2.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+/*
+ * Feed name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type Tweet as closed
+{
+  id: int64
+};
+
+create dataset Tweets0(Tweet) primary key id;
+
+create feed ` a` with {
+ "adapter-name" : "socket_adapter",
+ "sockets" : "127.0.0.1:10001",
+ "address-type" : "IP",
+ "type-name" : "Tweet",
+ "format" : "adm"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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 test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.1.ddl.sqlpp
new file mode 100644
index 0000000..3a9f7fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Empty feed policy name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create ingestion policy `` from path 'data/feed-policy/policy.properties'
+  definition 'someString';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.2.ddl.sqlpp
new file mode 100644
index 0000000..c5d3179
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.2.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Feed policy name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create ingestion policy ` a` from path 'data/feed-policy/policy.properties'
+  definition 'someString';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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 test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.1.ddl.sqlpp
new file mode 100644
index 0000000..ffad79e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Empty index name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id int not unknown, x int not unknown) primary key id;
+
+create index `` on t1(x);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.2.ddl.sqlpp
new file mode 100644
index 0000000..6e0bd7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.2.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Index name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id int not unknown, x int not unknown) primary key id;
+
+create index ` a` on t1(x);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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 test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.1.ddl.sqlpp
new file mode 100644
index 0000000..ad1de03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.1.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/*
+ * Empty node group name -> Error
+ */
+
+create nodegroup `` if not exists on asterix_nc1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.2.ddl.sqlpp
new file mode 100644
index 0000000..315d287
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.2.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/*
+ * Node group name starts with a space character -> Error
+ */
+
+create nodegroup ` a` if not exists on asterix_nc1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.1.ddl.sqlpp
new file mode 100644
index 0000000..ddaac27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Empty type name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type `` as closed {
+  id: int
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.2.ddl.sqlpp
new file mode 100644
index 0000000..5ec9f83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.2.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Type name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type ` a` as closed {
+  id: int
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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 test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.1.ddl.sqlpp
new file mode 100644
index 0000000..0c9b27d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Empty function name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create function ``() {
+  1
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.2.ddl.sqlpp
new file mode 100644
index 0000000..fb6e8e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.2.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Function name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create function ` a`() {
+  1
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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 test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 6d50376..032097a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3911,6 +3911,62 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="ddl">
+      <compilation-unit name="invalid-dataverse-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-dataset-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-feed-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-feed-policy-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-index-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-nodegroup-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-type-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-udf-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: "&lt;empty&gt;"</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
       <compilation-unit name="dataset-and-index-same-dataverse">
         <output-dir compare="Text">dataset-and-index-same-dataverse</output-dir>
       </compilation-unit>
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 1d18cf6..21c3416 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
@@ -203,6 +203,7 @@ public class ErrorCode {
     public static final int INVALID_PROPERTY_FORMAT = 1112;
     public static final int INVALID_REGEX_PATTERN = 1113;
     public static final int EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES = 1114;
+    public static final int INVALID_DATABASE_OBJECT_NAME = 1115;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
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 a2780ba..5012c7a 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -198,6 +198,7 @@
 1112 = Invalid format for property \"%1$s\"
 1113 = Invalid pattern %1$s
 1114 = The provided external dataset configuration returned no files from the external source
+1115 = Invalid name for a database object: \"%1$s\"
 
 # Feed Errors
 3001 = Illegal state.