You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/07/17 01:01:35 UTC

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )


Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................

[NO ISSUE][FUN] Support format in date/time constructors

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

Details:
- Add date/time constructor functions that
  accept format as a second argument
- Add testcases

Change-Id: I746cc04d25bda808094fd25168392772a0fba728
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
25 files changed, 820 insertions(+), 248 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/66/12366/1

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
new file mode 100644
index 0000000..be6b9c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '10/30/2010',
+  '31-12-2020'
+],
+fmt = [
+  'MM/DD/YYYY',
+  'DD-MM-YYYY'
+]
+
+select i, date(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
index 443aa80..6a157f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": date(0),
   "null_3": date(time("01:02:03")),
   "null_4": date([]),
-  "null_5": date({})
+  "null_5": date({}),
+  "null_6": date("12/31/2020", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
new file mode 100644
index 0000000..98b3ee6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '10/30/2010 58-57-10',
+  '31-12-2020 11/58/59'
+],
+fmt = [
+  'MM/DD/YYYY ss-mm-hh',
+  'DD-MM-YYYY hh/mm/ss'
+]
+
+select i, datetime(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
index 1a9f391..082d840 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": datetime(0),
   "null_3": datetime(time("01:02:03")),
   "null_4": datetime([]),
-  "null_5": datetime({})
+  "null_5": datetime({}),
+  "null_6": datetime("1951-12-27T12:20:15Z", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
new file mode 100644
index 0000000..37eba54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '11/58/59',
+  '58-57-10'
+],
+fmt = [
+  'hh/mm/ss',
+  'ss-mm-hh'
+]
+
+select i, time(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
index dbdcbc1..b97ec3e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": time(0),
   "null_3": time(date("2020-01-02")),
   "null_4": time([]),
-  "null_5": time({})
+  "null_5": time({}),
+  "null_6": time("11:58:59", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
new file mode 100644
index 0000000..d1d2f7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": date("2010-10-30") }
+{ "i": 1, "actual": date("2020-12-31") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
new file mode 100644
index 0000000..702522c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": datetime("2010-10-30T10:57:58.000Z") }
+{ "i": 1, "actual": datetime("2020-12-31T11:58:59.000Z") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
new file mode 100644
index 0000000..d854daa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": time("11:58:59.000Z") }
+{ "i": 1, "actual": time("10:57:58.000Z") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
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 b9e38a1..74dc47b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3638,6 +3638,7 @@
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
@@ -3654,6 +3655,7 @@
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type time (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
@@ -3824,6 +3826,7 @@
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor/uuid">
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index bd3b216..4080173 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1122,10 +1122,16 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "polygon", 1);
     public static final FunctionIdentifier TIME_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 1);
+    public static final FunctionIdentifier TIME_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 2);
     public static final FunctionIdentifier DATE_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 1);
+    public static final FunctionIdentifier DATE_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 2);
     public static final FunctionIdentifier DATETIME_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 1);
+    public static final FunctionIdentifier DATETIME_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 2);
     public static final FunctionIdentifier DURATION_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration", 1);
     public static final FunctionIdentifier UUID_CONSTRUCTOR =
@@ -1661,7 +1667,9 @@
         addFunction(RANDOM_WITH_SEED, NumericUnaryTypeComputer.INSTANCE_DOUBLE, false);
 
         addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(DATE_CONSTRUCTOR_WITH_FORMAT, ADateTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(DATETIME_CONSTRUCTOR_WITH_FORMAT, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DOUBLE_CONSTRUCTOR, ADoubleTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DURATION_CONSTRUCTOR, ADurationTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(YEAR_MONTH_DURATION_CONSTRUCTOR, AYearMonthDurationTypeComputer.INSTANCE_NULLABLE, true);
@@ -2222,6 +2230,7 @@
 
         addFunction(TID, AInt64TypeComputer.INSTANCE, true);
         addFunction(TIME_CONSTRUCTOR, ATimeTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(TIME_CONSTRUCTOR_WITH_FORMAT, ATimeTypeComputer.INSTANCE_NULLABLE, true);
         addPrivateFunction(TYPE_OF, AnyTypeComputer.INSTANCE, true);
         addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(WORD_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 9dcc0a4..12ab1a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -19,27 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ADateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -58,77 +46,16 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableDate aDate = new AMutableDate(0);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ADate> dateSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case DATE:
-                                result.set(inputArg);
-                                break;
-                            case DATETIME:
-                                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                aDate.setValue(GregorianCalendarSystem.getInstance().getChrononInDays(chronon));
-                                resultStorage.reset();
-                                dateSerde.serialize(aDate, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseDate(utf8Ptr, aDate)) {
-                                    resultStorage.reset();
-                                    dateSerde.serialize(aDate, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
+                return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADateConstructorDescriptor.this.getIdentifier();
                     }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ADATE;
-                    }
                 };
             }
         };
     }
 
-    private static boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
-        int stringLength = textPtr.getUTF8Length();
-        if (stringLength < 8) {
-            return false;
-        }
-        try {
-            long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
-                    stringLength);
-            int chrononInDays = GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
-            result.setValue(chrononInDays);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.DATE_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..9d51f12
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@MissingNullInOutFunction
+public class ADateConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADateConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.DATE_ONLY, false)) {
+                                result.setValue((int) (aInt64.getLongValue() / GregorianCalendarSystem.CHRONON_OF_DAY));
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ADateConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DATE_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index aa1068b..323ffcf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -19,28 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ADateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -59,52 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ADateTime> datetimeSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case DATETIME:
-                                result.set(inputArg);
-                                break;
-                            case DATE:
-                                int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
-                                resultStorage.reset();
-                                datetimeSerde.serialize(aDateTime, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseDateTime(utf8Ptr, aDateTime)) {
-                                    resultStorage.reset();
-                                    datetimeSerde.serialize(aDateTime, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ADATETIME;
-                    }
-
+                return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADateTimeConstructorDescriptor.this.getIdentifier();
@@ -114,35 +56,6 @@
         };
     }
 
-    private static boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
-        int stringLength = textPtr.getUTF8Length();
-        // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
-        if (stringLength < 14) {
-            return false;
-        }
-        byte[] bytes = textPtr.getByteArray();
-        int charStartOffset = textPtr.getCharStartOffset();
-        // +1 if it is negative (-)
-        short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
-        timeOffset += 8;
-        if (bytes[charStartOffset + timeOffset] != 'T') {
-            timeOffset += 2;
-            if (bytes[charStartOffset + timeOffset] != 'T') {
-                return false;
-            }
-        }
-        try {
-            long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
-            int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
-                    stringLength - timeOffset - 1);
-            chronon += timeInMs;
-            result.setValue(chronon);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.DATETIME_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..715a7df
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ADateTimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADateTimeConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.DATETIME, false)) {
+                                result.setValue(aInt64.getLongValue());
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ADateTimeConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DATETIME_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index 714abad..b621441 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -19,27 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
-import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ATimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -58,53 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableTime aTime = new AMutableTime(0);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ATime> timeSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case TIME:
-                                result.set(inputArg);
-                                break;
-                            case DATETIME:
-                                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                int chrononTime = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
-                                aTime.setValue(chrononTime);
-                                resultStorage.reset();
-                                timeSerde.serialize(aTime, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseTime(utf8Ptr, aTime)) {
-                                    resultStorage.reset();
-                                    timeSerde.serialize(aTime, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ATIME;
-                    }
-
+                return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ATimeConstructorDescriptor.this.getIdentifier();
@@ -114,25 +56,6 @@
         };
     }
 
-    private static boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
-        int stringLength = textPtr.getUTF8Length();
-        // the string to be parsed should be at least 6 characters: hhmmss
-        if (stringLength < 6) {
-            return false;
-        }
-        try {
-            int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
-                    stringLength);
-            if (chronon < 0) {
-                chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-            result.setValue(chronon);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.TIME_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..e2df1be
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ATimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ATimeConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.TIME_ONLY, false)) {
+                                result.setValue((int) aInt64.getLongValue());
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ATimeConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.TIME_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
index 6287fe6..e7c4cd5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
@@ -61,13 +61,21 @@
 
     @Override
     public final void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-        inputEval.evaluate(tuple, inputArg);
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
+        evaluateInput(tuple);
+        if (checkAndSetMissingOrNull(result)) {
             return;
         }
         evaluateImpl(result);
     }
 
+    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+        inputEval.evaluate(tuple, inputArg);
+    }
+
+    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+        return PointableHelper.checkAndSetMissingOrNull(result, inputArg);
+    }
+
     protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
new file mode 100644
index 0000000..2ef54b1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableDate aDate = new AMutableDate(0);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADate> dateSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractDateConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case DATE:
+                result.set(inputArg);
+                break;
+            case DATETIME:
+                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                aDate.setValue(GregorianCalendarSystem.getInstance().getChrononInDays(chronon));
+                resultStorage.reset();
+                dateSerde.serialize(aDate, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseDate(utf8Ptr, aDate)) {
+                    resultStorage.reset();
+                    dateSerde.serialize(aDate, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+        int stringLength = textPtr.getUTF8Length();
+        if (stringLength < 8) {
+            return false;
+        }
+        try {
+            long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+                    stringLength);
+            int chrononInDays = GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
+            result.setValue(chrononInDays);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ADATE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
new file mode 100644
index 0000000..6765d90
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADateTime> datetimeSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractDateTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case DATETIME:
+                result.set(inputArg);
+                break;
+            case DATE:
+                int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
+                resultStorage.reset();
+                datetimeSerde.serialize(aDateTime, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseDateTime(utf8Ptr, aDateTime)) {
+                    resultStorage.reset();
+                    datetimeSerde.serialize(aDateTime, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+        int stringLength = textPtr.getUTF8Length();
+        // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
+        if (stringLength < 14) {
+            return false;
+        }
+        byte[] bytes = textPtr.getByteArray();
+        int charStartOffset = textPtr.getCharStartOffset();
+        // +1 if it is negative (-)
+        short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
+        timeOffset += 8;
+        if (bytes[charStartOffset + timeOffset] != 'T') {
+            timeOffset += 2;
+            if (bytes[charStartOffset + timeOffset] != 'T') {
+                return false;
+            }
+        }
+        try {
+            long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
+            int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
+                    stringLength - timeOffset - 1);
+            chronon += timeInMs;
+            result.setValue(chronon);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ADATETIME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
new file mode 100644
index 0000000..44a11a5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableTime aTime = new AMutableTime(0);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ATime> timeSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case TIME:
+                result.set(inputArg);
+                break;
+            case DATETIME:
+                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                int chrononTime = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
+                aTime.setValue(chrononTime);
+                resultStorage.reset();
+                timeSerde.serialize(aTime, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseTime(utf8Ptr, aTime)) {
+                    resultStorage.reset();
+                    timeSerde.serialize(aTime, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+        int stringLength = textPtr.getUTF8Length();
+        // the string to be parsed should be at least 6 characters: hhmmss
+        if (stringLength < 6) {
+            return false;
+        }
+        try {
+            int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+                    stringLength);
+            if (chronon < 0) {
+                chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+            result.setValue(chronon);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ATIME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 18d55aa..ca42280 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -265,7 +265,9 @@
 import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
@@ -285,6 +287,7 @@
 import org.apache.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ClosedRecordConstructorDescriptor;
@@ -1052,8 +1055,11 @@
         fc.add(ACircleConstructorDescriptor.FACTORY);
         fc.add(ARectangleConstructorDescriptor.FACTORY);
         fc.add(ATimeConstructorDescriptor.FACTORY);
+        fc.add(ATimeConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADateConstructorDescriptor.FACTORY);
+        fc.add(ADateConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADateTimeConstructorDescriptor.FACTORY);
+        fc.add(ADateTimeConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADurationConstructorDescriptor.FACTORY);
         fc.add(AYearMonthDurationConstructorDescriptor.FACTORY);
         fc.add(ADayTimeDurationConstructorDescriptor.FACTORY);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Hello Anon. E. Moose #1000171, Jenkins, 

I'd like you to reexamine a change. Please visit

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366

to look at the new patch set (#3).

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................

[NO ISSUE][FUN] Support format in date/time constructors

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

Details:
- Add date/time constructor functions that
  accept format as a second argument
- Add testcases

Change-Id: I746cc04d25bda808094fd25168392772a0fba728
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast
A asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast
A asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
28 files changed, 952 insertions(+), 248 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/66/12366/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 3
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: newpatchset

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )


Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................

[NO ISSUE][FUN] Support format in date/time constructors

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

Details:
- Add date/time constructor functions that
  accept format as a second argument
- Add testcases

Change-Id: I746cc04d25bda808094fd25168392772a0fba728
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
25 files changed, 820 insertions(+), 248 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/66/12366/1

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
new file mode 100644
index 0000000..be6b9c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '10/30/2010',
+  '31-12-2020'
+],
+fmt = [
+  'MM/DD/YYYY',
+  'DD-MM-YYYY'
+]
+
+select i, date(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
index 443aa80..6a157f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": date(0),
   "null_3": date(time("01:02:03")),
   "null_4": date([]),
-  "null_5": date({})
+  "null_5": date({}),
+  "null_6": date("12/31/2020", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
new file mode 100644
index 0000000..98b3ee6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '10/30/2010 58-57-10',
+  '31-12-2020 11/58/59'
+],
+fmt = [
+  'MM/DD/YYYY ss-mm-hh',
+  'DD-MM-YYYY hh/mm/ss'
+]
+
+select i, datetime(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
index 1a9f391..082d840 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": datetime(0),
   "null_3": datetime(time("01:02:03")),
   "null_4": datetime([]),
-  "null_5": datetime({})
+  "null_5": datetime({}),
+  "null_6": datetime("1951-12-27T12:20:15Z", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
new file mode 100644
index 0000000..37eba54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '11/58/59',
+  '58-57-10'
+],
+fmt = [
+  'hh/mm/ss',
+  'ss-mm-hh'
+]
+
+select i, time(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
index dbdcbc1..b97ec3e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": time(0),
   "null_3": time(date("2020-01-02")),
   "null_4": time([]),
-  "null_5": time({})
+  "null_5": time({}),
+  "null_6": time("11:58:59", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
new file mode 100644
index 0000000..d1d2f7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": date("2010-10-30") }
+{ "i": 1, "actual": date("2020-12-31") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
new file mode 100644
index 0000000..702522c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": datetime("2010-10-30T10:57:58.000Z") }
+{ "i": 1, "actual": datetime("2020-12-31T11:58:59.000Z") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
new file mode 100644
index 0000000..d854daa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": time("11:58:59.000Z") }
+{ "i": 1, "actual": time("10:57:58.000Z") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
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 b9e38a1..74dc47b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3638,6 +3638,7 @@
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
@@ -3654,6 +3655,7 @@
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type time (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
@@ -3824,6 +3826,7 @@
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor/uuid">
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index bd3b216..4080173 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1122,10 +1122,16 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "polygon", 1);
     public static final FunctionIdentifier TIME_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 1);
+    public static final FunctionIdentifier TIME_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 2);
     public static final FunctionIdentifier DATE_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 1);
+    public static final FunctionIdentifier DATE_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 2);
     public static final FunctionIdentifier DATETIME_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 1);
+    public static final FunctionIdentifier DATETIME_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 2);
     public static final FunctionIdentifier DURATION_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration", 1);
     public static final FunctionIdentifier UUID_CONSTRUCTOR =
@@ -1661,7 +1667,9 @@
         addFunction(RANDOM_WITH_SEED, NumericUnaryTypeComputer.INSTANCE_DOUBLE, false);
 
         addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(DATE_CONSTRUCTOR_WITH_FORMAT, ADateTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(DATETIME_CONSTRUCTOR_WITH_FORMAT, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DOUBLE_CONSTRUCTOR, ADoubleTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DURATION_CONSTRUCTOR, ADurationTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(YEAR_MONTH_DURATION_CONSTRUCTOR, AYearMonthDurationTypeComputer.INSTANCE_NULLABLE, true);
@@ -2222,6 +2230,7 @@
 
         addFunction(TID, AInt64TypeComputer.INSTANCE, true);
         addFunction(TIME_CONSTRUCTOR, ATimeTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(TIME_CONSTRUCTOR_WITH_FORMAT, ATimeTypeComputer.INSTANCE_NULLABLE, true);
         addPrivateFunction(TYPE_OF, AnyTypeComputer.INSTANCE, true);
         addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(WORD_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 9dcc0a4..12ab1a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -19,27 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ADateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -58,77 +46,16 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableDate aDate = new AMutableDate(0);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ADate> dateSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case DATE:
-                                result.set(inputArg);
-                                break;
-                            case DATETIME:
-                                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                aDate.setValue(GregorianCalendarSystem.getInstance().getChrononInDays(chronon));
-                                resultStorage.reset();
-                                dateSerde.serialize(aDate, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseDate(utf8Ptr, aDate)) {
-                                    resultStorage.reset();
-                                    dateSerde.serialize(aDate, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
+                return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADateConstructorDescriptor.this.getIdentifier();
                     }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ADATE;
-                    }
                 };
             }
         };
     }
 
-    private static boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
-        int stringLength = textPtr.getUTF8Length();
-        if (stringLength < 8) {
-            return false;
-        }
-        try {
-            long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
-                    stringLength);
-            int chrononInDays = GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
-            result.setValue(chrononInDays);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.DATE_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..9d51f12
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@MissingNullInOutFunction
+public class ADateConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADateConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.DATE_ONLY, false)) {
+                                result.setValue((int) (aInt64.getLongValue() / GregorianCalendarSystem.CHRONON_OF_DAY));
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ADateConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DATE_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index aa1068b..323ffcf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -19,28 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ADateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -59,52 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ADateTime> datetimeSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case DATETIME:
-                                result.set(inputArg);
-                                break;
-                            case DATE:
-                                int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
-                                resultStorage.reset();
-                                datetimeSerde.serialize(aDateTime, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseDateTime(utf8Ptr, aDateTime)) {
-                                    resultStorage.reset();
-                                    datetimeSerde.serialize(aDateTime, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ADATETIME;
-                    }
-
+                return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADateTimeConstructorDescriptor.this.getIdentifier();
@@ -114,35 +56,6 @@
         };
     }
 
-    private static boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
-        int stringLength = textPtr.getUTF8Length();
-        // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
-        if (stringLength < 14) {
-            return false;
-        }
-        byte[] bytes = textPtr.getByteArray();
-        int charStartOffset = textPtr.getCharStartOffset();
-        // +1 if it is negative (-)
-        short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
-        timeOffset += 8;
-        if (bytes[charStartOffset + timeOffset] != 'T') {
-            timeOffset += 2;
-            if (bytes[charStartOffset + timeOffset] != 'T') {
-                return false;
-            }
-        }
-        try {
-            long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
-            int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
-                    stringLength - timeOffset - 1);
-            chronon += timeInMs;
-            result.setValue(chronon);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.DATETIME_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..715a7df
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ADateTimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADateTimeConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.DATETIME, false)) {
+                                result.setValue(aInt64.getLongValue());
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ADateTimeConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DATETIME_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index 714abad..b621441 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -19,27 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
-import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ATimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -58,53 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableTime aTime = new AMutableTime(0);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ATime> timeSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case TIME:
-                                result.set(inputArg);
-                                break;
-                            case DATETIME:
-                                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                int chrononTime = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
-                                aTime.setValue(chrononTime);
-                                resultStorage.reset();
-                                timeSerde.serialize(aTime, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseTime(utf8Ptr, aTime)) {
-                                    resultStorage.reset();
-                                    timeSerde.serialize(aTime, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ATIME;
-                    }
-
+                return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ATimeConstructorDescriptor.this.getIdentifier();
@@ -114,25 +56,6 @@
         };
     }
 
-    private static boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
-        int stringLength = textPtr.getUTF8Length();
-        // the string to be parsed should be at least 6 characters: hhmmss
-        if (stringLength < 6) {
-            return false;
-        }
-        try {
-            int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
-                    stringLength);
-            if (chronon < 0) {
-                chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-            result.setValue(chronon);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.TIME_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..e2df1be
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ATimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ATimeConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.TIME_ONLY, false)) {
+                                result.setValue((int) aInt64.getLongValue());
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ATimeConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.TIME_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
index 6287fe6..e7c4cd5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
@@ -61,13 +61,21 @@
 
     @Override
     public final void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-        inputEval.evaluate(tuple, inputArg);
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
+        evaluateInput(tuple);
+        if (checkAndSetMissingOrNull(result)) {
             return;
         }
         evaluateImpl(result);
     }
 
+    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+        inputEval.evaluate(tuple, inputArg);
+    }
+
+    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+        return PointableHelper.checkAndSetMissingOrNull(result, inputArg);
+    }
+
     protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
new file mode 100644
index 0000000..2ef54b1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableDate aDate = new AMutableDate(0);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADate> dateSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractDateConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case DATE:
+                result.set(inputArg);
+                break;
+            case DATETIME:
+                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                aDate.setValue(GregorianCalendarSystem.getInstance().getChrononInDays(chronon));
+                resultStorage.reset();
+                dateSerde.serialize(aDate, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseDate(utf8Ptr, aDate)) {
+                    resultStorage.reset();
+                    dateSerde.serialize(aDate, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+        int stringLength = textPtr.getUTF8Length();
+        if (stringLength < 8) {
+            return false;
+        }
+        try {
+            long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+                    stringLength);
+            int chrononInDays = GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
+            result.setValue(chrononInDays);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ADATE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
new file mode 100644
index 0000000..6765d90
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADateTime> datetimeSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractDateTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case DATETIME:
+                result.set(inputArg);
+                break;
+            case DATE:
+                int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
+                resultStorage.reset();
+                datetimeSerde.serialize(aDateTime, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseDateTime(utf8Ptr, aDateTime)) {
+                    resultStorage.reset();
+                    datetimeSerde.serialize(aDateTime, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+        int stringLength = textPtr.getUTF8Length();
+        // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
+        if (stringLength < 14) {
+            return false;
+        }
+        byte[] bytes = textPtr.getByteArray();
+        int charStartOffset = textPtr.getCharStartOffset();
+        // +1 if it is negative (-)
+        short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
+        timeOffset += 8;
+        if (bytes[charStartOffset + timeOffset] != 'T') {
+            timeOffset += 2;
+            if (bytes[charStartOffset + timeOffset] != 'T') {
+                return false;
+            }
+        }
+        try {
+            long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
+            int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
+                    stringLength - timeOffset - 1);
+            chronon += timeInMs;
+            result.setValue(chronon);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ADATETIME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
new file mode 100644
index 0000000..44a11a5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableTime aTime = new AMutableTime(0);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ATime> timeSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case TIME:
+                result.set(inputArg);
+                break;
+            case DATETIME:
+                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                int chrononTime = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
+                aTime.setValue(chrononTime);
+                resultStorage.reset();
+                timeSerde.serialize(aTime, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseTime(utf8Ptr, aTime)) {
+                    resultStorage.reset();
+                    timeSerde.serialize(aTime, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+        int stringLength = textPtr.getUTF8Length();
+        // the string to be parsed should be at least 6 characters: hhmmss
+        if (stringLength < 6) {
+            return false;
+        }
+        try {
+            int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+                    stringLength);
+            if (chronon < 0) {
+                chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+            result.setValue(chronon);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ATIME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 18d55aa..ca42280 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -265,7 +265,9 @@
 import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
@@ -285,6 +287,7 @@
 import org.apache.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ClosedRecordConstructorDescriptor;
@@ -1052,8 +1055,11 @@
         fc.add(ACircleConstructorDescriptor.FACTORY);
         fc.add(ARectangleConstructorDescriptor.FACTORY);
         fc.add(ATimeConstructorDescriptor.FACTORY);
+        fc.add(ATimeConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADateConstructorDescriptor.FACTORY);
+        fc.add(ADateConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADateTimeConstructorDescriptor.FACTORY);
+        fc.add(ADateTimeConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADurationConstructorDescriptor.FACTORY);
         fc.add(AYearMonthDurationConstructorDescriptor.FACTORY);
         fc.add(ADayTimeDurationConstructorDescriptor.FACTORY);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/bKUXmRmKb6fEMAKP8 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 3
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 19 Jul 2021 22:18:32 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/gsFDMionC9RccUPRA : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sat, 17 Jul 2021 04:41:29 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/Mz1wTfqmERfgruhAA : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 3
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 20 Jul 2021 02:09:03 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 3: Code-Review+1


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 3
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 20 Jul 2021 01:04:00 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 1: Integration-Tests-1

Integration Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/12213/ : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Sat, 17 Jul 2021 02:09:27 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Glenn Galvizo <gg...@uci.edu>:

Glenn Galvizo has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 3: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 3
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 20 Jul 2021 15:39:41 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/93efJRbajcWYGGAt5 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 1
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-CC: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sat, 17 Jul 2021 01:09:08 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO ISSUE][FUN] Support format in date/time constructors

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Dmitry Lychagin <dm...@couchbase.com>:

Dmitry Lychagin has submitted this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366 )

Change subject: [NO ISSUE][FUN] Support format in date/time constructors
......................................................................

[NO ISSUE][FUN] Support format in date/time constructors

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

Details:
- Add date/time constructor functions that
  accept format as a second argument
- Add testcases

Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
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: Glenn Galvizo <gg...@uci.edu>
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast
A asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast
A asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
28 files changed, 952 insertions(+), 248 deletions(-)

Approvals:
  Dmitry Lychagin: Looks good to me, but someone else must approve
  Glenn Galvizo: Looks good to me, approved
  Jenkins: Verified; Verified

Objections:
  Anon. E. Moose #1000171: Violations found



diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
new file mode 100644
index 0000000..be6b9c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '10/30/2010',
+  '31-12-2020'
+],
+fmt = [
+  'MM/DD/YYYY',
+  'DD-MM-YYYY'
+]
+
+select i, date(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
index 443aa80..6a157f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": date(0),
   "null_3": date(time("01:02:03")),
   "null_4": date([]),
-  "null_5": date({})
+  "null_5": date({}),
+  "null_6": date("12/31/2020", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
new file mode 100644
index 0000000..98b3ee6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '10/30/2010 58-57-10',
+  '31-12-2020 11/58/59'
+],
+fmt = [
+  'MM/DD/YYYY ss-mm-hh',
+  'DD-MM-YYYY hh/mm/ss'
+]
+
+select i, datetime(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
index 1a9f391..082d840 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": datetime(0),
   "null_3": datetime(time("01:02:03")),
   "null_4": datetime([]),
-  "null_5": datetime({})
+  "null_5": datetime({}),
+  "null_6": datetime("1951-12-27T12:20:15Z", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
new file mode 100644
index 0000000..37eba54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+  '11/58/59',
+  '58-57-10'
+],
+fmt = [
+  'hh/mm/ss',
+  'ss-mm-hh'
+]
+
+select i, time(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
index dbdcbc1..b97ec3e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
@@ -26,5 +26,6 @@
   "null_2": time(0),
   "null_3": time(date("2020-01-02")),
   "null_4": time([]),
-  "null_5": time({})
+  "null_5": time({}),
+  "null_6": time("11:58:59", "INVALID_FORMAT")
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
new file mode 100644
index 0000000..d1d2f7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": date("2010-10-30") }
+{ "i": 1, "actual": date("2020-12-31") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
new file mode 100644
index 0000000..702522c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": datetime("2010-10-30T10:57:58.000Z") }
+{ "i": 1, "actual": datetime("2020-12-31T11:58:59.000Z") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
new file mode 100644
index 0000000..d854daa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": time("11:58:59.000Z") }
+{ "i": 1, "actual": time("10:57:58.000Z") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
index 4d80977..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
@@ -1 +1 @@
-{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast
new file mode 100644
index 0000000..9aa4f45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast
@@ -0,0 +1,44 @@
+Query:
+Let Variable [ Name=$test ]
+  :=
+  OrderedListConstructor [
+    LiteralExpr [STRING] [10/30/2010]
+    LiteralExpr [STRING] [31-12-2020]
+  ]
+Let Variable [ Name=$fmt ]
+  :=
+  OrderedListConstructor [
+    LiteralExpr [STRING] [MM/DD/YYYY]
+    LiteralExpr [STRING] [DD-MM-YYYY]
+  ]
+SELECT [
+Variable [ Name=$i ]
+i
+FunctionCall asterix.date@2[
+  IndexAccessor [
+    Variable [ Name=$test ]
+    Index:     Variable [ Name=$i ]
+  ]
+  IndexAccessor [
+    Variable [ Name=$fmt ]
+    Index:     Variable [ Name=$i ]
+  ]
+]
+actual
+]
+FROM [  FunctionCall asterix.range@2[
+    LiteralExpr [LONG] [0]
+    OperatorExpr [
+      FunctionCall asterix.len@1[
+        Variable [ Name=$test ]
+      ]
+      -
+      LiteralExpr [LONG] [1]
+    ]
+  ]
+  AS Variable [ Name=$i ]
+]
+Orderby
+  Variable [ Name=$i ]
+  ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast
new file mode 100644
index 0000000..8e644d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast
@@ -0,0 +1,44 @@
+Query:
+Let Variable [ Name=$test ]
+  :=
+  OrderedListConstructor [
+    LiteralExpr [STRING] [10/30/2010 58-57-10]
+    LiteralExpr [STRING] [31-12-2020 11/58/59]
+  ]
+Let Variable [ Name=$fmt ]
+  :=
+  OrderedListConstructor [
+    LiteralExpr [STRING] [MM/DD/YYYY ss-mm-hh]
+    LiteralExpr [STRING] [DD-MM-YYYY hh/mm/ss]
+  ]
+SELECT [
+Variable [ Name=$i ]
+i
+FunctionCall asterix.datetime@2[
+  IndexAccessor [
+    Variable [ Name=$test ]
+    Index:     Variable [ Name=$i ]
+  ]
+  IndexAccessor [
+    Variable [ Name=$fmt ]
+    Index:     Variable [ Name=$i ]
+  ]
+]
+actual
+]
+FROM [  FunctionCall asterix.range@2[
+    LiteralExpr [LONG] [0]
+    OperatorExpr [
+      FunctionCall asterix.len@1[
+        Variable [ Name=$test ]
+      ]
+      -
+      LiteralExpr [LONG] [1]
+    ]
+  ]
+  AS Variable [ Name=$i ]
+]
+Orderby
+  Variable [ Name=$i ]
+  ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast
new file mode 100644
index 0000000..39c441c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast
@@ -0,0 +1,44 @@
+Query:
+Let Variable [ Name=$test ]
+  :=
+  OrderedListConstructor [
+    LiteralExpr [STRING] [11/58/59]
+    LiteralExpr [STRING] [58-57-10]
+  ]
+Let Variable [ Name=$fmt ]
+  :=
+  OrderedListConstructor [
+    LiteralExpr [STRING] [hh/mm/ss]
+    LiteralExpr [STRING] [ss-mm-hh]
+  ]
+SELECT [
+Variable [ Name=$i ]
+i
+FunctionCall asterix.time@2[
+  IndexAccessor [
+    Variable [ Name=$test ]
+    Index:     Variable [ Name=$i ]
+  ]
+  IndexAccessor [
+    Variable [ Name=$fmt ]
+    Index:     Variable [ Name=$i ]
+  ]
+]
+actual
+]
+FROM [  FunctionCall asterix.range@2[
+    LiteralExpr [LONG] [0]
+    OperatorExpr [
+      FunctionCall asterix.len@1[
+        Variable [ Name=$test ]
+      ]
+      -
+      LiteralExpr [LONG] [1]
+    ]
+  ]
+  AS Variable [ Name=$i ]
+]
+Orderby
+  Variable [ Name=$i ]
+  ASC
+
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 b9e38a1..74dc47b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3638,6 +3638,7 @@
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
@@ -3654,6 +3655,7 @@
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type time (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
@@ -3824,6 +3826,7 @@
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
         <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor/uuid">
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index bd3b216..4080173 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1122,10 +1122,16 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "polygon", 1);
     public static final FunctionIdentifier TIME_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 1);
+    public static final FunctionIdentifier TIME_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 2);
     public static final FunctionIdentifier DATE_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 1);
+    public static final FunctionIdentifier DATE_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 2);
     public static final FunctionIdentifier DATETIME_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 1);
+    public static final FunctionIdentifier DATETIME_CONSTRUCTOR_WITH_FORMAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 2);
     public static final FunctionIdentifier DURATION_CONSTRUCTOR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration", 1);
     public static final FunctionIdentifier UUID_CONSTRUCTOR =
@@ -1661,7 +1667,9 @@
         addFunction(RANDOM_WITH_SEED, NumericUnaryTypeComputer.INSTANCE_DOUBLE, false);
 
         addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(DATE_CONSTRUCTOR_WITH_FORMAT, ADateTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(DATETIME_CONSTRUCTOR_WITH_FORMAT, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DOUBLE_CONSTRUCTOR, ADoubleTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(DURATION_CONSTRUCTOR, ADurationTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(YEAR_MONTH_DURATION_CONSTRUCTOR, AYearMonthDurationTypeComputer.INSTANCE_NULLABLE, true);
@@ -2222,6 +2230,7 @@
 
         addFunction(TID, AInt64TypeComputer.INSTANCE, true);
         addFunction(TIME_CONSTRUCTOR, ATimeTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(TIME_CONSTRUCTOR_WITH_FORMAT, ATimeTypeComputer.INSTANCE_NULLABLE, true);
         addPrivateFunction(TYPE_OF, AnyTypeComputer.INSTANCE, true);
         addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(WORD_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 9dcc0a4..12ab1a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -19,27 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ADateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -58,77 +46,16 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableDate aDate = new AMutableDate(0);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ADate> dateSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case DATE:
-                                result.set(inputArg);
-                                break;
-                            case DATETIME:
-                                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                aDate.setValue(GregorianCalendarSystem.getInstance().getChrononInDays(chronon));
-                                resultStorage.reset();
-                                dateSerde.serialize(aDate, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseDate(utf8Ptr, aDate)) {
-                                    resultStorage.reset();
-                                    dateSerde.serialize(aDate, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
+                return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADateConstructorDescriptor.this.getIdentifier();
                     }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ADATE;
-                    }
                 };
             }
         };
     }
 
-    private static boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
-        int stringLength = textPtr.getUTF8Length();
-        if (stringLength < 8) {
-            return false;
-        }
-        try {
-            long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
-                    stringLength);
-            int chrononInDays = GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
-            result.setValue(chrononInDays);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.DATE_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..9d51f12
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@MissingNullInOutFunction
+public class ADateConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADateConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.DATE_ONLY, false)) {
+                                result.setValue((int) (aInt64.getLongValue() / GregorianCalendarSystem.CHRONON_OF_DAY));
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ADateConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DATE_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index aa1068b..323ffcf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -19,28 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ADateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -59,52 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ADateTime> datetimeSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case DATETIME:
-                                result.set(inputArg);
-                                break;
-                            case DATE:
-                                int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
-                                resultStorage.reset();
-                                datetimeSerde.serialize(aDateTime, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseDateTime(utf8Ptr, aDateTime)) {
-                                    resultStorage.reset();
-                                    datetimeSerde.serialize(aDateTime, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ADATETIME;
-                    }
-
+                return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADateTimeConstructorDescriptor.this.getIdentifier();
@@ -114,35 +56,6 @@
         };
     }
 
-    private static boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
-        int stringLength = textPtr.getUTF8Length();
-        // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
-        if (stringLength < 14) {
-            return false;
-        }
-        byte[] bytes = textPtr.getByteArray();
-        int charStartOffset = textPtr.getCharStartOffset();
-        // +1 if it is negative (-)
-        short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
-        timeOffset += 8;
-        if (bytes[charStartOffset + timeOffset] != 'T') {
-            timeOffset += 2;
-            if (bytes[charStartOffset + timeOffset] != 'T') {
-                return false;
-            }
-        }
-        try {
-            long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
-            int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
-                    stringLength - timeOffset - 1);
-            chronon += timeInMs;
-            result.setValue(chronon);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.DATETIME_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..715a7df
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ADateTimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADateTimeConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.DATETIME, false)) {
+                                result.setValue(aInt64.getLongValue());
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ADateTimeConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DATETIME_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index 714abad..b621441 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -19,27 +19,15 @@
 package org.apache.asterix.runtime.evaluators.constructors;
 
 import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
-import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
 @MissingNullInOutFunction
 public class ATimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -58,53 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new AbstractConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
-
-                    private final AMutableTime aTime = new AMutableTime(0);
-                    @SuppressWarnings("unchecked")
-                    private final ISerializerDeserializer<ATime> timeSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
-                    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
-                    @Override
-                    protected void evaluateImpl(IPointable result) throws HyracksDataException {
-                        byte[] bytes = inputArg.getByteArray();
-                        int startOffset = inputArg.getStartOffset();
-                        int len = inputArg.getLength();
-                        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
-                        switch (inputType) {
-                            case TIME:
-                                result.set(inputArg);
-                                break;
-                            case DATETIME:
-                                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
-                                int chrononTime = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
-                                aTime.setValue(chrononTime);
-                                resultStorage.reset();
-                                timeSerde.serialize(aTime, out);
-                                result.set(resultStorage);
-                                break;
-                            case STRING:
-                                utf8Ptr.set(bytes, startOffset + 1, len - 1);
-                                if (parseTime(utf8Ptr, aTime)) {
-                                    resultStorage.reset();
-                                    timeSerde.serialize(aTime, out);
-                                    result.set(resultStorage);
-                                } else {
-                                    handleParseError(utf8Ptr, result);
-                                }
-                                break;
-                            default:
-                                handleUnsupportedType(inputType, result);
-                                break;
-                        }
-                    }
-
-                    @Override
-                    protected BuiltinType getTargetType() {
-                        return BuiltinType.ATIME;
-                    }
-
+                return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ATimeConstructorDescriptor.this.getIdentifier();
@@ -114,25 +56,6 @@
         };
     }
 
-    private static boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
-        int stringLength = textPtr.getUTF8Length();
-        // the string to be parsed should be at least 6 characters: hhmmss
-        if (stringLength < 6) {
-            return false;
-        }
-        try {
-            int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
-                    stringLength);
-            if (chronon < 0) {
-                chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-            result.setValue(chronon);
-            return true;
-        } catch (HyracksDataException e) {
-            return false;
-        }
-    }
-
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.TIME_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..e2df1be
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ATimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ATimeConstructorWithFormatDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+                    private final IScalarEvaluator formatEval = args[1].createScalarEvaluator(ctx);
+                    private final IPointable formatArg = new VoidPointable();
+                    private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+                    @Override
+                    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+                        super.evaluateInput(tuple);
+                        formatEval.evaluate(tuple, formatArg);
+                    }
+
+                    @Override
+                    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+                        return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+                    }
+
+                    @Override
+                    protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+                        byte[] formatBytes = formatArg.getByteArray();
+                        int formatStartOffset = formatArg.getStartOffset();
+                        int formatLength = formatArg.getLength();
+                        if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                            return false;
+                        }
+                        formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+                        try {
+                            if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+                                    textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+                                    formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+                                    DateTimeFormatUtils.DateTimeParseMode.TIME_ONLY, false)) {
+                                result.setValue((int) aInt64.getLongValue());
+                                return true;
+                            } else {
+                                return false;
+                            }
+                        } catch (AsterixTemporalTypeParseException e) {
+                            // shouldn't happen
+                            return false;
+                        }
+                    }
+
+                    @Override
+                    protected FunctionIdentifier getIdentifier() {
+                        return ATimeConstructorWithFormatDescriptor.this.getIdentifier();
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.TIME_CONSTRUCTOR_WITH_FORMAT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
index 6287fe6..e7c4cd5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
@@ -61,13 +61,21 @@
 
     @Override
     public final void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-        inputEval.evaluate(tuple, inputArg);
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
+        evaluateInput(tuple);
+        if (checkAndSetMissingOrNull(result)) {
             return;
         }
         evaluateImpl(result);
     }
 
+    protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+        inputEval.evaluate(tuple, inputArg);
+    }
+
+    protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+        return PointableHelper.checkAndSetMissingOrNull(result, inputArg);
+    }
+
     protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
new file mode 100644
index 0000000..2ef54b1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableDate aDate = new AMutableDate(0);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADate> dateSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractDateConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case DATE:
+                result.set(inputArg);
+                break;
+            case DATETIME:
+                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                aDate.setValue(GregorianCalendarSystem.getInstance().getChrononInDays(chronon));
+                resultStorage.reset();
+                dateSerde.serialize(aDate, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseDate(utf8Ptr, aDate)) {
+                    resultStorage.reset();
+                    dateSerde.serialize(aDate, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+        int stringLength = textPtr.getUTF8Length();
+        if (stringLength < 8) {
+            return false;
+        }
+        try {
+            long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+                    stringLength);
+            int chrononInDays = GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
+            result.setValue(chrononInDays);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ADATE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
new file mode 100644
index 0000000..6765d90
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADateTime> datetimeSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractDateTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case DATETIME:
+                result.set(inputArg);
+                break;
+            case DATE:
+                int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
+                resultStorage.reset();
+                datetimeSerde.serialize(aDateTime, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseDateTime(utf8Ptr, aDateTime)) {
+                    resultStorage.reset();
+                    datetimeSerde.serialize(aDateTime, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+        int stringLength = textPtr.getUTF8Length();
+        // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
+        if (stringLength < 14) {
+            return false;
+        }
+        byte[] bytes = textPtr.getByteArray();
+        int charStartOffset = textPtr.getCharStartOffset();
+        // +1 if it is negative (-)
+        short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
+        timeOffset += 8;
+        if (bytes[charStartOffset + timeOffset] != 'T') {
+            timeOffset += 2;
+            if (bytes[charStartOffset + timeOffset] != 'T') {
+                return false;
+            }
+        }
+        try {
+            long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
+            int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
+                    stringLength - timeOffset - 1);
+            chronon += timeInMs;
+            result.setValue(chronon);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ADATETIME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
new file mode 100644
index 0000000..44a11a5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+    private final AMutableTime aTime = new AMutableTime(0);
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ATime> timeSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+    private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+    protected AbstractTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+            SourceLocation sourceLoc) {
+        super(ctx, inputEval, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        byte[] bytes = inputArg.getByteArray();
+        int startOffset = inputArg.getStartOffset();
+        int len = inputArg.getLength();
+        ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+        switch (inputType) {
+            case TIME:
+                result.set(inputArg);
+                break;
+            case DATETIME:
+                long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+                int chrononTime = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
+                aTime.setValue(chrononTime);
+                resultStorage.reset();
+                timeSerde.serialize(aTime, out);
+                result.set(resultStorage);
+                break;
+            case STRING:
+                utf8Ptr.set(bytes, startOffset + 1, len - 1);
+                if (parseTime(utf8Ptr, aTime)) {
+                    resultStorage.reset();
+                    timeSerde.serialize(aTime, out);
+                    result.set(resultStorage);
+                } else {
+                    handleParseError(utf8Ptr, result);
+                }
+                break;
+            default:
+                handleUnsupportedType(inputType, result);
+                break;
+        }
+    }
+
+    protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+        int stringLength = textPtr.getUTF8Length();
+        // the string to be parsed should be at least 6 characters: hhmmss
+        if (stringLength < 6) {
+            return false;
+        }
+        try {
+            int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+                    stringLength);
+            if (chronon < 0) {
+                chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+            result.setValue(chronon);
+            return true;
+        } catch (HyracksDataException e) {
+            return false;
+        }
+    }
+
+    @Override
+    protected final BuiltinType getTargetType() {
+        return BuiltinType.ATIME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 18d55aa..ca42280 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -265,7 +265,9 @@
 import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
@@ -285,6 +287,7 @@
 import org.apache.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorWithFormatDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ClosedRecordConstructorDescriptor;
@@ -1052,8 +1055,11 @@
         fc.add(ACircleConstructorDescriptor.FACTORY);
         fc.add(ARectangleConstructorDescriptor.FACTORY);
         fc.add(ATimeConstructorDescriptor.FACTORY);
+        fc.add(ATimeConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADateConstructorDescriptor.FACTORY);
+        fc.add(ADateConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADateTimeConstructorDescriptor.FACTORY);
+        fc.add(ADateTimeConstructorWithFormatDescriptor.FACTORY);
         fc.add(ADurationConstructorDescriptor.FACTORY);
         fc.add(AYearMonthDurationConstructorDescriptor.FACTORY);
         fc.add(ADayTimeDurationConstructorDescriptor.FACTORY);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12366
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I746cc04d25bda808094fd25168392772a0fba728
Gerrit-Change-Number: 12366
Gerrit-PatchSet: 4
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: merged