You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by zs...@apache.org on 2021/10/01 06:17:42 UTC

[ignite] branch sql-calcite updated: IGNITE-15534 Calcite. Actualize existing ScriptTestSuite tests - Fixes #9439.

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

zstan pushed a commit to branch sql-calcite
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/sql-calcite by this push:
     new fe11fdd  IGNITE-15534 Calcite. Actualize existing ScriptTestSuite tests - Fixes #9439.
fe11fdd is described below

commit fe11fddaa7ff6a5b6944668ae8d67ffb6db91df5
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Fri Oct 1 09:16:28 2021 +0300

    IGNITE-15534 Calcite. Actualize existing ScriptTestSuite tests - Fixes #9439.
    
    Signed-off-by: zstan <st...@gmail.com>
---
 .../query/calcite/logical/SqlScriptRunner.java     |  35 ++-
 .../ignite/testsuites/IgniteCalciteTestSuite.java  |   2 +
 .../sql/aggregate/aggregates/test_aggr_string.test |   3 +-
 .../aggregate/aggregates/test_aggregate_types.test |   1 -
 .../aggregate/aggregates/test_null_aggregates.test |   2 +-
 .../sql/aggregate/aggregates/test_perfect_ht.test  |   2 +-
 .../sql/aggregate/aggregates/test_scalar_aggr.test |   2 +-
 .../test/sql/aggregate/group/test_group_by.test    |   7 +-
 .../sql/aggregate/group/test_group_by.test_ignore  |   6 +-
 .../sql/aggregate/group/test_group_by_alias.test   |  58 ----
 ..._alias.test => test_group_by_alias.test_ignore} |  30 +-
 .../having/test_scalar_having.test_ignore          |  38 +++
 .../src/test/sql/cast/test_boolean_cast.test       |  90 ++----
 ...ean_cast.test => test_boolean_cast.test_ignore} |  72 ++---
 ...cast.test => test_exponent_in_cast.test_ignore} |   3 +-
 .../src/test/sql/cast/test_string_cast.test        |  10 +-
 .../calcite/src/test/sql/delete/test_delete.test   |  19 --
 ...e_delete.test => test_large_delete.test_ignore} |   1 +
 ...letes.test => test_segment_deletes.test_ignore} |   3 +-
 ..._clause.test => test_filter_clause.test_ignore} |   3 +-
 .../src/test/sql/filter/test_obsolete_filters.test |   9 -
 ...ters.test => test_obsolete_filters.test_ignore} |   1 +
 ...rs.test => test_transitive_filters.test_ignore} |   1 +
 ...map.test_slow => test_zonemap.test_slow_ignore} |   1 +
 .../src/test/sql/function/date/date_part.test      |  21 +-
 .../test/sql/function/date/date_part.test_ignore   |  16 +-
 .../src/test/sql/function/date/test_extract.test   |  37 ++-
 .../test/sql/function/generic/test_between.test    |   8 +-
 .../test/sql/function/generic/test_coalesce.test   |   3 +-
 .../src/test/sql/function/generic/test_in.test     |  28 +-
 .../test/sql/function/numeric/test_floor_ceil.test |  26 +-
 .../src/test/sql/function/numeric/test_trigo.test  |  60 ++--
 .../sql/function/operator/test_arithmetic.test     |  28 +-
 .../src/test/sql/function/string/regex_search.test |  13 -
 ...{regex_search.test => regex_search.test_ignore} |   1 +
 .../test/sql/function/string/test_caseconvert.test |   2 +-
 .../src/test/sql/function/string/test_concat.test  |   8 +-
 .../src/test/sql/function/string/test_length.test  |   4 +-
 .../test/sql/insert/big_insert_rollback.test_slow  |  25 --
 .../src/test/sql/insert/insert_rollback.test       |  25 --
 .../insert/interleaved_insert_rollback.test_slow   |  49 ---
 .../{null_values.test => null_values.test_ignore}  |   4 +-
 .../src/test/sql/insert/test_big_insert.test       |   3 +-
 .../{test_insert.test => test_insert.test_ignore}  |   1 +
 .../src/test/sql/insert/test_insert_invalid.test   |   9 -
 .../src/test/sql/insert/test_insert_query.test     |   2 +-
 ...sert_type.test => test_insert_type.test_ignore} |   1 +
 .../sql/insert/unaligned_interleaved_appends.test  |  81 -----
 ... test_full_outer_join_many_matches.test_ignore} |   1 +
 .../src/test/sql/join/inner/test_eq_ineq_join.test |  16 -
 ...neq_join.test => test_eq_ineq_join.test_ignore} |   1 +
 ...cates.test => test_join_duplicates.test_ignore} |   1 +
 ...join_types.test => test_join_types.test_ignore} |   6 +-
 ...{test_lt_join.test => test_lt_join.test_ignore} |   1 +
 ...st => test_unequal_join_duplicates.test_ignore} |   5 +-
 .../src/test/sql/join/inner/test_using_chain.test  |  41 ---
 ...ing_chain.test => test_using_chain.test_ignore} |   1 +
 .../src/test/sql/join/inner/test_using_join.test   |  21 --
 ...using_join.test => test_using_join.test_ignore} |   3 +
 .../sql/join/left_outer/left_join_issue_1172.test  |  27 +-
 ...types.test => test_mark_join_types.test_ignore} |   6 +-
 ...{natural_join.test => natural_join.test_ignore} |   1 +
 ... => test_cross_product_parallelism.test_ignore} |   1 +
 .../src/test/sql/join/test_not_distinct_from.test  | 174 +++-------
 ...rom.test => test_not_distinct_from.test_ignore} | 127 ++++----
 modules/calcite/src/test/sql/order/test_limit.test | 143 +--------
 .../{test_limit.test => test_limit.test_ignore}    |   4 +-
 .../src/test/sql/order/test_nulls_first.test       |   2 +-
 .../calcite/src/test/sql/order/test_order_by.test  |   2 +-
 .../src/test/sql/order/test_order_by.test_ignore   |   2 +-
 .../test/sql/order/test_order_by_exceptions.test   |   2 +-
 ...der_large.test => test_order_large.test_ignore} |   1 +
 .../src/test/sql/order/test_order_pragma.test      |  18 +-
 ...> test_order_variable_size_payload.test_ignore} |   3 +
 .../test/sql/subquery/any_all/test_any_all.test    |  12 +-
 ...ll.test => test_correlated_any_all.test_ignore} |   1 +
 ...ny_all.test => test_scalar_any_all.test_ignore} |   1 +
 .../test/sql/subquery/any_all/test_scalar_in.test  |  48 +--
 ...t_scalar_in.test => test_scalar_in.test_ignore} |   1 +
 ..._not_in.test => test_simple_not_in.test_ignore} |   1 +
 .../any_all/test_uncorrelated_all_subquery.test    | 136 +++-----
 ... => test_uncorrelated_all_subquery.test_ignore} | 161 +++++-----
 .../any_all/test_uncorrelated_any_subquery.test    |  90 ++----
 ... => test_uncorrelated_any_subquery.test_ignore} |  86 ++---
 .../subquery/exists/test_correlated_exists.test    | 115 +------
 ...sts.test => test_correlated_exists.test_ignore} |  73 +++--
 .../sql/subquery/exists/test_scalar_exists.test    |  14 +-
 .../exists/test_uncorrelated_exists_subquery.test  |  14 +-
 .../scalar/test_complex_correlated_subquery.test   | 111 -------
 ...> test_complex_correlated_subquery.test_ignore} |   6 +
 .../test_complex_nested_correlated_subquery.test   |  28 +-
 ...complex_nested_correlated_subquery.test_ignore} |   5 +-
 .../scalar/test_correlated_aggregate_subquery.test | 350 +--------------------
 ...test_correlated_aggregate_subquery.test_ignore} |  22 +-
 .../subquery/scalar/test_correlated_subquery.test  |  81 +----
 ...y.test => test_correlated_subquery.test_ignore} |   2 +
 .../scalar/test_correlated_subquery_cte.test       | 122 +------
 ...st => test_correlated_subquery_cte.test_ignore} |   2 +
 .../scalar/test_correlated_subquery_where.test     |   4 +-
 .../subquery/scalar/test_count_star_subquery.test  |  82 +----
 ...y.test => test_count_star_subquery.test_ignore} |   2 +
 .../scalar/test_grouped_correlated_subquery.test   | 123 +-------
 ...> test_grouped_correlated_subquery.test_ignore} |  56 ++--
 .../sql/subquery/scalar/test_join_in_subquery.test |   2 +-
 ...st => test_many_correlated_columns.test_ignore} |   0
 ...=> test_nested_correlated_subquery.test_ignore} |   0
 .../sql/subquery/scalar/test_scalar_subquery.test  |  56 +---
 ...query.test => test_scalar_subquery.test_ignore} |  25 +-
 .../subquery/scalar/test_scalar_subquery_cte.test  |  56 +---
 ...e.test => test_scalar_subquery_cte.test_ignore} |  19 +-
 .../scalar/test_tpcds_correlated_subquery.test     |   7 -
 ... => test_tpcds_correlated_subquery.test_ignore} |   4 +-
 .../scalar/test_uncorrelated_scalar_subquery.test  |  86 -----
 ... test_uncorrelated_scalar_subquery.test_ignore} |   1 +
 .../scalar/test_uncorrelated_varchar_subquery.test |   7 +-
 .../sql/subquery/scalar/test_update_subquery.test  |  12 -
 ...query.test => test_update_subquery.test_ignore} |   0
 ...> test_varchar_correlated_subquery.test_ignore} |   0
 ...t => test_window_function_subquery.test_ignore} |   3 +-
 ...est_aliasing.test => test_aliasing.test_ignore} |   1 +
 .../subquery/table/test_nested_table_subquery.test |   4 +-
 .../sql/subquery/table/test_table_subquery.test    |  18 +-
 ...bquery.test => test_table_subquery.test_ignore} |   9 +-
 .../blob/{test_blob.test => test_blob.test_ignore} |   5 +-
 ...t_blob_cast.test => test_blob_cast.test_ignore} |   7 +-
 ...unction.test => test_blob_function.test_ignore} |  25 +-
 ...perator.test => test_blob_operator.test_ignore} |   3 +-
 ...ob_string.test => test_blob_string.test_ignore} |   3 +-
 ...{date_parsing.test => date_parsing.test_ignore} |   1 +
 ...est_bc_dates.test => test_bc_dates.test_ignore} |  25 +-
 .../date/{test_date.test => test_date.test_ignore} |   1 +
 ...nstants.test => interval_constants.test_ignore} |   1 +
 ...est_interval.test => test_interval.test_ignore} |   1 +
 ...ion.test => test_interval_addition.test_ignore} |   1 +
 ...n.test => test_interval_comparison.test_ignore} |   1 +
 ...rval_ops.test => test_interval_ops.test_ignore} |   1 +
 .../list/{array_agg.test => array_agg.test_ignore} |   1 +
 .../test/sql/types/list/lineitem_list.test_slow    | 120 -------
 .../list/{list.test_slow => list.test_slow_ignore} |   6 +-
 ...aggregates.test => list_aggregates.test_ignore} |   3 +-
 .../src/test/sql/types/list/test_list_extract.test | 182 -----------
 .../src/test/sql/types/list/test_list_index.test   |  62 ----
 .../src/test/sql/types/list/test_nested_list.test  | 265 ----------------
 .../src/test/sql/types/list/test_scalar_list.test  | 140 ---------
 .../src/test/sql/types/null/test_boolean_null.test |   2 +-
 .../calcite/src/test/sql/types/null/test_null.test |   6 +-
 ...t_null_aggr.test => test_null_aggr.test_ignore} |   1 +
 .../time/{test_time.test => test_time.test_ignore} |   1 +
 ...{time_parsing.test => time_parsing.test_ignore} |   4 +-
 ...{bc_timestamp.test => bc_timestamp.test_ignore} |   1 +
 ...p.test => test_incorrect_timestamp.test_ignore} |   1 +
 ...t_timestamp.test => test_timestamp.test_ignore} |   1 +
 ...stamp_ms.test => test_timestamp_ms.test_ignore} |   1 +
 .../sql/update/null_update_merge_transaction.test  | 120 -------
 ...ate.test => test_big_string_update.test_ignore} |   0
 .../sql/update/test_big_table_update.test_slow     | 111 -------
 ...ll_update.test => test_null_update.test_ignore} |   1 +
 ...est => test_repeated_string_update.test_ignore} |   1 +
 ..._update.test => test_string_update.test_ignore} |   1 +
 ...=> test_string_update_many_strings.test_ignore} |   1 +
 ...ll.test => test_string_update_null.test_ignore} |   1 +
 ...est => test_string_update_rollback.test_ignore} |   1 +
 ...> test_string_update_rollback_null.test_ignore} |   1 +
 .../{test_update.test => test_update.test_ignore}  |   1 +
 ...t => test_update_delete_same_tuple.test_ignore} |   1 +
 ...date_from.test => test_update_from.test_ignore} |   1 +
 ....test => test_update_many_updaters.test_ignore} |   1 +
 ...=> test_update_many_updaters_nulls.test_ignore} |   1 +
 ...update_mix.test => test_update_mix.test_ignore} |   1 +
 .../sql/update/test_update_same_string_value.test  |  41 ---
 ...lue.test => test_update_same_value.test_ignore} |   1 +
 171 files changed, 890 insertions(+), 3791 deletions(-)

diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/logical/SqlScriptRunner.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/logical/SqlScriptRunner.java
index 4fa2416..9ff891d 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/logical/SqlScriptRunner.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/logical/SqlScriptRunner.java
@@ -27,6 +27,7 @@ import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -67,6 +68,21 @@ public class SqlScriptRunner {
     /** Default schema. */
     private static final String schemaPublic = "PUBLIC";
 
+    /** Comparator for "rowsort" sort mode. */
+    private static final Comparator<List<?>> ROW_COMPARATOR = (r1, r2) -> {
+        int rows = r1.size();
+
+        for (int i = 0; i < rows; ++i) {
+            String s1 = String.valueOf(r1.get(i));
+            String s2 = String.valueOf(r2.get(i));
+
+            if (!s1.equals(s2))
+                return s1.compareTo(s2);
+        }
+
+        return 0;
+    };
+
     /** Test script path. */
     private final Path test;
 
@@ -385,7 +401,6 @@ public class SqlScriptRunner {
         /** {@inheritDoc} */
         @Override void execute() {
             for (String qry : queries) {
-
                 String[] toks = qry.split("\\s+");
 
                 if (ignoredStmts.contains(toks[0])) {
@@ -564,19 +579,10 @@ public class SqlScriptRunner {
         /** */
         void checkResult(List<List<?>> res) {
             if (sortType == SortType.ROWSORT) {
-                res.sort((l1, l2) -> {
-                    int rows = l1.size();
-
-                    for (int i = 0; i < rows; ++i) {
-                        String s1 = String.valueOf(l1.get(i));
-                        String s2 = String.valueOf(l2.get(i));
+                res.sort(ROW_COMPARATOR);
 
-                        if (!s1.equals(s2))
-                            return s1.compareTo(s2);
-                    }
-
-                    return 0;
-                });
+                if (expectedRes != null)
+                    expectedRes.sort(ROW_COMPARATOR);
             }
 
             if (expectedHash != null)
@@ -612,6 +618,9 @@ public class SqlScriptRunner {
 
         /** */
         private void checkEquals(String msg, String expectedStr, Object actual) {
+            if (actual == null && String.valueOf(actual).equalsIgnoreCase(expectedStr))
+                return;
+
             if (actual != null ^ expectedStr != null)
                 throw new AssertionError(msg);
 
diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
index 25b827c..661d0b5 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
@@ -39,6 +39,8 @@ import org.junit.runners.Suite;
     QueryCheckerTest.class,
     SqlDdlParserTest.class,
     IgniteSqlFunctionsTest.class,
+
+    ScriptTestSuite.class,
 })
 public class IgniteCalciteTestSuite {
 }
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
index 6753827..29c005b 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
@@ -29,7 +29,7 @@ SELECT a, COUNT(*), COUNT(s) FROM test GROUP BY a ORDER BY a;
 
 # group by the strings
 query TR
-SELECT s, SUM(a) FROM test GROUP BY s ORDER BY s;
+SELECT s, SUM(a) FROM test GROUP BY s ORDER BY s NULLS LAST;
 ----
 hello	11.000000
 world	12.000000
@@ -79,3 +79,4 @@ SELECT MAX(s), MIN(s) FROM test_strings2;
 ----
 aa	A
 
+
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggregate_types.test b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggregate_types.test
index f36e825..7496c6e 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggregate_types.test
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggregate_types.test
@@ -144,4 +144,3 @@ NULL
 22
 22
 22.000000
-
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_null_aggregates.test b/modules/calcite/src/test/sql/aggregate/aggregates/test_null_aggregates.test
index 9dea875..360b941 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_null_aggregates.test
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_null_aggregates.test
@@ -309,7 +309,7 @@ statement ok
 INSERT INTO t1 VALUES(9196517019233481682,NULL);
 
 query IIII
-SELECT c0, sum(c1), min(c1), max(c1) FROM t1 GROUP BY c0 ORDER BY 1, 2, 3, 4
+SELECT c0, sum(c1), min(c1), max(c1) FROM t1 GROUP BY c0 ORDER BY 1 NULLS LAST, 2, 3, 4
 ----
 -9121942514766415310	NULL	NULL	NULL
 -9113483941634330359	NULL	NULL	NULL
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_perfect_ht.test b/modules/calcite/src/test/sql/aggregate/aggregates/test_perfect_ht.test
index f0580f5..a1724d2 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_perfect_ht.test
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_perfect_ht.test
@@ -15,7 +15,7 @@ statement ok
 INSERT INTO timeseries VALUES (1996, 10), (1997, 12), (1996, 20), (2001, 30), (NULL, 1), (1996, NULL);
 
 query IIII
-SELECT year, SUM(val), COUNT(val), COUNT(*) FROM timeseries GROUP BY year ORDER BY year;
+SELECT year, SUM(val), COUNT(val), COUNT(*) FROM timeseries GROUP BY year ORDER BY year NULLS LAST;
 ----
 1996	30	2	3
 1997	12	1	1
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_scalar_aggr.test b/modules/calcite/src/test/sql/aggregate/aggregates/test_scalar_aggr.test
index 81d09b8..cfa25a6 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_scalar_aggr.test
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_scalar_aggr.test
@@ -4,7 +4,7 @@
 
 
 query III
-SELECT NULL as a, NULL as b, 1 as id UNION SELECT CAST('00:00:00' AS TIME) as a, CAST('12:34:56' AS TIME) as b, 2 as id ORDER BY 1
+SELECT NULL as a, NULL as b, 1 as id UNION SELECT CAST('00:00:00' AS TIME) as a, CAST('12:34:56' AS TIME) as b, 2 as id ORDER BY 1 NULLS LAST
 ----
 00:00:00	12:34:56	2
 NULL	NULL	1
diff --git a/modules/calcite/src/test/sql/aggregate/group/test_group_by.test b/modules/calcite/src/test/sql/aggregate/group/test_group_by.test
index 335c7aa..e1fbc5d 100644
--- a/modules/calcite/src/test/sql/aggregate/group/test_group_by.test
+++ b/modules/calcite/src/test/sql/aggregate/group/test_group_by.test
@@ -153,9 +153,8 @@ statement error
 SELECT 1 AS k, SUM(i) FROM integers GROUP BY k+1 ORDER BY 2;
 
 # group by column refs should be recognized, even if one uses an explicit table specifier and the other does not
-query IR
+query II
 SELECT test.b, SUM(a) FROM test GROUP BY b ORDER BY COUNT(a) DESC;
 ----
-21	48.000000
-22	24.000000
-
+21	48
+22	24
diff --git a/modules/calcite/src/test/sql/aggregate/group/test_group_by.test_ignore b/modules/calcite/src/test/sql/aggregate/group/test_group_by.test_ignore
index c9d7179..22efd58 100644
--- a/modules/calcite/src/test/sql/aggregate/group/test_group_by.test_ignore
+++ b/modules/calcite/src/test/sql/aggregate/group/test_group_by.test_ignore
@@ -174,9 +174,9 @@ statement error
 SELECT 1 AS k, SUM(i) FROM integers GROUP BY k+1 ORDER BY 2;
 
 # group by column refs should be recognized, even if one uses an explicit table specifier and the other does not
-query IR
+query II
 SELECT test.b, SUM(a) FROM test GROUP BY b ORDER BY COUNT(a) DESC;
 ----
-21	48.000000
-22	24.000000
+21	48
+22	24
 
diff --git a/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test b/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test
index 72d7fe6..4b70467 100644
--- a/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test
+++ b/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test
@@ -3,59 +3,11 @@
 # group: [group]
 
 statement ok
-PRAGMA enable_verification
-
-statement ok
 CREATE TABLE integers(i INTEGER)
 
 statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
-# use alias in HAVING clause
-# CONTROVERSIAL: this query DOES NOT work in PostgreSQL
-query IR
-SELECT i % 2 AS k, SUM(i) FROM integers WHERE i IS NOT NULL GROUP BY k HAVING k>0;
-----
-1	4.000000
-
-# this is identical to this query
-# CONTROVERSIAL: this query does not work in MonetDB
-query IR
-SELECT i % 2 AS k, SUM(i) FROM integers WHERE i IS NOT NULL GROUP BY k HAVING i%2>0;
-----
-1	4.000000
-
-# select groups by constant (similar to order by constant)
-query IR
-SELECT i % 2 AS k, SUM(i) FROM integers WHERE i IS NOT NULL GROUP BY 1 HAVING i%2>0;
-----
-1	4.000000
-
-# constant out of range
-statement error
-SELECT i % 2 AS k, SUM(i) FROM integers WHERE i IS NOT NULL GROUP BY 42 HAVING i%2>0;
-
-# entry in GROUP BY should refer to base column
-# ...BUT the alias in ORDER BY should refer to the alias from the select list
-# note that both Postgres and MonetDB reject this query because of ambiguity. SQLite accepts it though so we do
-# too.
-query IIR
-SELECT i, i % 2 AS i, SUM(i) FROM integers GROUP BY i ORDER BY i, 3;
-----
-NULL	NULL	NULL
-2	0	2.000000
-1	1	1.000000
-3	1	3.000000
-
-# changing the name of the alias makes it more explicit what should happen
-query IIR
-SELECT i, i % 2 AS k, SUM(i) FROM integers GROUP BY i ORDER BY k, 3;
-----
-NULL	NULL	NULL
-2	0	2.000000
-1	1	1.000000
-3	1	3.000000
-
 # this now orders by the actual grouping column
 query IIR
 SELECT i, i % 2 AS k, SUM(i) FROM integers GROUP BY i ORDER BY i;
@@ -84,13 +36,3 @@ NULL	NULL
 # aggregate
 statement error
 SELECT (10-i) AS k, SUM(i) FROM integers GROUP BY k ORDER BY i;
-
-# we can manually get this behavior by pushing FIRST
-query IR
-SELECT (10-i) AS k, SUM(i) FROM integers GROUP BY k ORDER BY FIRST(i);
-----
-NULL	NULL
-9	1.000000
-8	2.000000
-7	3.000000
-
diff --git a/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test b/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test_ignore
similarity index 70%
copy from modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test
copy to modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test_ignore
index 72d7fe6..846468d 100644
--- a/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test
+++ b/modules/calcite/src/test/sql/aggregate/group/test_group_by_alias.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/aggregate/group/test_group_by_alias.test
 # description: Test aliases in group by/aggregation
 # group: [group]
+# Ignored: https://issues.apache.org/jira/browse/IGNITE-14885
 
 statement ok
 PRAGMA enable_verification
@@ -56,35 +57,6 @@ NULL	NULL	NULL
 1	1	1.000000
 3	1	3.000000
 
-# this now orders by the actual grouping column
-query IIR
-SELECT i, i % 2 AS k, SUM(i) FROM integers GROUP BY i ORDER BY i;
-----
-NULL	NULL	NULL
-1	1	1.000000
-2	0	2.000000
-3	1	3.000000
-
-# cannot use GROUP BY column in an aggregation...
-statement error
-SELECT i % 2 AS k, SUM(k) FROM integers GROUP BY k
-
-# ...unless it is one of the base columns
-query IR
-SELECT i, SUM(i) FROM integers GROUP BY i ORDER BY i
-----
-NULL	NULL
-1	1.000000
-2	2.000000
-3	3.000000
-
-# ORDER on a non-grouping column
-# this query is refused by Postgres and MonetDB
-# but SQLite resolves it by first pushing a "FIRST(i)" aggregate into the projection, and then ordering by that
-# aggregate
-statement error
-SELECT (10-i) AS k, SUM(i) FROM integers GROUP BY k ORDER BY i;
-
 # we can manually get this behavior by pushing FIRST
 query IR
 SELECT (10-i) AS k, SUM(i) FROM integers GROUP BY k ORDER BY FIRST(i);
diff --git a/modules/calcite/src/test/sql/aggregate/having/test_scalar_having.test_ignore b/modules/calcite/src/test/sql/aggregate/having/test_scalar_having.test_ignore
index d3c4dd5..c3988a8 100644
--- a/modules/calcite/src/test/sql/aggregate/having/test_scalar_having.test_ignore
+++ b/modules/calcite/src/test/sql/aggregate/having/test_scalar_having.test_ignore
@@ -5,6 +5,8 @@
 # CONTROVERSIAL: HAVING without GROUP BY works in PostgreSQL, but not in SQLite
 # scalar HAVING queries
 # constants only
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14541
+
 query I
 SELECT 42 HAVING 42 > 20
 ----
@@ -79,3 +81,39 @@ query R
 SELECT SUM(a) FROM test HAVING COUNT(*)>10;
 ----
 
+# HAVING with condition on group
+query IR
+SELECT b, SUM(a) AS sum FROM test GROUP BY b HAVING b=21 ORDER BY b;
+----
+21	12.000000
+
+# HAVING with condition on sum
+query IR
+SELECT b, SUM(a) FROM test GROUP BY b HAVING SUM(a) < 20 ORDER BY b;
+----
+21	12.000000
+
+# HAVING without alias
+query IR
+SELECT b, SUM(a) AS sum FROM test GROUP BY b HAVING SUM(a) < 20 ORDER BY b;
+----
+21	12.000000
+
+# HAVING on column not in aggregate
+query IR
+SELECT b, SUM(a) AS sum FROM test GROUP BY b HAVING COUNT(*) = 1 ORDER BY b;
+----
+21	12.000000
+
+# expression in having
+query IR
+SELECT b, SUM(a) FROM test GROUP BY b HAVING SUM(a)+10>28;
+----
+22	24.000000
+
+# uncorrelated subquery in having
+query IR
+SELECT b, SUM(a) FROM test GROUP BY b HAVING SUM(a)>(SELECT SUM(t.a)*0.5 FROM test t);
+----
+22	24.000000
+
diff --git a/modules/calcite/src/test/sql/cast/test_boolean_cast.test b/modules/calcite/src/test/sql/cast/test_boolean_cast.test
index 6fb092f..d72fde4 100644
--- a/modules/calcite/src/test/sql/cast/test_boolean_cast.test
+++ b/modules/calcite/src/test/sql/cast/test_boolean_cast.test
@@ -1,6 +1,7 @@
 # name: test/sql/cast/test_boolean_cast.test
 # description: Test boolean casts
 # group: [cast]
+# Ignored with https://issues.apache.org/jira/browse/IGNITE-15559
 
 statement ok
 PRAGMA enable_verification
@@ -8,42 +9,32 @@ PRAGMA enable_verification
 query T
 SELECT CAST(1=1 AS VARCHAR)
 ----
-true
+TRUE
 
 query T
 SELECT CAST(1=0 AS VARCHAR)
 ----
-false
+FALSE
 
 query T
 SELECT CAST('true' AS BOOLEAN)
 ----
-1
-
-query T
-SELECT CAST('t' AS BOOLEAN)
-----
-1
+true
 
 query T
 SELECT CAST('TRUE' AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST('false' AS BOOLEAN)
 ----
-0
-
-query T
-SELECT CAST('f' AS BOOLEAN)
-----
-0
+false
 
 query T
 SELECT CAST('FALSE' AS BOOLEAN)
 ----
-0
+false
 
 statement error
 SELECT CAST('12345' AS BOOLEAN)
@@ -51,120 +42,89 @@ SELECT CAST('12345' AS BOOLEAN)
 query T
 SELECT CAST(CAST('12345' AS INTEGER) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS INTEGER) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS tinyint) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS tinyint) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS smallint) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS smallint) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS integer) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS integer) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS bigint) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS bigint) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal(1,0)) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal(1,0)) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal(9,0)) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal(9,0)) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal(38,0)) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal(38,0)) AS BOOLEAN)
 ----
-0
-
-query T
-SELECT CAST(CAST('1' AS float) AS BOOLEAN)
-----
-1
-
-query T
-SELECT CAST(CAST('0' AS float) AS BOOLEAN)
-----
-0
-
-query T
-SELECT CAST(CAST('1' AS double) AS BOOLEAN)
-----
-1
-
-query T
-SELECT CAST(CAST('0' AS double) AS BOOLEAN)
-----
-0
-
-query T
-SELECT CAST(CAST('1' AS HUGEINT) AS BOOLEAN)
-----
-1
-
-query T
-SELECT CAST(CAST('0' AS HUGEINT) AS BOOLEAN)
-----
-0
-
+false
diff --git a/modules/calcite/src/test/sql/cast/test_boolean_cast.test b/modules/calcite/src/test/sql/cast/test_boolean_cast.test_ignore
similarity index 89%
copy from modules/calcite/src/test/sql/cast/test_boolean_cast.test
copy to modules/calcite/src/test/sql/cast/test_boolean_cast.test_ignore
index 6fb092f..7981202 100644
--- a/modules/calcite/src/test/sql/cast/test_boolean_cast.test
+++ b/modules/calcite/src/test/sql/cast/test_boolean_cast.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/cast/test_boolean_cast.test
 # description: Test boolean casts
 # group: [cast]
+# Ignored with https://issues.apache.org/jira/browse/IGNITE-15559
 
 statement ok
 PRAGMA enable_verification
@@ -8,42 +9,32 @@ PRAGMA enable_verification
 query T
 SELECT CAST(1=1 AS VARCHAR)
 ----
-true
+TRUE
 
 query T
 SELECT CAST(1=0 AS VARCHAR)
 ----
-false
+FALSE
 
 query T
 SELECT CAST('true' AS BOOLEAN)
 ----
-1
-
-query T
-SELECT CAST('t' AS BOOLEAN)
-----
-1
+true
 
 query T
 SELECT CAST('TRUE' AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST('false' AS BOOLEAN)
 ----
-0
-
-query T
-SELECT CAST('f' AS BOOLEAN)
-----
-0
+false
 
 query T
 SELECT CAST('FALSE' AS BOOLEAN)
 ----
-0
+false
 
 statement error
 SELECT CAST('12345' AS BOOLEAN)
@@ -51,120 +42,119 @@ SELECT CAST('12345' AS BOOLEAN)
 query T
 SELECT CAST(CAST('12345' AS INTEGER) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS INTEGER) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS tinyint) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS tinyint) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS smallint) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS smallint) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS integer) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS integer) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS bigint) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS bigint) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal(1,0)) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal(1,0)) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal(9,0)) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal(9,0)) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS decimal(38,0)) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS decimal(38,0)) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS float) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS float) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS double) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS double) AS BOOLEAN)
 ----
-0
+false
 
 query T
 SELECT CAST(CAST('1' AS HUGEINT) AS BOOLEAN)
 ----
-1
+true
 
 query T
 SELECT CAST(CAST('0' AS HUGEINT) AS BOOLEAN)
 ----
-0
-
+false
diff --git a/modules/calcite/src/test/sql/cast/test_exponent_in_cast.test b/modules/calcite/src/test/sql/cast/test_exponent_in_cast.test_ignore
similarity index 90%
rename from modules/calcite/src/test/sql/cast/test_exponent_in_cast.test
rename to modules/calcite/src/test/sql/cast/test_exponent_in_cast.test_ignore
index 58e7a21..f5764ae3 100644
--- a/modules/calcite/src/test/sql/cast/test_exponent_in_cast.test
+++ b/modules/calcite/src/test/sql/cast/test_exponent_in_cast.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/cast/test_exponent_in_cast.test
 # description: Test exponents in cast
 # group: [cast]
+# Ignored https://issues.apache.org/jira/browse/IGNITE-15560
 
 statement ok
 PRAGMA enable_verification
@@ -41,4 +42,4 @@ SELECT CAST('1e1' AS DOUBLE);
 query I
 SELECT CAST('   1e1' AS DOUBLE);
 ----
-10.0
\ No newline at end of file
+10.0
diff --git a/modules/calcite/src/test/sql/cast/test_string_cast.test b/modules/calcite/src/test/sql/cast/test_string_cast.test
index b421001..a6a1c92 100644
--- a/modules/calcite/src/test/sql/cast/test_string_cast.test
+++ b/modules/calcite/src/test/sql/cast/test_string_cast.test
@@ -8,9 +8,9 @@ PRAGMA enable_verification
 query TTT
 SELECT (1=1)::VARCHAR, (1=0)::VARCHAR, NULL::BOOLEAN::VARCHAR
 ----
-true
-false
-NULL
+TRUE
+FALSE
+null
 
 query TTT
 SELECT 1::TINYINT::VARCHAR, 12::TINYINT::VARCHAR, (-125)::TINYINT::VARCHAR
@@ -43,13 +43,13 @@ SELECT 1::BIGINT::VARCHAR, 1244295295289253::BIGINT::VARCHAR, (-2000000111551166
 query TTT
 SELECT 2::FLOAT::VARCHAR, 0.5::FLOAT::VARCHAR, (-128.5)::FLOAT::VARCHAR
 ----
-2.0
+2
 0.5
 -128.5
 
 query TTT
 SELECT 2::DOUBLE::VARCHAR, 0.5::DOUBLE::VARCHAR, (-128.5)::DOUBLE::VARCHAR
 ----
-2.0
+2
 0.5
 -128.5
diff --git a/modules/calcite/src/test/sql/delete/test_delete.test b/modules/calcite/src/test/sql/delete/test_delete.test
index 268c532..2da09e6 100644
--- a/modules/calcite/src/test/sql/delete/test_delete.test
+++ b/modules/calcite/src/test/sql/delete/test_delete.test
@@ -30,23 +30,4 @@ SELECT COUNT(*) FROM a;
 ----
 1
 
-# now test rollback of deletion
-statement ok
-BEGIN TRANSACTION
-
-statement ok
-DELETE FROM a;
-
-query I
-SELECT COUNT(*) FROM a;
-----
-0
-
-statement ok
-ROLLBACK
-
-query I
-SELECT COUNT(*) FROM a;
-----
-1
 
diff --git a/modules/calcite/src/test/sql/delete/test_large_delete.test b/modules/calcite/src/test/sql/delete/test_large_delete.test_ignore
similarity index 82%
rename from modules/calcite/src/test/sql/delete/test_large_delete.test
rename to modules/calcite/src/test/sql/delete/test_large_delete.test_ignore
index fb25565..6e1b8d5 100644
--- a/modules/calcite/src/test/sql/delete/test_large_delete.test
+++ b/modules/calcite/src/test/sql/delete/test_large_delete.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/delete/test_large_delete.test
 # description: Test scan with large deletions
 # group: [delete]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 CREATE TABLE a AS SELECT * FROM range(0, 10000, 1) t1(i);
diff --git a/modules/calcite/src/test/sql/delete/test_segment_deletes.test b/modules/calcite/src/test/sql/delete/test_segment_deletes.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/delete/test_segment_deletes.test
rename to modules/calcite/src/test/sql/delete/test_segment_deletes.test_ignore
index b8e18fe..8cc8ed6 100644
--- a/modules/calcite/src/test/sql/delete/test_segment_deletes.test
+++ b/modules/calcite/src/test/sql/delete/test_segment_deletes.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/delete/test_segment_deletes.test
 # description: Test deletions
 # group: [delete]
+# https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok con1
 CREATE TABLE a(i INTEGER);
@@ -176,4 +177,4 @@ COMMIT
 query I con1
 SELECT COUNT(*) FROM a;
 ----
-20400
\ No newline at end of file
+20400
diff --git a/modules/calcite/src/test/sql/filter/test_filter_clause.test b/modules/calcite/src/test/sql/filter/test_filter_clause.test_ignore
similarity index 99%
rename from modules/calcite/src/test/sql/filter/test_filter_clause.test
rename to modules/calcite/src/test/sql/filter/test_filter_clause.test_ignore
index 6b261c1..58d2188 100644
--- a/modules/calcite/src/test/sql/filter/test_filter_clause.test
+++ b/modules/calcite/src/test/sql/filter/test_filter_clause.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/filter/test_filter_clause.test
 # description: Test aggregation with filter clause
 # group: [filter]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 PRAGMA enable_verification
@@ -539,4 +540,4 @@ insert into integers values (1),(1),(2),(2),(3),(4),(8);
 query I
 select  sum(distinct i) filter (where i >1 and i < 5) from integers;
 ----
-9
\ No newline at end of file
+9
diff --git a/modules/calcite/src/test/sql/filter/test_obsolete_filters.test b/modules/calcite/src/test/sql/filter/test_obsolete_filters.test
index 45ed607..a8561b6 100644
--- a/modules/calcite/src/test/sql/filter/test_obsolete_filters.test
+++ b/modules/calcite/src/test/sql/filter/test_obsolete_filters.test
@@ -191,14 +191,6 @@ query II
 SELECT * FROM integers WHERE a<>2 AND a=2
 ----
 
-query II
-SELECT * FROM integers WHERE 0
-----
-
-query II
-SELECT * FROM integers WHERE a<2 AND 0
-----
-
 # Test string expressions with obsolete filters
 statement ok
 CREATE TABLE strings(s VARCHAR)
@@ -277,4 +269,3 @@ query T
 SELECT * FROM strings WHERE s<>'world' AND s>='hello'
 ----
 hello
-
diff --git a/modules/calcite/src/test/sql/filter/test_obsolete_filters.test b/modules/calcite/src/test/sql/filter/test_obsolete_filters.test_ignore
similarity index 98%
copy from modules/calcite/src/test/sql/filter/test_obsolete_filters.test
copy to modules/calcite/src/test/sql/filter/test_obsolete_filters.test_ignore
index 45ed607..a2ec2e5 100644
--- a/modules/calcite/src/test/sql/filter/test_obsolete_filters.test
+++ b/modules/calcite/src/test/sql/filter/test_obsolete_filters.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/filter/test_obsolete_filters.test
 # description: Test expressions with obsolete filters
 # group: [filter]
+# Ignored https://issues.apache.org/jira/browse/IGNITE-15561
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/filter/test_transitive_filters.test b/modules/calcite/src/test/sql/filter/test_transitive_filters.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/filter/test_transitive_filters.test
rename to modules/calcite/src/test/sql/filter/test_transitive_filters.test_ignore
index 0055dfd..0a0f720 100644
--- a/modules/calcite/src/test/sql/filter/test_transitive_filters.test
+++ b/modules/calcite/src/test/sql/filter/test_transitive_filters.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/filter/test_transitive_filters.test
 # description: Test expressions with transitive filters
 # group: [filter]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/filter/test_zonemap.test_slow b/modules/calcite/src/test/sql/filter/test_zonemap.test_slow_ignore
similarity index 96%
rename from modules/calcite/src/test/sql/filter/test_zonemap.test_slow
rename to modules/calcite/src/test/sql/filter/test_zonemap.test_slow_ignore
index c1b5553..69a93bd 100644
--- a/modules/calcite/src/test/sql/filter/test_zonemap.test_slow
+++ b/modules/calcite/src/test/sql/filter/test_zonemap.test_slow_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/filter/test_zonemap.test_slow
 # description: Test expressions with transitive filters
 # group: [filter]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 # FIXME: temporarily removed because of too much memory usage on 32-bit
 # should be re-enabled when validity segments are no longer limited to 80 vectors
diff --git a/modules/calcite/src/test/sql/function/date/date_part.test b/modules/calcite/src/test/sql/function/date/date_part.test
index 57ad833..a98de38 100644
--- a/modules/calcite/src/test/sql/function/date/date_part.test
+++ b/modules/calcite/src/test/sql/function/date/date_part.test
@@ -31,6 +31,14 @@ SELECT EXTRACT(millennium FROM d) FROM dates;
 ----
 
 query I
+SELECT EXTRACT(microsecond FROM d) FROM dates;
+----
+
+query I
+SELECT EXTRACT(millisecond FROM d) FROM dates;
+----
+
+query I
 SELECT EXTRACT(second FROM d) FROM dates;
 ----
 
@@ -120,6 +128,18 @@ SELECT EXTRACT(second FROM d) as s FROM dates ORDER BY s;
 0
 
 query I
+SELECT EXTRACT(microsecond FROM d) FROM dates;
+----
+0
+0
+
+query I
+SELECT EXTRACT(millisecond FROM d) FROM dates;
+----
+0
+0
+
+query I
 SELECT EXTRACT(minute FROM d) as m FROM dates ORDER BY m;
 ----
 0
@@ -161,7 +181,6 @@ SELECT EXTRACT(quarter FROM d) as q FROM dates ORDER BY q;
 1
 4
 
-
 query I
 SELECT DAYOFMONTH(d) as d FROM dates ORDER BY d;
 ----
diff --git a/modules/calcite/src/test/sql/function/date/date_part.test_ignore b/modules/calcite/src/test/sql/function/date/date_part.test_ignore
index 4e0ce92..5cab434 100644
--- a/modules/calcite/src/test/sql/function/date/date_part.test_ignore
+++ b/modules/calcite/src/test/sql/function/date/date_part.test_ignore
@@ -32,11 +32,11 @@ SELECT EXTRACT(millennium FROM d) FROM dates;
 ----
 
 query I
-SELECT EXTRACT(microseconds FROM d) FROM dates;
+SELECT EXTRACT(microsecond FROM d) FROM dates;
 ----
 
 query I
-SELECT EXTRACT(milliseconds FROM d) FROM dates;
+SELECT EXTRACT(millisecond FROM d) FROM dates;
 ----
 
 query I
@@ -137,6 +137,18 @@ SELECT EXTRACT(second FROM d) as s FROM dates ORDER BY s;
 0
 
 query I
+SELECT EXTRACT(microsecond FROM d) FROM dates;
+----
+0
+0
+
+query I
+SELECT EXTRACT(millisecond FROM d) FROM dates;
+----
+0
+0
+
+query I
 SELECT EXTRACT(minute FROM d) as m FROM dates ORDER BY m;
 ----
 0
diff --git a/modules/calcite/src/test/sql/function/date/test_extract.test b/modules/calcite/src/test/sql/function/date/test_extract.test
index c15cd35..018d470 100644
--- a/modules/calcite/src/test/sql/function/date/test_extract.test
+++ b/modules/calcite/src/test/sql/function/date/test_extract.test
@@ -11,90 +11,101 @@ INSERT INTO dates VALUES (DATE '1993-08-14'), (NULL)
 # extract various parts of the date
 # year
 query I
-SELECT EXTRACT(year FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(year FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 1993
 NULL
 
 # month
 query I
-SELECT EXTRACT(month FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(month FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 8
 NULL
 
 # day
 query I
-SELECT EXTRACT(day FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(day FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 14
 NULL
 
 # decade
 query I
-SELECT EXTRACT(decade FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(decade FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 199
 NULL
 
 # century
 query I
-SELECT EXTRACT(century FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(century FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 20
 NULL
 
 # day of the week (Sunday = 1, Saturday = 7)
 query I
-SELECT EXTRACT(DOW FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(DOW FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 7
 NULL
 
 # day of the year (1 - 365/366)
 query I
-SELECT EXTRACT(DOY FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(DOY FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 226
 NULL
 
 # epoch
 query I
-SELECT EXTRACT(epoch FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(epoch FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 745286400
 NULL
 
 # isodow (Monday = 1, Sunday = 7)
 query I
-SELECT EXTRACT(ISODOW FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(ISODOW FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 6
 NULL
 
 # millenium (change of millenium is January 1, X001)
 query I
-SELECT EXTRACT(millennium FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(millennium FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 2
 NULL
 
 # timestamp variants all give 0 for date
 query I
-SELECT EXTRACT(second FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(second FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 0
 NULL
 
 query I
-SELECT EXTRACT(minute FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(minute FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 0
 NULL
 
 query I
-SELECT EXTRACT(hour FROM i) FROM dates ORDER BY i
+SELECT EXTRACT(hour FROM i) FROM dates ORDER BY i NULLS LAST
 ----
 0
 NULL
 
+query I
+SELECT EXTRACT(millisecond FROM i) FROM dates ORDER BY i NULLS LAST
+----
+0
+NULL
+
+query I
+SELECT EXTRACT(microsecond FROM i) FROM dates ORDER BY i NULLS LAST
+----
+0
+NULL
diff --git a/modules/calcite/src/test/sql/function/generic/test_between.test b/modules/calcite/src/test/sql/function/generic/test_between.test
index a9470a5..634c80b 100644
--- a/modules/calcite/src/test/sql/function/generic/test_between.test
+++ b/modules/calcite/src/test/sql/function/generic/test_between.test
@@ -63,7 +63,7 @@ statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
 query T
-SELECT i BETWEEN 1 AND 2 FROM integers ORDER BY i
+SELECT i BETWEEN 1 AND 2 FROM integers ORDER BY i NULLS LAST
 ----
 true
 true
@@ -71,15 +71,15 @@ false
 NULL
 
 query T
-SELECT i BETWEEN NULL AND 2 FROM integers ORDER BY i
+SELECT i BETWEEN NULL AND 2 FROM integers ORDER BY i NULLS FIRST
 ----
 NULL
 NULL
-false
 NULL
+false
 
 query T
-SELECT i BETWEEN 2 AND NULL FROM integers ORDER BY i
+SELECT i BETWEEN 2 AND NULL FROM integers ORDER BY i NULLS LAST
 ----
 false
 NULL
diff --git a/modules/calcite/src/test/sql/function/generic/test_coalesce.test b/modules/calcite/src/test/sql/function/generic/test_coalesce.test
index db50017..14da577 100644
--- a/modules/calcite/src/test/sql/function/generic/test_coalesce.test
+++ b/modules/calcite/src/test/sql/function/generic/test_coalesce.test
@@ -47,7 +47,6 @@ INSERT INTO test3 VALUES (null, 22), (12, null), (13, 22)
 query T
 SELECT COALESCE(CAST(a AS VARCHAR), 11) FROM test3 ORDER BY a;
 ----
+11
 12
 13
-11
-
diff --git a/modules/calcite/src/test/sql/function/generic/test_in.test b/modules/calcite/src/test/sql/function/generic/test_in.test
index 6fb0203..d423314 100644
--- a/modules/calcite/src/test/sql/function/generic/test_in.test
+++ b/modules/calcite/src/test/sql/function/generic/test_in.test
@@ -22,7 +22,7 @@ SELECT * FROM integers WHERE i IN (1, 2, 3, 4, 5, 6, 7, 8) ORDER BY i
 3
 
 query IT
-SELECT i, i IN (1, 2, 3, 4, 5, 6, 7, 8) FROM integers ORDER BY i
+SELECT i, i IN (1, 2, 3, 4, 5, 6, 7, 8) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	true
@@ -30,7 +30,7 @@ SELECT i, i IN (1, 2, 3, 4, 5, 6, 7, 8) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, i NOT IN (1, 2, 3, 4, 5, 6, 7, 8) FROM integers ORDER BY i
+SELECT i, i NOT IN (1, 2, 3, 4, 5, 6, 7, 8) FROM integers ORDER BY i NULLS LAST
 ----
 1	false
 2	false
@@ -38,7 +38,7 @@ SELECT i, i NOT IN (1, 2, 3, 4, 5, 6, 7, 8) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, i IN (1, 2, NULL, 4, 5, 6, 7, 8) FROM integers ORDER BY i
+SELECT i, i IN (1, 2, NULL, 4, 5, 6, 7, 8) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	true
@@ -46,7 +46,7 @@ SELECT i, i IN (1, 2, NULL, 4, 5, 6, 7, 8) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, i IN (i + 1) FROM integers ORDER BY i
+SELECT i, i IN (i + 1) FROM integers ORDER BY i NULLS LAST
 ----
 1	false
 2	false
@@ -54,7 +54,7 @@ SELECT i, i IN (i + 1) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, i IN (i + 1, 42, i) FROM integers ORDER BY i
+SELECT i, i IN (i + 1, 42, i) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	true
@@ -62,7 +62,7 @@ SELECT i, i IN (i + 1, 42, i) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, 1 IN (i - 1, i, i + 1) FROM integers ORDER BY i
+SELECT i, 1 IN (i - 1, i, i + 1) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	true
@@ -70,7 +70,7 @@ SELECT i, 1 IN (i - 1, i, i + 1) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, 1 NOT IN (i - 1, i, i + 1) FROM integers ORDER BY i
+SELECT i, 1 NOT IN (i - 1, i, i + 1) FROM integers ORDER BY i NULLS LAST
 ----
 1	false
 2	false
@@ -78,7 +78,7 @@ SELECT i, 1 NOT IN (i - 1, i, i + 1) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, i IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i
+SELECT i, i IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	true
@@ -86,7 +86,7 @@ SELECT i, i IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i
 NULL	NULL
 
 query IT
-SELECT i, i NOT IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i
+SELECT i, i NOT IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i NULLS LAST
 ----
 1	false
 2	false
@@ -94,7 +94,7 @@ SELECT i, i NOT IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY
 NULL	NULL
 
 query IT
-SELECT i, 1 IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i
+SELECT i, 1 IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	true
@@ -102,7 +102,7 @@ SELECT i, 1 IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i
 NULL	true
 
 query IT
-SELECT i, 1 NOT IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i
+SELECT i, 1 NOT IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY i NULLS LAST
 ----
 1	false
 2	false
@@ -110,7 +110,7 @@ SELECT i, 1 NOT IN (11, 12, 13, 14, 15, 16, 17, 18, 1, i) FROM integers ORDER BY
 NULL	false
 
 query IT
-SELECT i, i IN ((SELECT MAX(i) FROM integers), (SELECT MIN(i) FROM integers)) FROM integers ORDER BY i
+SELECT i, i IN ((SELECT MAX(i) FROM integers), (SELECT MIN(i) FROM integers)) FROM integers ORDER BY i NULLS LAST
 ----
 1	true
 2	false
@@ -118,7 +118,7 @@ SELECT i, i IN ((SELECT MAX(i) FROM integers), (SELECT MIN(i) FROM integers)) FR
 NULL	NULL
 
 query III
-SELECT i, (SELECT MAX(i) FROM integers WHERE i <> i1.i), (SELECT MIN(i) FROM integers WHERE i <= i1.i) FROM integers i1 ORDER BY i
+SELECT i, (SELECT MAX(i) FROM integers WHERE i <> i1.i), (SELECT MIN(i) FROM integers WHERE i <= i1.i) FROM integers i1 ORDER BY i NULLS LAST
 ----
 1	3	1
 2	3	1
@@ -126,7 +126,7 @@ SELECT i, (SELECT MAX(i) FROM integers WHERE i <> i1.i), (SELECT MIN(i) FROM int
 NULL	NULL	NULL
 
 query IT
-SELECT i, i IN ((SELECT MAX(i) FROM integers WHERE i <> i1.i), (SELECT MIN(i) FROM integers WHERE i <= i1.i)) FROM integers i1 ORDER BY i
+SELECT i, i IN ((SELECT MAX(i) FROM integers WHERE i <> i1.i), (SELECT MIN(i) FROM integers WHERE i <= i1.i)) FROM integers i1 ORDER BY i NULLS LAST
 ----
 1	true
 2	false
diff --git a/modules/calcite/src/test/sql/function/numeric/test_floor_ceil.test b/modules/calcite/src/test/sql/function/numeric/test_floor_ceil.test
index da338a7..fc0c010 100644
--- a/modules/calcite/src/test/sql/function/numeric/test_floor_ceil.test
+++ b/modules/calcite/src/test/sql/function/numeric/test_floor_ceil.test
@@ -9,7 +9,7 @@ statement ok
 INSERT INTO numbers VALUES (NULL),(-42.8),(-42.2),(0), (42.2), (42.8)
 
 query I
-SELECT cast(CEIL(n::tinyint) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEIL(n::tinyint) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -19,7 +19,7 @@ SELECT cast(CEIL(n::tinyint) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(CEIL(n::smallint) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEIL(n::smallint) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -29,7 +29,7 @@ SELECT cast(CEIL(n::smallint) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(CEIL(n::integer) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEIL(n::integer) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -39,7 +39,7 @@ SELECT cast(CEIL(n::integer) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(CEIL(n::bigint) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEIL(n::bigint) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -49,7 +49,7 @@ SELECT cast(CEIL(n::bigint) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(CEIL(n::float) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEIL(n::float) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -59,7 +59,7 @@ SELECT cast(CEIL(n::float) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(CEIL(n::double) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEIL(n::double) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -69,7 +69,7 @@ SELECT cast(CEIL(n::double) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(CEILING(n::double) as bigint) FROM numbers ORDER BY n
+SELECT cast(CEILING(n::double) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -79,7 +79,7 @@ SELECT cast(CEILING(n::double) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(FLOOR(n::tinyint) as bigint) FROM numbers ORDER BY n
+SELECT cast(FLOOR(n::tinyint) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -89,7 +89,7 @@ SELECT cast(FLOOR(n::tinyint) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(FLOOR(n::smallint) as bigint) FROM numbers ORDER BY n
+SELECT cast(FLOOR(n::smallint) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -99,7 +99,7 @@ SELECT cast(FLOOR(n::smallint) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(FLOOR(n::integer) as bigint) FROM numbers ORDER BY n
+SELECT cast(FLOOR(n::integer) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -109,7 +109,7 @@ SELECT cast(FLOOR(n::integer) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(FLOOR(n::bigint) as bigint) FROM numbers ORDER BY n
+SELECT cast(FLOOR(n::bigint) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -42
 -42
@@ -119,7 +119,7 @@ SELECT cast(FLOOR(n::bigint) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(FLOOR(n::float) as bigint) FROM numbers ORDER BY n
+SELECT cast(FLOOR(n::float) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -43
 -43
@@ -129,7 +129,7 @@ SELECT cast(FLOOR(n::float) as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(FLOOR(n::double) as bigint) FROM numbers ORDER BY n
+SELECT cast(FLOOR(n::double) as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -43
 -43
diff --git a/modules/calcite/src/test/sql/function/numeric/test_trigo.test b/modules/calcite/src/test/sql/function/numeric/test_trigo.test
index 1bb03ac..628b302 100644
--- a/modules/calcite/src/test/sql/function/numeric/test_trigo.test
+++ b/modules/calcite/src/test/sql/function/numeric/test_trigo.test
@@ -9,7 +9,7 @@ statement ok
 INSERT INTO numbers VALUES (-42),(-1),(0), (1), (42), (NULL)
 
 query I
-SELECT cast(SIN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(SIN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 916
 -841
@@ -19,7 +19,7 @@ SELECT cast(SIN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(SIN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(SIN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 916
 -841
@@ -29,7 +29,7 @@ SELECT cast(SIN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(SIN(n::integer)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(SIN(n::integer)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 916
 -841
@@ -39,7 +39,7 @@ SELECT cast(SIN(n::integer)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(SIN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(SIN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 916
 -841
@@ -49,7 +49,7 @@ SELECT cast(SIN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(SIN(n::float)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(SIN(n::float)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 916
 -841
@@ -59,7 +59,7 @@ SELECT cast(SIN(n::float)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(SIN(n::double)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(SIN(n::double)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 916
 -841
@@ -69,7 +69,7 @@ SELECT cast(SIN(n::double)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(COS(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(COS(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -399
 540
@@ -79,7 +79,7 @@ SELECT cast(COS(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(COS(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(COS(n::smallint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -399
 540
@@ -89,7 +89,7 @@ SELECT cast(COS(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(COS(n::integer)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(COS(n::integer)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -399
 540
@@ -99,7 +99,7 @@ SELECT cast(COS(n::integer)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(COS(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(COS(n::bigint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -399
 540
@@ -109,7 +109,7 @@ SELECT cast(COS(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(COS(n::float)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(COS(n::float)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -399
 540
@@ -119,7 +119,7 @@ SELECT cast(COS(n::float)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(COS(n::double)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(COS(n::double)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -399
 540
@@ -129,7 +129,7 @@ SELECT cast(COS(n::double)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(TAN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(TAN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -2291
 -1557
@@ -139,7 +139,7 @@ SELECT cast(TAN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(TAN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(TAN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -2291
 -1557
@@ -149,7 +149,7 @@ SELECT cast(TAN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(TAN(n::integer)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(TAN(n::integer)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -2291
 -1557
@@ -159,7 +159,7 @@ SELECT cast(TAN(n::integer)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(TAN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(TAN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -2291
 -1557
@@ -169,7 +169,7 @@ SELECT cast(TAN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(TAN(n::float)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(TAN(n::float)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -2291
 -1557
@@ -179,7 +179,7 @@ SELECT cast(TAN(n::float)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(TAN(n::double)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(TAN(n::double)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -2291
 -1557
@@ -189,7 +189,7 @@ SELECT cast(TAN(n::double)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -1546
 -785
@@ -199,7 +199,7 @@ SELECT cast(ATAN(n::tinyint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -1546
 -785
@@ -209,7 +209,7 @@ SELECT cast(ATAN(n::smallint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN(n::integer)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN(n::integer)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -1546
 -785
@@ -219,7 +219,7 @@ SELECT cast(ATAN(n::integer)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -1546
 -785
@@ -229,7 +229,7 @@ SELECT cast(ATAN(n::bigint)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN(n::float)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN(n::float)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -1546
 -785
@@ -239,7 +239,7 @@ SELECT cast(ATAN(n::float)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN(n::double)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN(n::double)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -1546
 -785
@@ -384,7 +384,7 @@ SELECT cast(COT(n::double)*1000 as bigint) FROM numbers  WHERE n > 0.1 OR N < -0
 436
 
 query I
-SELECT cast(ATAN2(n::tinyint, 42)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN2(n::tinyint, 42)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -785
 -23
@@ -394,7 +394,7 @@ SELECT cast(ATAN2(n::tinyint, 42)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN2(n::smallint, 42)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN2(n::smallint, 42)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -785
 -23
@@ -404,7 +404,7 @@ SELECT cast(ATAN2(n::smallint, 42)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN2(n::integer, 42)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN2(n::integer, 42)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -785
 -23
@@ -414,7 +414,7 @@ SELECT cast(ATAN2(n::integer, 42)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN2(n::bigint, 42)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN2(n::bigint, 42)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -785
 -23
@@ -424,7 +424,7 @@ SELECT cast(ATAN2(n::bigint, 42)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN2(n::float, 42)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN2(n::float, 42)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -785
 -23
@@ -434,7 +434,7 @@ SELECT cast(ATAN2(n::float, 42)*1000 as bigint) FROM numbers ORDER BY n
 NULL
 
 query I
-SELECT cast(ATAN2(n::double, 42)*1000 as bigint) FROM numbers ORDER BY n
+SELECT cast(ATAN2(n::double, 42)*1000 as bigint) FROM numbers ORDER BY n NULLS LAST
 ----
 -785
 -23
diff --git a/modules/calcite/src/test/sql/function/operator/test_arithmetic.test b/modules/calcite/src/test/sql/function/operator/test_arithmetic.test
index 837d75f..e80ec0b 100644
--- a/modules/calcite/src/test/sql/function/operator/test_arithmetic.test
+++ b/modules/calcite/src/test/sql/function/operator/test_arithmetic.test
@@ -14,7 +14,7 @@ INSERT INTO integers VALUES (1), (2), (3), (NULL)
 # addition is unordered (i.e. i+2=2+i)
 # i+2=5 => i=3
 query TT
-SELECT i+2=5, 5=i+2 FROM integers ORDER BY i
+SELECT i+2=5, 5=i+2 FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 false	false
@@ -23,7 +23,7 @@ NULL	NULL
 
 # 2+i=5 => i=3
 query TT
-SELECT 2+i=5, 5=2+i FROM integers ORDER BY i
+SELECT 2+i=5, 5=2+i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 false	false
@@ -33,7 +33,7 @@ NULL	NULL
 # multiplication is unordered
 # i*2=6 => i=3
 query TT
-SELECT i*2=6, 6=i*2 FROM integers ORDER BY i
+SELECT i*2=6, 6=i*2 FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 false	false
@@ -42,7 +42,7 @@ NULL	NULL
 
 # 2*i=6 => i=3
 query TT
-SELECT 2*i=6, 6=2*i FROM integers ORDER BY i
+SELECT 2*i=6, 6=2*i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 false	false
@@ -51,7 +51,7 @@ NULL	NULL
 
 # i*2=5 (this comparison is always FALSE, except if i is NULL in which case it is NULL)
 query T
-SELECT i*2=5 FROM integers ORDER BY i
+SELECT i*2=5 FROM integers ORDER BY i NULLS LAST
 ----
 false
 false
@@ -60,7 +60,7 @@ NULL
 
 # i*0=5
 query T
-SELECT i*0=5 FROM integers ORDER BY i
+SELECT i*0=5 FROM integers ORDER BY i NULLS LAST
 ----
 false
 false
@@ -69,7 +69,7 @@ NULL
 
 # -i>-2 => i<2
 query T
-SELECT -i>-2 FROM integers ORDER BY i
+SELECT -i>-2 FROM integers ORDER BY i NULLS LAST
 ----
 true
 false
@@ -79,7 +79,7 @@ NULL
 # subtraction is ordered
 # i-2=1 => i=3
 query TT
-SELECT i-2=1, 1=i-2 FROM integers ORDER BY i
+SELECT i-2=1, 1=i-2 FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 false	false
@@ -88,7 +88,7 @@ NULL	NULL
 
 # 3-i=1 => i=2
 query TT
-SELECT 3-i=1, 1=3-i FROM integers ORDER BY i
+SELECT 3-i=1, 1=3-i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 true	true
@@ -98,7 +98,7 @@ NULL	NULL
 # non-equality comparisons should also be flipped in this case
 # 3-i<2 => i>2
 query TT
-SELECT 3-i<2, 2>3-i FROM integers ORDER BY i
+SELECT 3-i<2, 2>3-i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 true	true
@@ -106,7 +106,7 @@ true	true
 NULL	NULL
 
 query TT
-SELECT 3-i<=1, 1>=3-i FROM integers ORDER BY i
+SELECT 3-i<=1, 1>=3-i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 true	true
@@ -116,7 +116,7 @@ NULL	NULL
 # division is ordered
 # i/2=1 => i>=2 or i<=3
 query TT
-SELECT i/2=1, 1=i/2 FROM integers ORDER BY i
+SELECT i/2=1, 1=i/2 FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 true	true
@@ -125,7 +125,7 @@ NULL	NULL
 
 # 2/i=1 => i=2
 query TT
-SELECT 2/i=1, 1=2/i FROM integers ORDER BY i
+SELECT 2/i=1, 1=2/i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 true	true
@@ -134,7 +134,7 @@ NULL	NULL
 
 # 3/i=2 => i=2
 query TT
-SELECT 2/i=1, 1=2/i FROM integers ORDER BY i
+SELECT 2/i=1, 1=2/i FROM integers ORDER BY i NULLS LAST
 ----
 false	false
 true	true
diff --git a/modules/calcite/src/test/sql/function/string/regex_search.test b/modules/calcite/src/test/sql/function/string/regex_search.test
index 3796867..c031b92 100644
--- a/modules/calcite/src/test/sql/function/string/regex_search.test
+++ b/modules/calcite/src/test/sql/function/string/regex_search.test
@@ -108,19 +108,6 @@ SELECT 'asdf' ~ '.*SD.*'
 ----
 false
 
-# newlines
-query T
-SELECT 'hello
-world' ~ '.*'
-----
-true
-
-query T
-SELECT 'hello
-world' ~ '^.*$'
-----
-true
-
 # this also works with tables
 statement ok
 CREATE TABLE test(v VARCHAR);
diff --git a/modules/calcite/src/test/sql/function/string/regex_search.test b/modules/calcite/src/test/sql/function/string/regex_search.test_ignore
similarity index 96%
copy from modules/calcite/src/test/sql/function/string/regex_search.test
copy to modules/calcite/src/test/sql/function/string/regex_search.test_ignore
index 3796867..2ba8a85 100644
--- a/modules/calcite/src/test/sql/function/string/regex_search.test
+++ b/modules/calcite/src/test/sql/function/string/regex_search.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/function/string/regex_search.test
 # description: regex search test
 # group: [string]
+# Ignored https://issues.apache.org/jira/browse/IGNITE-15562
 
 # constant strings
 query T
diff --git a/modules/calcite/src/test/sql/function/string/test_caseconvert.test b/modules/calcite/src/test/sql/function/string/test_caseconvert.test
index e24433e..21840e3 100644
--- a/modules/calcite/src/test/sql/function/string/test_caseconvert.test
+++ b/modules/calcite/src/test/sql/function/string/test_caseconvert.test
@@ -54,7 +54,7 @@ hulld
 motörhead
 
 query T
-select LOWER(b) FROM strings ORDER BY b
+select LOWER(b) FROM strings ORDER BY b NULLS LAST
 ----
 räcks
 world
diff --git a/modules/calcite/src/test/sql/function/string/test_concat.test b/modules/calcite/src/test/sql/function/string/test_concat.test
index 9755f46..c8d054b 100644
--- a/modules/calcite/src/test/sql/function/string/test_concat.test
+++ b/modules/calcite/src/test/sql/function/string/test_concat.test
@@ -17,7 +17,7 @@ INSERT INTO strings VALUES ('hello'), ('world'), (NULL)
 
 # normal concat
 query T
-SELECT s || ' ' || s FROM strings ORDER BY s
+SELECT s || ' ' || s FROM strings ORDER BY s NULLS LAST
 ----
 hello hello
 world world
@@ -25,14 +25,14 @@ NULL
 
 # unicode concat
 query T
-SELECT s || ' ' || '🦆' FROM strings ORDER BY s
+SELECT s || ' ' || '🦆' FROM strings ORDER BY s NULLS LAST
 ----
 hello 🦆
 world 🦆
 NULL
 
 query T
-SELECT s || ' ' || '🦆' FROM strings ORDER BY s
+SELECT s || ' ' || '🦆' FROM strings ORDER BY s NULLS LAST
 ----
 hello 🦆
 world 🦆
@@ -68,7 +68,7 @@ NULL
 
 # this also applies to non-constant null values
 query T
-SELECT CONCAT('hello', ' ', s) FROM strings ORDER BY s
+SELECT CONCAT('hello', ' ', s) FROM strings ORDER BY s NULLS LAST
 ----
 hello hello
 hello world
diff --git a/modules/calcite/src/test/sql/function/string/test_length.test b/modules/calcite/src/test/sql/function/string/test_length.test
index e04f34a..d52501b 100644
--- a/modules/calcite/src/test/sql/function/string/test_length.test
+++ b/modules/calcite/src/test/sql/function/string/test_length.test
@@ -10,7 +10,7 @@ INSERT INTO strings VALUES ('hello'), ('world'), (NULL)
 
 # normal length
 query I
-SELECT length(s) FROM strings ORDER BY s
+SELECT length(s) FROM strings ORDER BY s NULLS LAST
 ----
 5
 5
@@ -18,7 +18,7 @@ NULL
 
 # length after concat
 query I
-SELECT length(s || ' ' || 'a') FROM strings ORDER BY s
+SELECT length(s || ' ' || 'a') FROM strings ORDER BY s NULLS LAST
 ----
 7
 7
diff --git a/modules/calcite/src/test/sql/insert/big_insert_rollback.test_slow b/modules/calcite/src/test/sql/insert/big_insert_rollback.test_slow
deleted file mode 100644
index ac0e38d..0000000
--- a/modules/calcite/src/test/sql/insert/big_insert_rollback.test_slow
+++ /dev/null
@@ -1,25 +0,0 @@
-# name: test/sql/insert/big_insert_rollback.test_slow
-# description: Rollback big insert statement
-# group: [insert]
-
-statement ok
-CREATE TABLE integers(i INTEGER)
-
-statement ok
-BEGIN TRANSACTION
-
-statement ok
-INSERT INTO integers SELECT * FROM range(0, 500000)
-
-query I
-SELECT COUNT(*) FROM integers
-----
-500000
-
-statement ok
-ROLLBACK
-
-query I
-SELECT COUNT(*) FROM integers
-----
-0
diff --git a/modules/calcite/src/test/sql/insert/insert_rollback.test b/modules/calcite/src/test/sql/insert/insert_rollback.test
deleted file mode 100644
index 93a7422..0000000
--- a/modules/calcite/src/test/sql/insert/insert_rollback.test
+++ /dev/null
@@ -1,25 +0,0 @@
-# name: test/sql/insert/insert_rollback.test
-# description: Rollback insert statement
-# group: [insert]
-
-statement ok
-CREATE TABLE integers(i INTEGER)
-
-statement ok
-BEGIN TRANSACTION
-
-statement ok
-INSERT INTO integers VALUES (0), (1), (2)
-
-query I
-SELECT COUNT(*) FROM integers
-----
-3
-
-statement ok
-ROLLBACK
-
-query I
-SELECT COUNT(*) FROM integers
-----
-0
diff --git a/modules/calcite/src/test/sql/insert/interleaved_insert_rollback.test_slow b/modules/calcite/src/test/sql/insert/interleaved_insert_rollback.test_slow
deleted file mode 100644
index 6164e30..0000000
--- a/modules/calcite/src/test/sql/insert/interleaved_insert_rollback.test_slow
+++ /dev/null
@@ -1,49 +0,0 @@
-# name: test/sql/insert/interleaved_insert_rollback.test_slow
-# description: Rollback interleaved big insert statements
-# group: [insert]
-
-statement ok
-CREATE TABLE integers(i INTEGER)
-
-statement ok con1
-BEGIN TRANSACTION
-
-statement ok con2
-BEGIN TRANSACTION
-
-statement ok con1
-INSERT INTO integers SELECT * FROM range(0, 500000)
-
-statement ok con2
-INSERT INTO integers SELECT * FROM range(0, 500000)
-
-query I con1
-SELECT COUNT(*) FROM integers
-----
-500000
-
-query I con2
-SELECT COUNT(*) FROM integers
-----
-500000
-
-statement ok con1
-ROLLBACK
-
-query I con1
-SELECT COUNT(*) FROM integers
-----
-0
-
-query I con2
-SELECT COUNT(*) FROM integers
-----
-500000
-
-statement ok con2
-ROLLBACK
-
-query I
-SELECT COUNT(*) FROM integers
-----
-0
\ No newline at end of file
diff --git a/modules/calcite/src/test/sql/insert/null_values.test b/modules/calcite/src/test/sql/insert/null_values.test_ignore
similarity index 85%
rename from modules/calcite/src/test/sql/insert/null_values.test
rename to modules/calcite/src/test/sql/insert/null_values.test_ignore
index d7a5629..bd514dc 100644
--- a/modules/calcite/src/test/sql/insert/null_values.test
+++ b/modules/calcite/src/test/sql/insert/null_values.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/insert/null_values.test
 # description: Test inserting many null values
 # group: [insert]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 CREATE TABLE integers(i INTEGER)
@@ -37,4 +39,4 @@ SELECT COUNT(i), SUM(i), MIN(i), MAX(i), COUNT(*) FROM integers
 query I
 SELECT SUM(CASE WHEN i IS NULL THEN 1 ELSE 0 END) FROM integers
 ----
-500
\ No newline at end of file
+500
diff --git a/modules/calcite/src/test/sql/insert/test_big_insert.test b/modules/calcite/src/test/sql/insert/test_big_insert.test
index b5c7265..4b90b97 100644
--- a/modules/calcite/src/test/sql/insert/test_big_insert.test
+++ b/modules/calcite/src/test/sql/insert/test_big_insert.test
@@ -44,7 +44,7 @@ INSERT INTO integers VALUES (DEFAULT+1, 4);
 statement ok
 INSERT INTO integers (i) SELECT j FROM integers;
 
-query II
+query II rowsort
 SELECT * FROM integers
 ----
 3	4
@@ -53,4 +53,3 @@ NULL	4
 4	NULL
 3	NULL
 4	NULL
-
diff --git a/modules/calcite/src/test/sql/insert/test_insert.test b/modules/calcite/src/test/sql/insert/test_insert.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/insert/test_insert.test
rename to modules/calcite/src/test/sql/insert/test_insert.test_ignore
index 794baec..19dabf6 100644
--- a/modules/calcite/src/test/sql/insert/test_insert.test
+++ b/modules/calcite/src/test/sql/insert/test_insert.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/insert/test_insert.test
 # description: Test insert into and updates of constant values
 # group: [insert]
+# Ignored https://issues.apache.org/jira/browse/IGNITE-14555
 
 statement ok
 CREATE TABLE integers(i INTEGER)
diff --git a/modules/calcite/src/test/sql/insert/test_insert_invalid.test b/modules/calcite/src/test/sql/insert/test_insert_invalid.test
index 5f210e76..52ce497 100644
--- a/modules/calcite/src/test/sql/insert/test_insert_invalid.test
+++ b/modules/calcite/src/test/sql/insert/test_insert_invalid.test
@@ -2,15 +2,6 @@
 # description: Test invalid INSERT INTO statements
 # group: [insert]
 
-statement ok
-CREATE TABLE strings(i STRING)
-
-statement error
-INSERT INTO strings VALUES ('�(')
-
-statement error
-SELECT * FROM strings WHERE i = '�('
-
 # Test insert with too few or too many cols
 statement ok
 CREATE TABLE a(i integer, j integer)
diff --git a/modules/calcite/src/test/sql/insert/test_insert_query.test b/modules/calcite/src/test/sql/insert/test_insert_query.test
index 8a40a74..71231da 100644
--- a/modules/calcite/src/test/sql/insert/test_insert_query.test
+++ b/modules/calcite/src/test/sql/insert/test_insert_query.test
@@ -11,7 +11,7 @@ INSERT INTO integers SELECT 42
 statement ok
 INSERT INTO integers SELECT CAST(NULL AS VARCHAR)
 
-query I
+query I rowsort
 SELECT * FROM integers
 ----
 42
diff --git a/modules/calcite/src/test/sql/insert/test_insert_type.test b/modules/calcite/src/test/sql/insert/test_insert_type.test_ignore
similarity index 89%
rename from modules/calcite/src/test/sql/insert/test_insert_type.test
rename to modules/calcite/src/test/sql/insert/test_insert_type.test_ignore
index 4b514cf..4f62804 100644
--- a/modules/calcite/src/test/sql/insert/test_insert_type.test
+++ b/modules/calcite/src/test/sql/insert/test_insert_type.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/insert/test_insert_type.test
 # description: Test insert into from wrong type
 # group: [insert]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15565
 
 statement ok
 CREATE TABLE strings(a VARCHAR)
diff --git a/modules/calcite/src/test/sql/insert/unaligned_interleaved_appends.test b/modules/calcite/src/test/sql/insert/unaligned_interleaved_appends.test
deleted file mode 100644
index c705fec..0000000
--- a/modules/calcite/src/test/sql/insert/unaligned_interleaved_appends.test
+++ /dev/null
@@ -1,81 +0,0 @@
-# name: test/sql/insert/unaligned_interleaved_appends.test
-# description: Test unaligned interleaved appends
-# group: [insert]
-
-statement ok
-CREATE TABLE integers(i INTEGER)
-
-statement ok con1
-BEGIN TRANSACTION
-
-# con2 insert 5 rows and begins a transaction
-statement ok con2
-INSERT INTO integers SELECT * FROM range(0, 5)
-
-statement ok con2
-BEGIN TRANSACTION
-
-# con3 inserts 17 rows and begins a transaction
-statement ok con3
-INSERT INTO integers SELECT * FROM range(0, 17)
-
-statement ok con3
-BEGIN TRANSACTION
-
-# con4 inserts 1007 rows and begins a transaction
-statement ok con4
-INSERT INTO integers SELECT * FROM range(0, 1007)
-
-statement ok con4
-BEGIN TRANSACTION
-
-# con5 inserts 3020 rows and begins a transaction
-statement ok con5
-INSERT INTO integers SELECT * FROM range(0, 3020)
-
-statement ok con5
-BEGIN TRANSACTION
-
-# con6 inserts 3 rows and begins a transaction
-statement ok con6
-INSERT INTO integers SELECT * FROM range(0, 3)
-
-statement ok con6
-BEGIN TRANSACTION
-
-# now we verify all the counts
-# con1: 0
-query I con1
-SELECT COUNT(*) FROM integers
-----
-0
-
-# con2: 5
-query I con2
-SELECT COUNT(*) FROM integers
-----
-5
-
-# con3: 22
-query I con3
-SELECT COUNT(*) FROM integers
-----
-22
-
-# con4: 1029
-query I con4
-SELECT COUNT(*) FROM integers
-----
-1029
-
-# con5: 4049
-query I con5
-SELECT COUNT(*) FROM integers
-----
-4049
-
-# con6: 4052
-query I con6
-SELECT COUNT(*) FROM integers
-----
-4052
diff --git a/modules/calcite/src/test/sql/join/full_outer/test_full_outer_join_many_matches.test b/modules/calcite/src/test/sql/join/full_outer/test_full_outer_join_many_matches.test_ignore
similarity index 92%
rename from modules/calcite/src/test/sql/join/full_outer/test_full_outer_join_many_matches.test
rename to modules/calcite/src/test/sql/join/full_outer/test_full_outer_join_many_matches.test_ignore
index 0f70043..5fc60ae 100644
--- a/modules/calcite/src/test/sql/join/full_outer/test_full_outer_join_many_matches.test
+++ b/modules/calcite/src/test/sql/join/full_outer/test_full_outer_join_many_matches.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/full_outer/test_full_outer_join_many_matches.test
 # description: Test FULL OUTER JOIN with many matches
 # group: [full_outer]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test b/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test
index 770ee8c..5587f51 100644
--- a/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test
+++ b/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test
@@ -71,22 +71,12 @@ SELECT * FROM test WHERE EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND tes
 13	3	c
 
 query IIT
-SELECT * FROM test WHERE EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<>test2.c) AND NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<test2.c);
-----
-12	2	b
-
-query IIT
 SELECT * FROM test WHERE NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<test2.c) ORDER BY a;
 ----
 11	1	a
 12	2	b
 
 query IIT
-SELECT * FROM test WHERE NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<test2.c) AND NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b>test2.c);
-----
-11	1	a
-
-query IIT
 SELECT * FROM test WHERE EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<>test2.c) AND test.a > 11 ORDER BY a;
 ----
 12	2	b
@@ -119,12 +109,6 @@ SELECT * FROM test WHERE NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND
 13	3	c
 
 query IIT
-SELECT * FROM test WHERE NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<test2.c AND test2.a=12) AND NOT EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b>test2.c AND test2.a=12) ORDER BY a;
-----
-11	1	a
-13	3	c
-
-query IIT
 SELECT * FROM test WHERE EXISTS(SELECT * FROM test2 WHERE test.a=test2.a AND test.b<>test2.c) AND test.a < 13 ORDER BY a;
 ----
 12	2	b
diff --git a/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test b/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test_ignore
similarity index 98%
copy from modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test
copy to modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test_ignore
index 770ee8c..bef7765 100644
--- a/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test
+++ b/modules/calcite/src/test/sql/join/inner/test_eq_ineq_join.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/inner/test_eq_ineq_join.test
 # description: Equality + inequality joins
 # group: [inner]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15566
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/inner/test_join_duplicates.test b/modules/calcite/src/test/sql/join/inner/test_join_duplicates.test_ignore
similarity index 89%
rename from modules/calcite/src/test/sql/join/inner/test_join_duplicates.test
rename to modules/calcite/src/test/sql/join/inner/test_join_duplicates.test_ignore
index 2fcc9ac..050089b 100644
--- a/modules/calcite/src/test/sql/join/inner/test_join_duplicates.test
+++ b/modules/calcite/src/test/sql/join/inner/test_join_duplicates.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/inner/test_join_duplicates.test
 # description: Test join with > STANDARD_VECTOR_SIZE duplicates
 # group: [inner]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15567
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/inner/test_join_types.test b/modules/calcite/src/test/sql/join/inner/test_join_types.test_ignore
similarity index 99%
rename from modules/calcite/src/test/sql/join/inner/test_join_types.test
rename to modules/calcite/src/test/sql/join/inner/test_join_types.test_ignore
index aeab988..c9a6256 100644
--- a/modules/calcite/src/test/sql/join/inner/test_join_types.test
+++ b/modules/calcite/src/test/sql/join/inner/test_join_types.test_ignore
@@ -1,14 +1,13 @@
 # name: test/sql/join/inner/test_join_types.test
 # description: Test joins with different types
 # group: [inner]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
 
 statement ok
 PRAGMA enable_verification
 
 # numeric types
 # tests for tinyint
-statement ok
-begin transaction
 
 statement ok
 create table a as select i::tinyint AS i from range(1, 101, 1) t1(i)
@@ -62,9 +61,6 @@ select count(*) from a, (SELECT 1::tinyint AS j) b where i = j AND i>1000
 ----
 0
 
-statement ok
-rollback
-
 # tests for smallint
 statement ok
 begin transaction
diff --git a/modules/calcite/src/test/sql/join/inner/test_lt_join.test b/modules/calcite/src/test/sql/join/inner/test_lt_join.test_ignore
similarity index 89%
rename from modules/calcite/src/test/sql/join/inner/test_lt_join.test
rename to modules/calcite/src/test/sql/join/inner/test_lt_join.test_ignore
index b3e6838..8364b0d 100644
--- a/modules/calcite/src/test/sql/join/inner/test_lt_join.test
+++ b/modules/calcite/src/test/sql/join/inner/test_lt_join.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/inner/test_lt_join.test
 # description: Test less than join
 # group: [inner]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/inner/test_unequal_join_duplicates.test b/modules/calcite/src/test/sql/join/inner/test_unequal_join_duplicates.test_ignore
similarity index 79%
rename from modules/calcite/src/test/sql/join/inner/test_unequal_join_duplicates.test
rename to modules/calcite/src/test/sql/join/inner/test_unequal_join_duplicates.test_ignore
index 4ca3f89..28c8f8f 100644
--- a/modules/calcite/src/test/sql/join/inner/test_unequal_join_duplicates.test
+++ b/modules/calcite/src/test/sql/join/inner/test_unequal_join_duplicates.test_ignore
@@ -1,9 +1,8 @@
 # name: test/sql/join/inner/test_unequal_join_duplicates.test
 # description: Test inequality join with > STANDARD_VECTOR_SIZE duplicates
 # group: [inner]
-
-statement ok
-PRAGMA enable_verification
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15567
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
 
 statement ok
 CREATE TABLE test (b INTEGER);
diff --git a/modules/calcite/src/test/sql/join/inner/test_using_chain.test b/modules/calcite/src/test/sql/join/inner/test_using_chain.test
index 2c6b0dc..ee000ca 100644
--- a/modules/calcite/src/test/sql/join/inner/test_using_chain.test
+++ b/modules/calcite/src/test/sql/join/inner/test_using_chain.test
@@ -23,13 +23,6 @@ CREATE TABLE t3 (c INTEGER, d INTEGER)
 statement ok
 INSERT INTO t3 VALUES (3, 4)
 
-# multiple joins with using
-# single column
-query IIII
-SELECT * FROM t1 JOIN t2 USING (b) JOIN t3 USING (c) ORDER BY 1, 2, 3, 4;
-----
-1	2	3	4
-
 # column does not exist on left side of join
 statement error
 SELECT * FROM t1 JOIN t2 USING (c)
@@ -37,37 +30,3 @@ SELECT * FROM t1 JOIN t2 USING (c)
 # column does not exist on right side of join
 statement error
 SELECT * FROM t1 JOIN t2 USING (a)
-
-statement ok
-DROP TABLE t1
-
-statement ok
-DROP TABLE t2
-
-statement ok
-DROP TABLE t3
-
-statement ok
-CREATE TABLE t1 (a INTEGER, b INTEGER, c INTEGER)
-
-statement ok
-INSERT INTO t1 VALUES (1, 2, 2)
-
-statement ok
-CREATE TABLE t2 (b INTEGER, c INTEGER, d INTEGER, e INTEGER)
-
-statement ok
-INSERT INTO t2 VALUES (2, 2, 3, 4)
-
-statement ok
-CREATE TABLE t3 (d INTEGER, e INTEGER)
-
-statement ok
-INSERT INTO t3 VALUES (3, 4)
-
-# multi column
-query IIIII
-SELECT * FROM t1 JOIN t2 USING (b, c) JOIN t3 USING (d, e);
-----
-1	2	2	3	4
-
diff --git a/modules/calcite/src/test/sql/join/inner/test_using_chain.test b/modules/calcite/src/test/sql/join/inner/test_using_chain.test_ignore
similarity index 95%
copy from modules/calcite/src/test/sql/join/inner/test_using_chain.test
copy to modules/calcite/src/test/sql/join/inner/test_using_chain.test_ignore
index 2c6b0dc..763a6f0 100644
--- a/modules/calcite/src/test/sql/join/inner/test_using_chain.test
+++ b/modules/calcite/src/test/sql/join/inner/test_using_chain.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/inner/test_using_chain.test
 # description: Test chaining USING joins
 # group: [inner]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15569
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/inner/test_using_join.test b/modules/calcite/src/test/sql/join/inner/test_using_join.test
index 6d46131..1fa212e 100644
--- a/modules/calcite/src/test/sql/join/inner/test_using_join.test
+++ b/modules/calcite/src/test/sql/join/inner/test_using_join.test
@@ -41,13 +41,6 @@ SELECT t2.a, t2.b, t2.c FROM t1 JOIN t2 USING(a,b,c)
 ----
 1	2	3
 
-# USING columns can be used without requiring a table specifier
-query I
-SELECT a+1 FROM t1 JOIN t2 USING(a) ORDER BY a
-----
-2
-2
-
 statement error
 SELECT t2.a, t2.b, t2.c FROM t1 JOIN t2 USING(a+b)
 
@@ -60,11 +53,6 @@ SELECT t2.a, t2.b, t2.c FROM t1 JOIN t2 USING(d)
 statement error
 SELECT t2.a, t2.b, t2.c FROM t1 JOIN t2 USING(t1.a)
 
-query IIII
-SELECT * FROM t1 JOIN t2 USING(a,b)
-----
-1	2	3	3
-
 # CONTROVERSIAL:
 # we do not allow this because it is ambiguous: "b" can be bind to both "t1.b" or "t2.b" and this would give
 # different results SQLite allows this, PostgreSQL does not
@@ -74,12 +62,3 @@ SELECT * FROM t1 JOIN t2 USING(a) JOIN t2 t2b USING (b);
 # this is the same, but now with a duplicate potential binding on the RHS
 statement error
 select * from (values (1)) tbl(i) join ((values (1)) tbl2(i) join  (values (1)) tbl3(i) on tbl2.i=tbl3.i) using (i)
-
-# a chain with the same column name is allowed though!
-query IIIIIII
-SELECT * FROM t1 JOIN t2 USING(a) JOIN t2 t2b USING (a) ORDER BY 1, 2, 3, 4, 5, 6, 7
-----
-1	2	3	2	3	2	3
-1	2	3	2	3	3	4
-1	2	3	3	4	2	3
-1	2	3	3	4	3	4
diff --git a/modules/calcite/src/test/sql/join/inner/test_using_join.test b/modules/calcite/src/test/sql/join/inner/test_using_join.test_ignore
similarity index 91%
copy from modules/calcite/src/test/sql/join/inner/test_using_join.test
copy to modules/calcite/src/test/sql/join/inner/test_using_join.test_ignore
index 6d46131..653e6f7 100644
--- a/modules/calcite/src/test/sql/join/inner/test_using_join.test
+++ b/modules/calcite/src/test/sql/join/inner/test_using_join.test_ignore
@@ -1,6 +1,9 @@
 # name: test/sql/join/inner/test_using_join.test
 # description: Test USING joins
 # group: [inner]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15570
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15569
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15571
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/left_outer/left_join_issue_1172.test b/modules/calcite/src/test/sql/join/left_outer/left_join_issue_1172.test
index 0044c6c..2e282f9 100644
--- a/modules/calcite/src/test/sql/join/left_outer/left_join_issue_1172.test
+++ b/modules/calcite/src/test/sql/join/left_outer/left_join_issue_1172.test
@@ -12,14 +12,10 @@ statement ok
 drop table if exists t2;
 
 statement ok
-create table t1 (id string);
+create table t1 (id varchar);
 
 statement ok
-create table t2 (id string);
-
-statement ok
-insert into t1 values
-(NULL);
+create table t2 (id varchar);
 
 statement ok
 insert into t2 values (1), (1);
@@ -27,17 +23,14 @@ insert into t2 values (1), (1);
 query II
 select * from t1 left join t2 on t1.id = t2.id;
 ----
-NULL	NULL
 
 query II
 select * from t1 left join t2 on t1.id > t2.id;
 ----
-NULL	NULL
 
 query II
 select * from t1 left join t2 on t1.id <> t2.id;
 ----
-NULL	NULL
 
 statement ok
 insert into t2 values (1);
@@ -45,17 +38,14 @@ insert into t2 values (1);
 query II
 select * from t1 left join t2 on t1.id = t2.id;
 ----
-NULL	NULL
 
 query II
 select * from t1 left join t2 on t1.id > t2.id;
 ----
-NULL	NULL
 
 query II
 select * from t1 left join t2 on t1.id <> t2.id;
 ----
-NULL	NULL
 
 statement ok
 insert into t2 values (NULL), (NULL);
@@ -63,17 +53,14 @@ insert into t2 values (NULL), (NULL);
 query II
 select * from t1 left join t2 on t1.id = t2.id;
 ----
-NULL	NULL
 
 query II
 select * from t1 left join t2 on t1.id > t2.id;
 ----
-NULL	NULL
 
 query II
 select * from t1 left join t2 on t1.id <> t2.id;
 ----
-NULL	NULL
 
 statement ok
 drop table if exists t1;
@@ -82,19 +69,19 @@ statement ok
 drop table if exists t2;
 
 statement ok
-create table t1 (id string);
+create table t1 (id varchar);
 
 statement ok
-create table t2 (id string);
+create table t2 (id varchar);
 
 statement ok
-insert into t1 (id) values (1), (1), (NULL);
+insert into t1 (id) values (1), (1), (DEFAULT);
 
 statement ok
 insert into t2 (id) values (1), (1), (1), (1), (1), (1);
 
 query II
-select * from t1 left join t2 on t1.id = t2.id order by 1, 2;
+select * from t1 left join t2 on t1.id = t2.id order by 1 NULLS FIRST, 2;
 ----
 NULL	NULL
 1	1
@@ -114,7 +101,7 @@ statement ok
 insert into t2 (id) values (1);
 
 query II
-select * from t1 left join t2 on t1.id = t2.id order by 1, 2;
+select * from t1 left join t2 on t1.id = t2.id order by 1 NULLS FIRST, 2;
 ----
 NULL	NULL
 1	1
diff --git a/modules/calcite/src/test/sql/join/mark/test_mark_join_types.test b/modules/calcite/src/test/sql/join/mark/test_mark_join_types.test_ignore
similarity index 99%
rename from modules/calcite/src/test/sql/join/mark/test_mark_join_types.test
rename to modules/calcite/src/test/sql/join/mark/test_mark_join_types.test_ignore
index d73a46c..c7984bf 100644
--- a/modules/calcite/src/test/sql/join/mark/test_mark_join_types.test
+++ b/modules/calcite/src/test/sql/join/mark/test_mark_join_types.test_ignore
@@ -1,14 +1,13 @@
 # name: test/sql/join/mark/test_mark_join_types.test
 # description: Test mark join with different types
 # group: [mark]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
 
 statement ok
 PRAGMA enable_verification
 
 # numeric types
 # tests for tinyint
-statement ok
-begin transaction
 
 statement ok
 create table a as select i::tinyint AS i from range(1, 101, 1) t1(i) order by random()
@@ -108,9 +107,6 @@ select * from (select * from a where i > 90) a WHERE i <> ANY((SELECT 96::tinyin
 99
 100
 
-statement ok
-rollback
-
 # tests for smallint
 statement ok
 begin transaction
diff --git a/modules/calcite/src/test/sql/join/natural/natural_join.test b/modules/calcite/src/test/sql/join/natural/natural_join.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/join/natural/natural_join.test
rename to modules/calcite/src/test/sql/join/natural/natural_join.test_ignore
index 3c2f29e..e80f869 100644
--- a/modules/calcite/src/test/sql/join/natural/natural_join.test
+++ b/modules/calcite/src/test/sql/join/natural/natural_join.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/natural/natural_join.test
 # description: Test natural joins
 # group: [natural]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15573
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/test_cross_product_parallelism.test_slow b/modules/calcite/src/test/sql/join/test_cross_product_parallelism.test_ignore
similarity index 90%
rename from modules/calcite/src/test/sql/join/test_cross_product_parallelism.test_slow
rename to modules/calcite/src/test/sql/join/test_cross_product_parallelism.test_ignore
index 35c26da..36f206e 100644
--- a/modules/calcite/src/test/sql/join/test_cross_product_parallelism.test_slow
+++ b/modules/calcite/src/test/sql/join/test_cross_product_parallelism.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/join/test_cross_product_parallelism.test_slow
 # description: Test cross product with parallelism
 # group: [join]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/join/test_not_distinct_from.test b/modules/calcite/src/test/sql/join/test_not_distinct_from.test
index ad29b2d..ce1f537 100644
--- a/modules/calcite/src/test/sql/join/test_not_distinct_from.test
+++ b/modules/calcite/src/test/sql/join/test_not_distinct_from.test
@@ -15,9 +15,8 @@ create table tbl_2 (b integer)
 statement ok
 insert into tbl_2 values (1),(2),(NULL)
 
-
-query I
- SELECT a from tbl_1 WHERE a IS DISTINCT FROM b
+query I rowsort
+SELECT a from tbl_1 WHERE a IS DISTINCT FROM b
 ----
 1
 2
@@ -27,24 +26,6 @@ SELECT a from tbl_1 WHERE a IS NOT DISTINCT FROM b
 ----
 NULL
 
-
-query II
-explain select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b)
-----
-physical_plan	<REGEX>:.*HASH_JOIN.*
-
-query II
-select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b)
-----
-1	1
-2	2
-NULL	NULL
-
-query II
-explain select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS DISTINCT FROM tbl_2.b)
-----
-physical_plan	<REGEX>:.*NESTED_LOOP_JOIN.*
-
 query II
 select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS DISTINCT FROM tbl_2.b) order by a,tbl_2.b
 ----
@@ -55,73 +36,72 @@ NULL	2
 2	NULL
 2	1
 
-
 #Vector with vector
 query III
-select a.a, b.b, a.a IS NOT DISTINCT FROM b.b AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES (1), (2), (NULL)) AS b (b);
-----
-1	1	1
-1	2	0
-1	NULL	0
-2	1	0
-2	2	1
-2	NULL	0
-NULL	1	0
-NULL	2	0
-NULL	NULL	1
+select a.a, b.b, a.a IS NOT DISTINCT FROM b.b AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES(1), (2), (NULL)) AS b (b) ORDER BY a.a NULLS LAST;
+----
+1	1	true
+1	2	false
+1	NULL	false
+2	1	false
+2	2	true
+2	NULL	false
+NULL	1	false
+NULL	2	false
+NULL	NULL	true
 
 query III
-select a.a, b.b, a.a IS DISTINCT FROM b.b AS "Is Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES (1), (2), (NULL)) AS b (b);
-----
-1	1	0
-1	2	1
-1	NULL	1
-2	1	1
-2	2	0
-2	NULL	1
-NULL	1	1
-NULL	2	1
-NULL	NULL	0
+select a.a, b.b, a.a IS DISTINCT FROM b.b AS "Is Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES (1), (2), (NULL)) AS b (b) ORDER BY a.a NULLS LAST;
+----
+1	1	false
+1	2	true
+1	NULL	true
+2	1	true
+2	2	false
+2	NULL	true
+NULL	1	true
+NULL	2	true
+NULL	NULL	false
 
 #constant with constant
 query I
 select null is not distinct from null;
 ----
-True
+true
 
 query I
 select null is distinct from null;
 ----
-False
+false
 
 # Vector with constant
 query II
-select a.a, a.a IS NOT DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS NOT DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	True
-2	False
-NULL	False
+1	true
+2	false
+NULL	false
 
 query II
-select a.a, a.a IS DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	False
-2	True
-NULL	True
+1	false
+2	true
+NULL	true
 
 query II
-select a.a, a.a IS NOT DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS NOT DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	False
-2	False
-NULL	True
+1	false
+2	false
+NULL	true
 
 query II
-select a.a, a.a IS DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	True
-2	True
-NULL	False
+1	true
+2	true
+NULL	false
 
 # More types (all numeric types, strings, dates, times, timestamps, interval)
 statement ok
@@ -203,75 +183,9 @@ select count(*) FROM test_types where g IS NOT DISTINCT FROM NULL
 query T
 SELECT INTERVAL '30' DAY is distinct from NULL
 ----
-True
+true
 
 query T
 SELECT INTERVAL '30' DAY is not distinct from NULL
 ----
-False
-
-# Use on a bigger table (~10K elements)
-statement ok
-CREATE table big as select range a from range(10000);
-
-statement ok
-insert into big values (NULL)
-
-query I
-select count(*) from big  inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b)
-----
-3
-
-# Use with filter (e.g. SELECT x IS NOT DISTINCT FROM y FROM tbl WHERE z%2=0, where the expression filters like 50% of the rows)
-query I
-select count(*) from big  inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b) where a >1  and a < 5000
-----
-1
-
-
-# Use with rowid column (non-flat and non-constant vector)
-query I
-select count(*) from big inner join tbl_2 on (big.rowid IS NOT DISTINCT FROM tbl_2.b)
-----
-2
-
-# Use in subqueries with correlated expressions
-query II
-SELECT a, (select count(*) from tbl_1 where tbl_1.a is distinct from t.b) from tbl_1 as t
-----
-1	2
-2	3
-NULL	2
-
-# Use in DEFAULT expression of a table
-load __TEST_DIR__/not_distinct_test.db
-
-statement ok
-CREATE SEQUENCE seq;
-
-statement ok
-CREATE TABLE test_default (a BOOL DEFAULT nextval('seq') is not distinct from nextval('seq'), b INTEGER);
-
-statement ok
-INSERT INTO test_default (b) VALUES (2), (4), (6);
-
-
-query II
-select * from test_default;
-----
-False	2
-False	4
-False	6
-
-# Use in DEFAULT expression of a table and store that table on disk (storage test)
-
-restart
-
-
-query II
-select * from test_default;
-----
-False	2
-False	4
-False	6
-
+false
diff --git a/modules/calcite/src/test/sql/join/test_not_distinct_from.test b/modules/calcite/src/test/sql/join/test_not_distinct_from.test_ignore
similarity index 78%
copy from modules/calcite/src/test/sql/join/test_not_distinct_from.test
copy to modules/calcite/src/test/sql/join/test_not_distinct_from.test_ignore
index ad29b2d..d6dfa4c 100644
--- a/modules/calcite/src/test/sql/join/test_not_distinct_from.test
+++ b/modules/calcite/src/test/sql/join/test_not_distinct_from.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/join/test_not_distinct_from.test
 # description: Test join on is not distinct from query
 # group: [join]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15574
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15412
 
 
 statement ok
@@ -15,9 +17,8 @@ create table tbl_2 (b integer)
 statement ok
 insert into tbl_2 values (1),(2),(NULL)
 
-
-query I
- SELECT a from tbl_1 WHERE a IS DISTINCT FROM b
+query I rowsort
+SELECT a from tbl_1 WHERE a IS DISTINCT FROM b
 ----
 1
 2
@@ -27,12 +28,6 @@ SELECT a from tbl_1 WHERE a IS NOT DISTINCT FROM b
 ----
 NULL
 
-
-query II
-explain select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b)
-----
-physical_plan	<REGEX>:.*HASH_JOIN.*
-
 query II
 select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b)
 ----
@@ -41,11 +36,6 @@ select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS NOT DISTINCT FROM tbl_2.b)
 NULL	NULL
 
 query II
-explain select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS DISTINCT FROM tbl_2.b)
-----
-physical_plan	<REGEX>:.*NESTED_LOOP_JOIN.*
-
-query II
 select a,tbl_2.b from tbl_1 inner join tbl_2 on (a IS DISTINCT FROM tbl_2.b) order by a,tbl_2.b
 ----
 NULL	1
@@ -55,73 +45,72 @@ NULL	2
 2	NULL
 2	1
 
-
 #Vector with vector
 query III
-select a.a, b.b, a.a IS NOT DISTINCT FROM b.b AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES (1), (2), (NULL)) AS b (b);
-----
-1	1	1
-1	2	0
-1	NULL	0
-2	1	0
-2	2	1
-2	NULL	0
-NULL	1	0
-NULL	2	0
-NULL	NULL	1
+select a.a, b.b, a.a IS NOT DISTINCT FROM b.b AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES(1), (2), (NULL)) AS b (b) ORDER BY a.a NULLS LAST;
+----
+1	1	true
+1	2	false
+1	NULL	false
+2	1	false
+2	2	true
+2	NULL	false
+NULL	1	false
+NULL	2	false
+NULL	NULL	true
 
 query III
-select a.a, b.b, a.a IS DISTINCT FROM b.b AS "Is Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES (1), (2), (NULL)) AS b (b);
-----
-1	1	0
-1	2	1
-1	NULL	1
-2	1	1
-2	2	0
-2	NULL	1
-NULL	1	1
-NULL	2	1
-NULL	NULL	0
+select a.a, b.b, a.a IS DISTINCT FROM b.b AS "Is Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a), (VALUES (1), (2), (NULL)) AS b (b) ORDER BY a.a NULLS LAST;
+----
+1	1	false
+1	2	true
+1	NULL	true
+2	1	true
+2	2	false
+2	NULL	true
+NULL	1	true
+NULL	2	true
+NULL	NULL	false
 
 #constant with constant
 query I
 select null is not distinct from null;
 ----
-True
+true
 
 query I
 select null is distinct from null;
 ----
-False
+false
 
 # Vector with constant
 query II
-select a.a, a.a IS NOT DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS NOT DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	True
-2	False
-NULL	False
+1	true
+2	false
+NULL	false
 
 query II
-select a.a, a.a IS DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS DISTINCT FROM 1 AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	False
-2	True
-NULL	True
+1	false
+2	true
+NULL	true
 
 query II
-select a.a, a.a IS NOT DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS NOT DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	False
-2	False
-NULL	True
+1	false
+2	false
+NULL	true
 
 query II
-select a.a, a.a IS DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a);
+select a.a, a.a IS DISTINCT FROM NULL AS "Is Not Distinct From" FROM (VALUES (1), (2), (NULL)) AS a (a) ORDER BY a.a NULLS LAST;
 ----
-1	True
-2	True
-NULL	False
+1	true
+2	true
+NULL	false
 
 # More types (all numeric types, strings, dates, times, timestamps, interval)
 statement ok
@@ -203,12 +192,12 @@ select count(*) FROM test_types where g IS NOT DISTINCT FROM NULL
 query T
 SELECT INTERVAL '30' DAY is distinct from NULL
 ----
-True
+true
 
 query T
 SELECT INTERVAL '30' DAY is not distinct from NULL
 ----
-False
+false
 
 # Use on a bigger table (~10K elements)
 statement ok
@@ -237,15 +226,12 @@ select count(*) from big inner join tbl_2 on (big.rowid IS NOT DISTINCT FROM tbl
 
 # Use in subqueries with correlated expressions
 query II
-SELECT a, (select count(*) from tbl_1 where tbl_1.a is distinct from t.b) from tbl_1 as t
+SELECT a, (select count(*) from tbl_1 where tbl_1.a is distinct from t.b) from tbl_1 as t ORDER BY a NULLS LAST
 ----
 1	2
 2	3
 NULL	2
 
-# Use in DEFAULT expression of a table
-load __TEST_DIR__/not_distinct_test.db
-
 statement ok
 CREATE SEQUENCE seq;
 
@@ -255,23 +241,16 @@ CREATE TABLE test_default (a BOOL DEFAULT nextval('seq') is not distinct from ne
 statement ok
 INSERT INTO test_default (b) VALUES (2), (4), (6);
 
-
 query II
 select * from test_default;
 ----
-False	2
-False	4
-False	6
-
-# Use in DEFAULT expression of a table and store that table on disk (storage test)
-
-restart
-
+false	2
+false	4
+false	6
 
 query II
 select * from test_default;
 ----
-False	2
-False	4
-False	6
-
+false	2
+false	4
+false	6
diff --git a/modules/calcite/src/test/sql/order/test_limit.test b/modules/calcite/src/test/sql/order/test_limit.test
index e2b8116..2c2c3d3 100644
--- a/modules/calcite/src/test/sql/order/test_limit.test
+++ b/modules/calcite/src/test/sql/order/test_limit.test
@@ -13,19 +13,7 @@ INSERT INTO test VALUES (11, 22), (12, 21), (13, 22)
 
 # constant limit
 query I
-SELECT a FROM test LIMIT 1
-----
-11
-
-# decimal limit
-query I
-SELECT a FROM test LIMIT 1.5
-----
-11
-
-# LIMIT with operation
-query I
-SELECT a FROM test LIMIT 2-1
+SELECT a FROM test ORDER BY a LIMIT 1
 ----
 11
 
@@ -47,138 +35,11 @@ SELECT a FROM test LIMIT row_number() OVER ()
 
 # LIMIT Bug #321 Crazy Result
 statement ok
-CREATE TABLE test2 (a STRING);
+CREATE TABLE test2 (a VARCHAR);
 
 statement ok
 INSERT INTO test2 VALUES ('Hello World')
 
-statement ok
-PREPARE v1 AS SELECT * FROM test2 LIMIT 3
-
-query I
-EXECUTE v1
-----
-Hello World
-
 # can only limit by integers
 statement error
 select 1 limit date '1992-01-01';
-
-
-# Test limits with no constants
-statement ok
-CREATE TABLE integers(i INTEGER);
-
-statement ok
-INSERT INTO integers VALUES (1), (2), (3), (4), (5);
-
-statement ok
-CREATE SEQUENCE seq START 3;
-
-statement ok
-PRAGMA disable_verification;
-
-query I
-SELECT * FROM integers LIMIT nextval('seq');
-----
-1
-2
-3
-
-query I
-SELECT * FROM integers LIMIT nextval('seq');
-----
-1
-2
-3
-4
-
-
-# non-foldable expression in limit
-statement ok
-SELECT * FROM integers LIMIT RANDOM();
-
-# subquery in limit
-query I
-SELECT * FROM integers as int LIMIT (SELECT MIN(integers.i) FROM integers);
-----
-1
-
-statement ok
-CREATE SEQUENCE of_seq START 1;
-
-query I
-SELECT * FROM integers OFFSET nextval('of_seq');
-----
-2
-3
-4
-5
-
-query I
-SELECT * FROM integers OFFSET nextval('of_seq');
-----
-3
-4
-5
-
-# non-foldable expression in limit
-statement ok
-SELECT * FROM integers OFFSET RANDOM();
-
-# subquery in limit
-query I
-SELECT * FROM integers as int OFFSET (SELECT MIN(integers.i) FROM integers);
-----
-2
-3
-4
-5
-
-#Both LIMIT and OFFSET together with subqueries
-query I
-SELECT * FROM integers as int LIMIT (SELECT MAX(integers.i) FROM integers) OFFSET (SELECT MIN(integers.i) FROM integers);
-----
-2
-3
-4
-5
-
-#Subqueries that return zero rows = Same as ommiting limit clause
-query I
-SELECT * FROM integers as int LIMIT (SELECT max(integers.i) FROM integers where i > 5)
-----
-1
-2
-3
-4
-5
-
-#Limit of NULL, offset of NULL
-query I
-SELECT * FROM integers as int LIMIT (SELECT max(integers.i) FROM integers where i > 5)
-----
-1
-2
-3
-4
-5
-
-#Subqueries that return NULL
-query I
-SELECT * FROM integers as int LIMIT (SELECT NULL)
-----
-1
-2
-3
-4
-5
-
-
-#Subqueries that return negative values
-statement error
-SELECT * FROM integers as int LIMIT (SELECT -1);
-
-#Subqueries that return strings that can't be converted to integers
-statement error
-SELECT * FROM integers as int LIMIT (SELECT 'ab');
\ No newline at end of file
diff --git a/modules/calcite/src/test/sql/order/test_limit.test b/modules/calcite/src/test/sql/order/test_limit.test_ignore
similarity index 94%
copy from modules/calcite/src/test/sql/order/test_limit.test
copy to modules/calcite/src/test/sql/order/test_limit.test_ignore
index e2b8116..6440544 100644
--- a/modules/calcite/src/test/sql/order/test_limit.test
+++ b/modules/calcite/src/test/sql/order/test_limit.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/order/test_limit.test
 # description: Test LIMIT keyword
 # group: [order]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15576
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15577
 
 statement ok
 PRAGMA enable_verification
@@ -181,4 +183,4 @@ SELECT * FROM integers as int LIMIT (SELECT -1);
 
 #Subqueries that return strings that can't be converted to integers
 statement error
-SELECT * FROM integers as int LIMIT (SELECT 'ab');
\ No newline at end of file
+SELECT * FROM integers as int LIMIT (SELECT 'ab');
diff --git a/modules/calcite/src/test/sql/order/test_nulls_first.test b/modules/calcite/src/test/sql/order/test_nulls_first.test
index a97084a..0d9b6ce 100644
--- a/modules/calcite/src/test/sql/order/test_nulls_first.test
+++ b/modules/calcite/src/test/sql/order/test_nulls_first.test
@@ -13,7 +13,7 @@ INSERT INTO integers VALUES (1), (NULL)
 
 # default is NULLS FIRST
 query I
-SELECT * FROM integers ORDER BY i
+SELECT * FROM integers ORDER BY i NULLS LAST
 ----
 1
 NULL
diff --git a/modules/calcite/src/test/sql/order/test_order_by.test b/modules/calcite/src/test/sql/order/test_order_by.test
index 7e98746..05b6e67 100644
--- a/modules/calcite/src/test/sql/order/test_order_by.test
+++ b/modules/calcite/src/test/sql/order/test_order_by.test
@@ -108,7 +108,7 @@ SELECT b, a FROM test WHERE a < 13 ORDER BY b DESC;
 
 # order by expression that is not in SELECT
 query II
-SELECT b % 2 AS f, a FROM test ORDER BY b % 4;
+SELECT b % 2 AS f, a FROM test ORDER BY b % 4, a;
 ----
 1	12
 0	11
diff --git a/modules/calcite/src/test/sql/order/test_order_by.test_ignore b/modules/calcite/src/test/sql/order/test_order_by.test_ignore
index 2536c60..fdc6a5b 100644
--- a/modules/calcite/src/test/sql/order/test_order_by.test_ignore
+++ b/modules/calcite/src/test/sql/order/test_order_by.test_ignore
@@ -116,7 +116,7 @@ SELECT b % 2 AS f, SUM(a) FROM test GROUP BY f ORDER BY b % 2;
 
 # order by expression that is not in SELECT
 query II
-SELECT b % 2 AS f, a FROM test ORDER BY b % 4;
+SELECT b % 2 AS f, a FROM test ORDER BY b % 4, a;
 ----
 1	12
 0	11
diff --git a/modules/calcite/src/test/sql/order/test_order_by_exceptions.test b/modules/calcite/src/test/sql/order/test_order_by_exceptions.test
index e6101cb..e636bbe 100644
--- a/modules/calcite/src/test/sql/order/test_order_by_exceptions.test
+++ b/modules/calcite/src/test/sql/order/test_order_by_exceptions.test
@@ -42,7 +42,7 @@ SELECT a % 2, b FROM test UNION SELECT b, a % 2 AS k ORDER BY a % 2
 
 # but works if not ambiguous
 query II
-SELECT a % 2, b FROM test UNION SELECT a % 2 AS k, b FROM test ORDER BY a % 2
+SELECT mod(a , 2) as a, b FROM test UNION SELECT a % 2 AS k, b FROM test ORDER BY a % 2
 ----
 0	21
 1	22
diff --git a/modules/calcite/src/test/sql/order/test_order_large.test b/modules/calcite/src/test/sql/order/test_order_large.test_ignore
similarity index 84%
rename from modules/calcite/src/test/sql/order/test_order_large.test
rename to modules/calcite/src/test/sql/order/test_order_large.test_ignore
index 64f0894..31f9642 100644
--- a/modules/calcite/src/test/sql/order/test_order_large.test
+++ b/modules/calcite/src/test/sql/order/test_order_large.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/order/test_order_large.test
 # description: Test ORDER BY with large table
 # group: [order]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 # order of a large table, with burgers and fries
 statement ok
diff --git a/modules/calcite/src/test/sql/order/test_order_pragma.test b/modules/calcite/src/test/sql/order/test_order_pragma.test
index 0e71815..9890cff 100644
--- a/modules/calcite/src/test/sql/order/test_order_pragma.test
+++ b/modules/calcite/src/test/sql/order/test_order_pragma.test
@@ -19,26 +19,10 @@ SELECT a FROM test ORDER BY a
 12
 13
 
-# we can change the default with a pragma
-statement ok
-PRAGMA default_order='DESCENDING'
-
 query I
-SELECT a FROM test ORDER BY a
+SELECT a FROM test ORDER BY a desc
 ----
 13
 12
 11
 
-statement ok
-PRAGMA default_order='ASC'
-
-statement error
-PRAGMA default_order())
-
-statement error
-PRAGMA default_order=UNKNOWN)
-
-statement error
-PRAGMA default_order=3)
-
diff --git a/modules/calcite/src/test/sql/order/test_order_variable_size_payload.test b/modules/calcite/src/test/sql/order/test_order_variable_size_payload.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/order/test_order_variable_size_payload.test
rename to modules/calcite/src/test/sql/order/test_order_variable_size_payload.test_ignore
index aaaeeaa..6c66004 100644
--- a/modules/calcite/src/test/sql/order/test_order_variable_size_payload.test
+++ b/modules/calcite/src/test/sql/order/test_order_variable_size_payload.test_ignore
@@ -1,6 +1,9 @@
 # name: test/sql/order/test_order_variable_size_payload.test
 # description: Test ORDER BY keyword (variable size sorting/payload columns)
 # group: [order]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14636
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14555
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_any_all.test b/modules/calcite/src/test/sql/subquery/any_all/test_any_all.test
index a2e76dd..f2f1258 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_any_all.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_any_all.test
@@ -15,22 +15,22 @@ INSERT INTO integers VALUES (1), (2), (3)
 query T
 SELECT 2 > ANY(SELECT * FROM integers)
 ----
-1
+true
 
 query T
 SELECT 1 > ANY(SELECT * FROM integers)
 ----
-0
+false
 
 query T
 SELECT 4 > ALL(SELECT * FROM integers)
 ----
-1
+true
 
 query T
 SELECT 1 > ALL(SELECT * FROM integers)
 ----
-0
+false
 
 # NULL input always results in NULL output
 query T
@@ -51,7 +51,7 @@ INSERT INTO integers VALUES (NULL)
 query T
 SELECT 2 > ANY(SELECT * FROM integers)
 ----
-1
+true
 
 query T
 SELECT 1 > ANY(SELECT * FROM integers)
@@ -67,7 +67,7 @@ NULL
 query T
 SELECT 1 > ALL(SELECT * FROM integers)
 ----
-0
+false
 
 # NULL input always results in NULL
 query T
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_correlated_any_all.test b/modules/calcite/src/test/sql/subquery/any_all/test_correlated_any_all.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/subquery/any_all/test_correlated_any_all.test
rename to modules/calcite/src/test/sql/subquery/any_all/test_correlated_any_all.test_ignore
index 468da7c..c68a40a 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_correlated_any_all.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_correlated_any_all.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/any_all/test_correlated_any_all.test
 # description: Test correlated ANY/ALL subqueries
 # group: [any_all]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15580
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_scalar_any_all.test b/modules/calcite/src/test/sql/subquery/any_all/test_scalar_any_all.test_ignore
similarity index 90%
rename from modules/calcite/src/test/sql/subquery/any_all/test_scalar_any_all.test
rename to modules/calcite/src/test/sql/subquery/any_all/test_scalar_any_all.test_ignore
index 18b1f39..116b9e6 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_scalar_any_all.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_scalar_any_all.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/any_all/test_scalar_any_all.test
 # description: Test scalar ANY/ALL queries
 # group: [any_all]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15582
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test b/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test
index 0deb5c9..a6f118c 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/any_all/test_scalar_in.test
 # description: Test scalar IN query
 # group: [any_all]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15582
 
 statement ok
 PRAGMA enable_verification
@@ -8,7 +9,7 @@ PRAGMA enable_verification
 query T
 SELECT 1 IN (SELECT 1)
 ----
-1
+true
 
 query T
 SELECT NULL IN (SELECT 1)
@@ -16,14 +17,9 @@ SELECT NULL IN (SELECT 1)
 NULL
 
 query T
-SELECT 1 IN (SELECT NULL)
-----
-NULL
-
-query T
 SELECT 1 IN (SELECT 2)
 ----
-0
+false
 
 statement ok
 CREATE TABLE integers(i INTEGER)
@@ -34,43 +30,9 @@ INSERT INTO integers VALUES (1), (2), (3)
 query T
 SELECT 4 IN (SELECT * FROM integers)
 ----
-0
+false
 
 query T
 SELECT 1 IN (SELECT * FROM integers)
 ----
-1
-
-query T
-SELECT 1 IN (SELECT * FROM integers) FROM integers
-----
-1
-1
-1
-
-statement ok
-INSERT INTO integers VALUES (NULL)
-
-query T
-SELECT 4 IN (SELECT * FROM integers)
-----
-NULL
-
-query T
-SELECT 1 IN (SELECT * FROM integers)
-----
-1
-
-query I
-SELECT * FROM integers WHERE (4 IN (SELECT * FROM integers)) IS NULL ORDER BY 1
-----
-NULL
-1
-2
-3
-
-query I
-SELECT * FROM integers WHERE (i IN (SELECT * FROM integers)) IS NULL ORDER BY 1
-----
-NULL
-
+true
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test b/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test_ignore
similarity index 94%
copy from modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test
copy to modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test_ignore
index 0deb5c9..16721d9 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_scalar_in.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/any_all/test_scalar_in.test
 # description: Test scalar IN query
 # group: [any_all]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15582
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_simple_not_in.test b/modules/calcite/src/test/sql/subquery/any_all/test_simple_not_in.test_ignore
similarity index 91%
rename from modules/calcite/src/test/sql/subquery/any_all/test_simple_not_in.test
rename to modules/calcite/src/test/sql/subquery/any_all/test_simple_not_in.test_ignore
index ad484b3..794e44f 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_simple_not_in.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_simple_not_in.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/any_all/test_simple_not_in.test
 # description: Test subqueries with (NOT) IN clause
 # group: [any_all]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15584
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test
index b7eccb8..f3d1717 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test
@@ -20,8 +20,8 @@ query IT
 SELECT i, i >= ALL(SELECT i FROM integers) FROM integers ORDER BY i
 ----
 NULL	NULL
-1	0
-2	0
+1	false
+2	false
 3	NULL
 
 query I
@@ -33,23 +33,23 @@ query IT
 SELECT i, i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i
 ----
 NULL	NULL
-1	0
-2	0
-3	1
+1	false
+2	false
+3	true
 
 query I
 SELECT i FROM integers WHERE i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL)
 ----
 3
 
-query I
-SELECT i FROM integers WHERE i > ALL(SELECT MIN(i) FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i > ALL(SELECT MIN(i) FROM integers);
 ----
 2
 3
 
-query I
-SELECT i FROM integers WHERE i < ALL(SELECT MAX(i) FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i < ALL(SELECT MAX(i) FROM integers);
 ----
 1
 2
@@ -63,28 +63,19 @@ SELECT i FROM integers WHERE i <= ALL(SELECT i FROM integers WHERE i IS NOT NULL
 ----
 1
 
-query I
-SELECT i FROM integers WHERE i = ALL(SELECT i FROM integers WHERE i=1)
-----
-1
-
-query I
+query I rowsort
 SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i=1)
 ----
 2
 3
 
 query I
-SELECT i FROM integers WHERE i = ALL(SELECT i FROM integers WHERE i IS NOT NULL)
-----
-
-query I
 SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i IS NOT NULL)
 ----
 
 # zero results always results in TRUE for ALL, even if "i" is NULL
 query I
-SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i>10) ORDER BY i;
+SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i>10) ORDER BY i NULLS FIRST
 ----
 NULL
 1
@@ -92,61 +83,53 @@ NULL
 3
 
 query IT
-SELECT i, i <> ALL(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i <> ALL(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	1
-1	1
-2	1
-3	1
+NULL	true
+1	true
+2	true
+3	true
 
 # zero results always results in FALSE for ANY
 query IT
-SELECT i, i > ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i > ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i = ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i = ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i >= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i >= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i <= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i <= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i < ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i < ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
-
-query IT
-SELECT i, i <> ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
-----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 # nested uncorrelated subqueries
 query I
@@ -157,52 +140,27 @@ SELECT (SELECT (SELECT (SELECT 42)))
 query T
 SELECT (SELECT EXISTS(SELECT * FROM integers WHERE i>2)) FROM integers;
 ----
-1
-1
-1
-1
-
-query IR
-SELECT (SELECT MAX(i) FROM integers) AS k, SUM(i) FROM integers GROUP BY k;
-----
-3	6.000000
-
-# subqueries in GROUP BY clause
-query TR
-SELECT i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL) AS k, SUM(i) FROM integers GROUP BY k ORDER BY k;
-----
-NULL	NULL
-0	3.000000
-1	3.000000
+true
+true
+true
+true
 
 query R
-SELECT SUM(i) FROM integers GROUP BY (i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL)) ORDER BY 1;
+SELECT SUM(i) FROM integers GROUP BY (i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL)) ORDER BY 1 NULLS FIRST;
 ----
 NULL
 3.000000
 3.000000
 
-query TR
-SELECT i >= ALL(SELECT MIN(i) FROM integers WHERE i IS NOT NULL) AS k, SUM(i) FROM integers GROUP BY k ORDER BY k;
-----
-NULL	NULL
-1	6.000000
-
 # subquery in CASE statement
 query IR
-SELECT i, SUM(CASE WHEN (i >= ALL(SELECT i FROM integers WHERE i=2)) THEN 1 ELSE 0 END) FROM integers GROUP BY i ORDER BY i;
+SELECT i, SUM(CASE WHEN (i >= ALL(SELECT i FROM integers WHERE i=2)) THEN 1 ELSE 0 END) FROM integers GROUP BY i ORDER BY i NULLS FIRST;
 ----
 NULL	0.000000
 1	0.000000
 2	1.000000
 3	1.000000
 
-# subquery in HAVING
-query IR
-SELECT i % 2 AS k, SUM(i) FROM integers GROUP BY k HAVING SUM(i) > (SELECT MAX(i) FROM integers)
-----
-1	4.000000
-
 query I
 SELECT i FROM integers WHERE NOT(i IN (SELECT i FROM integers WHERE i>1));
 ----
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test_ignore
similarity index 78%
copy from modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test
copy to modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test_ignore
index b7eccb8..e6a1855 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_all_subquery.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/any_all/test_uncorrelated_all_subquery.test
 # description: Test uncorrelated ALL subqueries
 # group: [any_all]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15583
 
 statement ok
 PRAGMA enable_verification
@@ -20,8 +21,8 @@ query IT
 SELECT i, i >= ALL(SELECT i FROM integers) FROM integers ORDER BY i
 ----
 NULL	NULL
-1	0
-2	0
+1	false
+2	false
 3	NULL
 
 query I
@@ -33,23 +34,23 @@ query IT
 SELECT i, i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i
 ----
 NULL	NULL
-1	0
-2	0
-3	1
+1	false
+2	false
+3	true
 
 query I
 SELECT i FROM integers WHERE i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL)
 ----
 3
 
-query I
-SELECT i FROM integers WHERE i > ALL(SELECT MIN(i) FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i > ALL(SELECT MIN(i) FROM integers);
 ----
 2
 3
 
-query I
-SELECT i FROM integers WHERE i < ALL(SELECT MAX(i) FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i < ALL(SELECT MAX(i) FROM integers);
 ----
 1
 2
@@ -69,22 +70,22 @@ SELECT i FROM integers WHERE i = ALL(SELECT i FROM integers WHERE i=1)
 1
 
 query I
+SELECT i FROM integers WHERE i = ALL(SELECT i FROM integers WHERE i IS NOT NULL)
+----
+
+query I rowsort
 SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i=1)
 ----
 2
 3
 
 query I
-SELECT i FROM integers WHERE i = ALL(SELECT i FROM integers WHERE i IS NOT NULL)
-----
-
-query I
 SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i IS NOT NULL)
 ----
 
 # zero results always results in TRUE for ALL, even if "i" is NULL
 query I
-SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i>10) ORDER BY i;
+SELECT i FROM integers WHERE i <> ALL(SELECT i FROM integers WHERE i>10) ORDER BY i NULLS FIRST
 ----
 NULL
 1
@@ -92,75 +93,61 @@ NULL
 3
 
 query IT
-SELECT i, i <> ALL(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i <> ALL(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	1
-1	1
-2	1
-3	1
+NULL	true
+1	true
+2	true
+3	true
 
 # zero results always results in FALSE for ANY
 query IT
-SELECT i, i > ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i > ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i = ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i = ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i >= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i >= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i <= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i <= ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i < ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i <> ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 query IT
-SELECT i, i <> ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i;
+SELECT i, i < ANY(SELECT i FROM integers WHERE i>10) FROM integers ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
-
-# nested uncorrelated subqueries
-query I
-SELECT (SELECT (SELECT (SELECT 42)))
-----
-42
-
-query T
-SELECT (SELECT EXISTS(SELECT * FROM integers WHERE i>2)) FROM integers;
-----
-1
-1
-1
-1
+NULL	false
+1	false
+2	false
+3	false
 
 query IR
 SELECT (SELECT MAX(i) FROM integers) AS k, SUM(i) FROM integers GROUP BY k;
@@ -169,40 +156,54 @@ SELECT (SELECT MAX(i) FROM integers) AS k, SUM(i) FROM integers GROUP BY k;
 
 # subqueries in GROUP BY clause
 query TR
-SELECT i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL) AS k, SUM(i) FROM integers GROUP BY k ORDER BY k;
+SELECT i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL) AS k, SUM(i) FROM integers GROUP BY k ORDER BY k NULLS FIRST;
+----
+NULL	NULL
+false	3.000000
+true	3.000000
+
+query TR
+SELECT i >= ALL(SELECT MIN(i) FROM integers WHERE i IS NOT NULL) AS k, SUM(i) FROM integers GROUP BY k ORDER BY k NULLS FIRST;
 ----
 NULL	NULL
-0	3.000000
-1	3.000000
+true	6.000000
+
+# subquery in HAVING
+query IR
+SELECT i % 2 AS k, SUM(i) FROM integers GROUP BY k HAVING SUM(i) > (SELECT MAX(i) FROM integers)
+----
+1	4.000000
+
+# nested uncorrelated subqueries
+query I
+SELECT (SELECT (SELECT (SELECT 42)))
+----
+42
+
+query T
+SELECT (SELECT EXISTS(SELECT * FROM integers WHERE i>2)) FROM integers;
+----
+true
+true
+true
+true
 
 query R
-SELECT SUM(i) FROM integers GROUP BY (i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL)) ORDER BY 1;
+SELECT SUM(i) FROM integers GROUP BY (i >= ALL(SELECT i FROM integers WHERE i IS NOT NULL)) ORDER BY 1 NULLS FIRST;
 ----
 NULL
 3.000000
 3.000000
 
-query TR
-SELECT i >= ALL(SELECT MIN(i) FROM integers WHERE i IS NOT NULL) AS k, SUM(i) FROM integers GROUP BY k ORDER BY k;
-----
-NULL	NULL
-1	6.000000
-
 # subquery in CASE statement
 query IR
-SELECT i, SUM(CASE WHEN (i >= ALL(SELECT i FROM integers WHERE i=2)) THEN 1 ELSE 0 END) FROM integers GROUP BY i ORDER BY i;
+SELECT i, SUM(CASE WHEN (i >= ALL(SELECT i FROM integers WHERE i=2)) THEN 1 ELSE 0 END) FROM integers GROUP BY i ORDER BY i NULLS FIRST;
 ----
 NULL	0.000000
 1	0.000000
 2	1.000000
 3	1.000000
 
-# subquery in HAVING
-query IR
-SELECT i % 2 AS k, SUM(i) FROM integers GROUP BY k HAVING SUM(i) > (SELECT MAX(i) FROM integers)
-----
-1	4.000000
-
 query I
 SELECT i FROM integers WHERE NOT(i IN (SELECT i FROM integers WHERE i>1));
 ----
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test
index d5b62ba..ea26867 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test
@@ -12,37 +12,37 @@ statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
 # uncorrelated ANY
-query I
-SELECT i FROM integers WHERE i <= ANY(SELECT i FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i <= ANY(SELECT i FROM integers);
 ----
 1
 2
 3
 
-query I
-SELECT i FROM integers WHERE i > ANY(SELECT i FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i > ANY(SELECT i FROM integers);
 ----
 2
 3
 
 query IT
-SELECT i, i > ANY(SELECT i FROM integers) FROM integers ORDER BY i
+SELECT i, i > ANY(SELECT i FROM integers) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
-2	1
-3	1
+2	true
+3	true
 
 query IT
-SELECT i, i > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i
+SELECT i, i > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	0
-2	1
-3	1
+1	false
+2	true
+3	true
 
 query IT
-SELECT i, NULL > ANY(SELECT i FROM integers) FROM integers ORDER BY i
+SELECT i, NULL > ANY(SELECT i FROM integers) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
@@ -50,14 +50,14 @@ NULL	NULL
 3	NULL
 
 query IT
-SELECT i, NULL > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i
+SELECT i, NULL > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
 2	NULL
 3	NULL
 
-query I
+query I rowsort
 SELECT i FROM integers WHERE i = ANY(SELECT i FROM integers)
 ----
 1
@@ -65,67 +65,17 @@ SELECT i FROM integers WHERE i = ANY(SELECT i FROM integers)
 3
 
 query IT
-SELECT i, i = ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i
+SELECT i, i = ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	0
-2	0
-3	1
+1	false
+2	false
+3	true
 
 query IT
-SELECT i, i = ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i
+SELECT i, i = ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
 2	NULL
-3	1
-
-query IT
-SELECT i, i <> ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i
-----
-NULL	NULL
-1	1
-2	1
-3	0
-
-query IT
-SELECT i, i <> ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i
-----
-NULL	NULL
-1	1
-2	1
-3	NULL
-
-# use a bunch of cross products to make bigger data sets (> STANDARD_VECTOR_SIZE)
-query IT
-SELECT i, i = ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL) FROM integers ORDER BY i
-----
-NULL	NULL
-1	1
-2	1
-3	1
-
-query IT
-SELECT i, i = ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL AND i1.i <> 2) FROM integers ORDER BY i
-----
-NULL	NULL
-1	1
-2	0
-3	1
-
-query IT
-SELECT i, i >= ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL) FROM integers ORDER BY i
-----
-NULL	NULL
-1	1
-2	1
-3	1
-
-query IT
-SELECT i, i >= ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL AND i1.i <> 1 LIMIT 1) FROM integers ORDER BY i
-----
-NULL	NULL
-1	0
-2	1
-3	1
-
+3	true
diff --git a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test_ignore
similarity index 69%
copy from modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test
copy to modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test_ignore
index d5b62ba..c4c7120 100644
--- a/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/any_all/test_uncorrelated_any_subquery.test_ignore
@@ -12,37 +12,37 @@ statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
 # uncorrelated ANY
-query I
-SELECT i FROM integers WHERE i <= ANY(SELECT i FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i <= ANY(SELECT i FROM integers);
 ----
 1
 2
 3
 
-query I
-SELECT i FROM integers WHERE i > ANY(SELECT i FROM integers)
+query I rowsort
+SELECT i FROM integers WHERE i > ANY(SELECT i FROM integers);
 ----
 2
 3
 
 query IT
-SELECT i, i > ANY(SELECT i FROM integers) FROM integers ORDER BY i
+SELECT i, i > ANY(SELECT i FROM integers) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
-2	1
-3	1
+2	true
+3	true
 
 query IT
-SELECT i, i > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i
+SELECT i, i > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	0
-2	1
-3	1
+1	false
+2	true
+3	true
 
 query IT
-SELECT i, NULL > ANY(SELECT i FROM integers) FROM integers ORDER BY i
+SELECT i, NULL > ANY(SELECT i FROM integers) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
@@ -50,14 +50,14 @@ NULL	NULL
 3	NULL
 
 query IT
-SELECT i, NULL > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i
+SELECT i, NULL > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
 2	NULL
 3	NULL
 
-query I
+query I rowsort
 SELECT i FROM integers WHERE i = ANY(SELECT i FROM integers)
 ----
 1
@@ -65,67 +65,67 @@ SELECT i FROM integers WHERE i = ANY(SELECT i FROM integers)
 3
 
 query IT
-SELECT i, i = ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i
+SELECT i, i = ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	0
-2	0
-3	1
+1	false
+2	false
+3	true
 
 query IT
-SELECT i, i = ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i
+SELECT i, i = ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
 1	NULL
 2	NULL
-3	1
+3	true
 
 query IT
-SELECT i, i <> ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i
+SELECT i, i <> ANY(SELECT i FROM integers WHERE i>2) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	1
-2	1
-3	0
+1	true
+2	true
+3	false
 
 query IT
-SELECT i, i <> ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i
+SELECT i, i <> ANY(SELECT i FROM integers WHERE i>2 OR i IS NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	1
-2	1
+1	true
+2	true
 3	NULL
 
 # use a bunch of cross products to make bigger data sets (> STANDARD_VECTOR_SIZE)
 query IT
-SELECT i, i = ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL) FROM integers ORDER BY i
+SELECT i, i = ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	1
-2	1
-3	1
+1	true
+2	true
+3	true
 
 query IT
-SELECT i, i = ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL AND i1.i <> 2) FROM integers ORDER BY i
+SELECT i, i = ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL AND i1.i <> 2) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	1
-2	0
-3	1
+1	true
+2	false
+3	true
 
 query IT
-SELECT i, i >= ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL) FROM integers ORDER BY i
+SELECT i, i >= ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	1
-2	1
-3	1
+1	true
+2	true
+3	true
 
 query IT
-SELECT i, i >= ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL AND i1.i <> 1 LIMIT 1) FROM integers ORDER BY i
+SELECT i, i >= ANY(SELECT i1.i FROM integers i1, integers i2, integers i3, integers i4, integers i5, integers i6 WHERE i1.i IS NOT NULL AND i1.i <> 1 LIMIT 1) FROM integers ORDER BY i NULLS FIRST
 ----
 NULL	NULL
-1	0
-2	1
-3	1
+1	false
+2	true
+3	true
 
diff --git a/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test b/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test
index d6a2d0ea..9d00e09 100644
--- a/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test
+++ b/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test
@@ -13,114 +13,19 @@ INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
 # correlated EXISTS
 query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>2) FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>2) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	1
+NULL	false
+1	false
+2	false
+3	true
 
 query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i=i1.i) FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(SELECT i FROM integers WHERE i IS NULL OR i>i1.i*10) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	1
-2	1
-3	1
+NULL	true
+1	true
+2	true
+3	true
 
-query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i IS NULL OR i>i1.i*10) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	1
-2	1
-3	1
-
-query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>i OR i1.i IS NULL) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	0
-2	1
-3	1
-
-query I
-SELECT i FROM integers i1 WHERE EXISTS(SELECT i FROM integers WHERE i=i1.i) ORDER BY i;
-----
-1
-2
-3
-
-# correlated EXISTS with aggregations
-query T
-SELECT EXISTS(SELECT i FROM integers WHERE i>MIN(i1.i)) FROM integers i1;
-----
-1
-
-query IR
-SELECT i, SUM(i) FROM integers i1 GROUP BY i HAVING EXISTS(SELECT i FROM integers WHERE i>MIN(i1.i)) ORDER BY i;
-----
-1	1.000000
-2	2.000000
-
-query T
-SELECT EXISTS(SELECT i+MIN(i1.i) FROM integers WHERE i=3) FROM integers i1;
-----
-1
-
-query T
-SELECT EXISTS(SELECT i+MIN(i1.i) FROM integers WHERE i=5) FROM integers i1;
-----
-0
-
-# GROUP BY correlated exists
-query TI
-SELECT EXISTS(SELECT i FROM integers WHERE i=i1.i) AS g, COUNT(*) FROM integers i1 GROUP BY g ORDER BY g;
-----
-0	1
-1	3
-
-# SUM on exists
-query R
-SELECT SUM(CASE WHEN EXISTS(SELECT i FROM integers WHERE i=i1.i) THEN 1 ELSE 0 END) FROM integers i1;
-----
-3.000000
-
-# aggregates with multiple parameters
-query R
-SELECT (SELECT COVAR_POP(i1.i, i2.i) FROM integers i2) FROM integers i1 ORDER BY 1
-----
-NULL
-0.000000
-0.000000
-0.000000
-
-query R
-SELECT (SELECT COVAR_POP(i2.i, i1.i) FROM integers i2) FROM integers i1 ORDER BY 1
-----
-NULL
-0.000000
-0.000000
-0.000000
-
-query R
-SELECT (SELECT COVAR_POP(i1.i+i2.i, i1.i+i2.i) FROM integers i2) FROM integers i1 ORDER BY 1
-----
-NULL
-0.666667
-0.666667
-0.666667
-
-query R
-SELECT (SELECT COVAR_POP(i2.i, i2.i) FROM integers i2) FROM integers i1 ORDER BY 1;
-----
-0.666667
-0.666667
-0.666667
-0.666667
-
-query R
-SELECT (SELECT COVAR_POP(i1.i, i1.i) FROM integers i2 LIMIT 1) FROM integers i1 ORDER BY 1;
-----
-0.666667
 
diff --git a/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test b/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test_ignore
similarity index 72%
copy from modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test
copy to modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test_ignore
index d6a2d0ea..b63b773 100644
--- a/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test
+++ b/modules/calcite/src/test/sql/subquery/exists/test_correlated_exists.test_ignore
@@ -1,6 +1,12 @@
 # name: test/sql/subquery/exists/test_correlated_exists.test
 # description: Test correlated exists
 # group: [exists]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15585
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15586
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15587
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15588
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15589
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15583
 
 statement ok
 PRAGMA enable_verification
@@ -13,36 +19,28 @@ INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
 # correlated EXISTS
 query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>2) FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>2) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	1
+NULL	false
+1	false
+2	false
+3	true
 
 query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i=i1.i) FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(SELECT i FROM integers WHERE i=i1.i) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	1
-2	1
-3	1
+NULL	false
+1	true
+2	true
+3	true
 
 query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i IS NULL OR i>i1.i*10) FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>i OR i1.i IS NULL) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	1
-1	1
-2	1
-3	1
-
-query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE i1.i>i OR i1.i IS NULL) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	0
-2	1
-3	1
+NULL	true
+1	false
+2	true
+3	true
 
 query I
 SELECT i FROM integers i1 WHERE EXISTS(SELECT i FROM integers WHERE i=i1.i) ORDER BY i;
@@ -55,7 +53,7 @@ SELECT i FROM integers i1 WHERE EXISTS(SELECT i FROM integers WHERE i=i1.i) ORDE
 query T
 SELECT EXISTS(SELECT i FROM integers WHERE i>MIN(i1.i)) FROM integers i1;
 ----
-1
+true
 
 query IR
 SELECT i, SUM(i) FROM integers i1 GROUP BY i HAVING EXISTS(SELECT i FROM integers WHERE i>MIN(i1.i)) ORDER BY i;
@@ -66,19 +64,19 @@ SELECT i, SUM(i) FROM integers i1 GROUP BY i HAVING EXISTS(SELECT i FROM integer
 query T
 SELECT EXISTS(SELECT i+MIN(i1.i) FROM integers WHERE i=3) FROM integers i1;
 ----
-1
+true
 
 query T
 SELECT EXISTS(SELECT i+MIN(i1.i) FROM integers WHERE i=5) FROM integers i1;
 ----
-0
+false
 
 # GROUP BY correlated exists
 query TI
 SELECT EXISTS(SELECT i FROM integers WHERE i=i1.i) AS g, COUNT(*) FROM integers i1 GROUP BY g ORDER BY g;
 ----
-0	1
-1	3
+false	1
+true	3
 
 # SUM on exists
 query R
@@ -86,17 +84,27 @@ SELECT SUM(CASE WHEN EXISTS(SELECT i FROM integers WHERE i=i1.i) THEN 1 ELSE 0 E
 ----
 3.000000
 
-# aggregates with multiple parameters
+48
+
+query IT
+SELECT i, EXISTS(SELECT i FROM integers WHERE i IS NULL OR i>i1.i*10) FROM integers i1 ORDER BY i NULLS FIRST;
+----
+NULL	true
+1	true
+2	true
+3	true
+
 query R
-SELECT (SELECT COVAR_POP(i1.i, i2.i) FROM integers i2) FROM integers i1 ORDER BY 1
+SELECT (SELECT COVAR_POP(i2.i, i1.i) FROM integers i2) FROM integers i1 ORDER BY 1 NULLS FIRST
 ----
 NULL
 0.000000
 0.000000
 0.000000
 
+# aggregates with multiple parameters
 query R
-SELECT (SELECT COVAR_POP(i2.i, i1.i) FROM integers i2) FROM integers i1 ORDER BY 1
+SELECT (SELECT COVAR_POP(i1.i, i2.i) FROM integers i2) FROM integers i1 ORDER BY 1 NULLS FIRST
 ----
 NULL
 0.000000
@@ -104,7 +112,7 @@ NULL
 0.000000
 
 query R
-SELECT (SELECT COVAR_POP(i1.i+i2.i, i1.i+i2.i) FROM integers i2) FROM integers i1 ORDER BY 1
+SELECT (SELECT COVAR_POP(i1.i+i2.i, i1.i+i2.i) FROM integers i2) FROM integers i1 ORDER BY 1 NULLS FIRST
 ----
 NULL
 0.666667
@@ -123,4 +131,3 @@ query R
 SELECT (SELECT COVAR_POP(i1.i, i1.i) FROM integers i2 LIMIT 1) FROM integers i1 ORDER BY 1;
 ----
 0.666667
-
diff --git a/modules/calcite/src/test/sql/subquery/exists/test_scalar_exists.test b/modules/calcite/src/test/sql/subquery/exists/test_scalar_exists.test
index 3231d3f..54cbff8 100644
--- a/modules/calcite/src/test/sql/subquery/exists/test_scalar_exists.test
+++ b/modules/calcite/src/test/sql/subquery/exists/test_scalar_exists.test
@@ -8,7 +8,7 @@ PRAGMA enable_verification
 query T
 SELECT EXISTS(SELECT 1)
 ----
-1
+true
 
 statement ok
 CREATE TABLE integers(i INTEGER)
@@ -19,18 +19,18 @@ INSERT INTO integers VALUES (1), (2), (3), (NULL)
 query T
 SELECT EXISTS(SELECT 1) FROM integers
 ----
-1
-1
-1
-1
+true
+true
+true
+true
 
 query T
 SELECT EXISTS(SELECT * FROM integers)
 ----
-1
+true
 
 query T
 SELECT EXISTS(SELECT * FROM integers WHERE i IS NULL)
 ----
-1
+true
 
diff --git a/modules/calcite/src/test/sql/subquery/exists/test_uncorrelated_exists_subquery.test b/modules/calcite/src/test/sql/subquery/exists/test_uncorrelated_exists_subquery.test
index d068a36..f688021 100644
--- a/modules/calcite/src/test/sql/subquery/exists/test_uncorrelated_exists_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/exists/test_uncorrelated_exists_subquery.test
@@ -44,34 +44,34 @@ NULL
 query T
 SELECT EXISTS(SELECT * FROM integers)
 ----
-1
+true
 
 query T
 SELECT EXISTS(SELECT * FROM integers WHERE i>10)
 ----
-0
+false
 
 # multiple exists
 query TT
 SELECT EXISTS(SELECT * FROM integers), EXISTS(SELECT * FROM integers)
 ----
-1	1
+true	true
 
 # exists used in operations
 query T
 SELECT EXISTS(SELECT * FROM integers) AND EXISTS(SELECT * FROM integers)
 ----
-1
+true
 
 # nested EXISTS
 query T
 SELECT EXISTS(SELECT EXISTS(SELECT * FROM integers))
 ----
-1
+true
 
 # uncorrelated IN
 query I
-SELECT * FROM integers WHERE 1 IN (SELECT 1) ORDER BY i
+SELECT * FROM integers WHERE 1 IN (SELECT 1) ORDER BY i NULLS FIRST
 ----
 NULL
 1
@@ -79,7 +79,7 @@ NULL
 3
 
 query I
-SELECT * FROM integers WHERE 1 IN (SELECT * FROM integers) ORDER BY i
+SELECT * FROM integers WHERE 1 IN (SELECT * FROM integers) ORDER BY i NULLS FIRST
 ----
 NULL
 1
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test
index cfe76c7..5c7bc20 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test
@@ -20,76 +20,10 @@ NULL	NULL
 2	2
 3	3
 
-# join on two subqueries that both have a correlated expression in them
-query II
-SELECT i, (SELECT s1.i FROM (SELECT i FROM integers WHERE i=i1.i) s1 INNER JOIN (SELECT i FROM integers WHERE i=4-i1.i) s2 ON s1.i>s2.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	NULL
-2	NULL
-3	3
-
-# implicit join with correlated expression in filter
-query II
-SELECT i, (SELECT s1.i FROM integers s1, integers s2 WHERE s1.i=s2.i AND s1.i=4-i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	3
-2	2
-3	1
-
-# join with a correlated expression in the join condition
-query II
-SELECT i, (SELECT s1.i FROM integers s1 INNER JOIN integers s2 ON s1.i=s2.i AND s1.i=4-i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	3
-2	2
-3	1
-
-# inner join on correlated subquery
-query II
-SELECT * FROM integers s1 INNER JOIN integers s2 ON (SELECT 2*SUM(i)*s1.i FROM integers)=(SELECT SUM(i)*s2.i FROM integers) ORDER BY s1.i;
-----
-1	2
-
-# inner join on non-equality subquery
-query II
-SELECT * FROM integers s1 INNER JOIN integers s2 ON (SELECT s1.i=s2.i) ORDER BY s1.i;
-----
-1	1
-2	2
-3	3
-
-query II
-SELECT * FROM integers s1 INNER JOIN integers s2 ON (SELECT s1.i=i FROM integers WHERE s2.i=i) ORDER BY s1.i;
-----
-1	1
-2	2
-3	3
-
-# left outer join on comparison between correlated subqueries
-query TT
-SELECT * FROM integers s1 LEFT OUTER JOIN integers s2 ON (SELECT 2*SUM(i)*s1.i FROM integers)=(SELECT SUM(i)*s2.i FROM integers) ORDER BY s1.i;
-----
-NULL	NULL
-1	2
-2	NULL
-3	NULL
-
 # left outer join on arbitrary correlated subquery: not supported
 statement error
 SELECT * FROM integers s1 LEFT OUTER JOIN integers s2 ON (SELECT CASE WHEN s1.i+s2.i>10 THEN TRUE ELSE FALSE END) ORDER BY s1.i;
 
-# left outer join on subquery only involving RHS works
-query TT
-SELECT * FROM integers s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i AND (SELECT CASE WHEN s2.i>2 THEN TRUE ELSE FALSE END) ORDER BY s1.i;
-----
-NULL	NULL
-1	NULL
-2	NULL
-3	3
-
 # left outer join on subquery only involving LHS is not supported
 statement error
 SELECT * FROM integers s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i AND (SELECT CASE WHEN s1.i>2 THEN TRUE ELSE FALSE END) ORDER BY s1.i;
@@ -110,48 +44,3 @@ SELECT i, (SELECT SUM(s1.i) FROM integers s1 FULL OUTER JOIN integers s2 ON s1.i
 # correlated expression inside window function not supported
 statement error
 SELECT i, (SELECT row_number() OVER (ORDER BY i)) FROM integers i1 ORDER BY i;
-
-# union with correlated expression
-query II
-SELECT i, (SELECT i FROM integers WHERE i=i1.i UNION SELECT i FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	1
-2	2
-3	3
-
-# except with correlated expression
-query II
-SELECT i, (SELECT i FROM integers WHERE i IS NOT NULL EXCEPT SELECT i FROM integers WHERE i<>i1.i) AS j FROM integers i1 WHERE i IS NOT NULL ORDER BY i;
-----
-1	1
-2	2
-3	3
-
-# intersect with correlated expression
-query II
-SELECT i, (SELECT i FROM integers WHERE i=i1.i INTERSECT SELECT i FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	1
-2	2
-3	3
-
-# multiple setops
-query II
-SELECT i, (SELECT i FROM integers WHERE i=i1.i UNION SELECT i FROM integers WHERE i<>i1.i EXCEPT SELECT i FROM integers WHERE i<>i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	1
-2	2
-3	3
-
-# uncorrelated query inside correlated query
-query IR
-SELECT i, (SELECT (SELECT SUM(i) FROM integers)+42+i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	49.000000
-2	50.000000
-3	51.000000
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test_ignore
similarity index 92%
copy from modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test_ignore
index cfe76c7..2fdb4b1 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_complex_correlated_subquery.test_ignore
@@ -1,6 +1,12 @@
 # name: test/sql/subquery/scalar/test_complex_correlated_subquery.test
 # description: Test complex correlated subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15595
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15594
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15593
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15588
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15592
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test
index 864072d..462909f 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test
@@ -14,35 +14,9 @@ INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-0
 query T
 SELECT EXISTS(SELECT t1.b+t1.c) FROM tbl t1
 ----
-1
+true
 
 query I
 SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1
 ----
 5
-
-query I
-SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01'))
-----
-1
-
-query T
-SELECT EXISTS(SELECT 1 WHERE (t1.c>100 OR 1) AND t1.d<100) FROM tbl t1
-----
-1
-
-query T
-SELECT EXISTS(SELECT t1.c,t1.d WHERE t1.d<100) FROM tbl t1
-----
-1
-
-query TTTTTTTTTTTTTT
-SELECT * FROM tbl t1 LEFT JOIN tbl t2 ON (SELECT t2.a)<100
-----
-1	2	3	4	5	1992-01-01	1992-01-01 00:00:00	1	2	3	4	5	1992-01-01	1992-01-01 00:00:00
-
-query TTTTTTTTTTTTTT
-SELECT * FROM tbl t1 LEFT JOIN tbl t2 ON (SELECT t2.a)>100
-----
-1	2	3	4	5	1992-01-01	1992-01-01 00:00:00	NULL	NULL	NULL	NULL	NULL	NULL	NULL
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test_ignore
similarity index 86%
copy from modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test_ignore
index 864072d..32b3961 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test_ignore
@@ -1,6 +1,9 @@
 # name: test/sql/subquery/scalar/test_complex_nested_correlated_subquery.test
 # description: Test nested correlated subqueries with multiple columns
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15594
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15596
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15584
 
 statement ok
 PRAGMA enable_verification
@@ -14,7 +17,7 @@ INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-0
 query T
 SELECT EXISTS(SELECT t1.b+t1.c) FROM tbl t1
 ----
-1
+true
 
 query I
 SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
index 5acec50..3afa462 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
@@ -1,7 +1,3 @@
-# name: test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
-# description: Test correlated aggregate subqueries
-# group: [scalar]
-
 statement ok
 PRAGMA enable_verification
 
@@ -11,69 +7,15 @@ CREATE TABLE integers(i INTEGER)
 statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
-# aggregate with correlation in final projection
-query II
-SELECT i, (SELECT MIN(i)+i1.i FROM integers) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	2
-2	3
-3	4
-
-# aggregate with correlation inside aggregation
-query II
-SELECT i, (SELECT MIN(i+2*i1.i) FROM integers) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	3
-2	5
-3	7
-
-query IRR
-SELECT i, SUM(i), (SELECT SUM(i)+SUM(i1.i) FROM integers) FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL	NULL
-1	1.000000	7.000000
-2	2.000000	8.000000
-3	3.000000	9.000000
-
-query IRR
-SELECT i, SUM(i), (SELECT SUM(i)+COUNT(i1.i) FROM integers) FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL	6.000000
-1	1.000000	7.000000
-2	2.000000	7.000000
-3	3.000000	7.000000
-
 # correlated COUNT(*)
 query II
-SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i) FROM integers i1 ORDER BY i;
+SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	0
 1	2
 2	1
 3	0
 
-# aggregate with correlation inside aggregation
-query II
-SELECT i, (SELECT MIN(i+2*i1.i) FROM integers) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	3
-2	5
-3	7
-
-# aggregate ONLY inside subquery
-query R
-SELECT (SELECT SUM(i1.i)) FROM integers i1;
-----
-6.000000
-
-# aggregate ONLY inside subquery, with column reference outside of subquery
-query IR
-SELECT MIN(i), (SELECT SUM(i1.i)) FROM integers i1;
-----
-1	6.000000
 
 # this will fail, because "i" is not an aggregate but the SUM(i1.i) turns this query into an aggregate
 statement error
@@ -81,293 +23,3 @@ SELECT i, (SELECT SUM(i1.i)) FROM integers i1;
 
 statement error
 SELECT i+1, (SELECT SUM(i1.i)) FROM integers i1;
-
-query IR
-SELECT MIN(i), (SELECT SUM(i1.i)) FROM integers i1;
-----
-1	6.000000
-
-query RR
-SELECT (SELECT SUM(i1.i)), (SELECT SUM(i1.i)) FROM integers i1;
-----
-6.000000	6.000000
-
-# subquery inside aggregation
-query RR
-SELECT SUM(i), SUM((SELECT i FROM integers WHERE i=i1.i)) FROM integers i1;
-----
-6.000000	6.000000
-
-query RR
-SELECT SUM(i), (SELECT SUM(i) FROM integers WHERE i>SUM(i1.i)) FROM integers i1;
-----
-6.000000	NULL
-
-# subquery with aggregation inside aggregation should fail
-statement error
-SELECT SUM((SELECT SUM(i))) FROM integers
-
-# aggregate with correlation in filter
-query II
-SELECT i, (SELECT MIN(i) FROM integers WHERE i>i1.i) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	2
-2	3
-3	NULL
-
-# aggregate with correlation in both filter and projection
-query II
-SELECT i, (SELECT MIN(i)+i1.i FROM integers WHERE i>i1.i) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	3
-2	5
-3	NULL
-
-# aggregate with correlation in GROUP BY
-query II
-SELECT i, (SELECT MIN(i) FROM integers GROUP BY i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	1
-2	1
-3	1
-
-# aggregate with correlation in HAVING clause
-query II
-SELECT i, (SELECT i FROM integers GROUP BY i HAVING i=i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	1
-2	2
-3	3
-
-# correlated subquery in HAVING
-query IR
-SELECT i1.i, SUM(i) FROM integers i1 GROUP BY i1.i HAVING SUM(i)=(SELECT MIN(i) FROM integers WHERE i<>i1.i+1) ORDER BY 1;
-----
-1	1.000000
-
-query IR
-SELECT i % 2 AS j, SUM(i) FROM integers i1 GROUP BY j HAVING SUM(i)=(SELECT SUM(i) FROM integers WHERE i<>j+1) ORDER BY 1;
-----
-1	4.000000
-
-# aggregate query with non-aggregate subquery without group by
-query R
-SELECT (SELECT i+SUM(i1.i) FROM integers WHERE i=1 LIMIT 1) FROM integers i1;
-----
-7.000000
-
-query R
-SELECT (SELECT SUM(i)+SUM(i1.i) FROM integers) FROM integers i1 ORDER BY 1;
-----
-12.000000
-
-query R
-SELECT (SELECT SUM(i)+SUM((CASE WHEN i IS NOT NULL THEN i*0 ELSE 0 END)+i1.i) FROM integers) FROM integers i1 ORDER BY 1;
-----
-NULL
-10.000000
-14.000000
-18.000000
-
-# aggregate query with non-aggregate subquery with group by
-query IR
-SELECT i, (SELECT i+SUM(i1.i) FROM integers WHERE i=1) FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL
-1	2.000000
-2	3.000000
-3	4.000000
-
-# subquery inside aggregate
-query R
-SELECT SUM((SELECT i+i1.i FROM integers WHERE i=1)) FROM integers i1;
-----
-9.000000
-
-query IRR
-SELECT i, SUM(i1.i), (SELECT SUM(i1.i) FROM integers) AS k FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL	NULL
-1	1.000000	1.000000
-2	2.000000	2.000000
-3	3.000000	3.000000
-
-# aggregation of both entries inside subquery
-# aggregate on group inside subquery
-query IR
-SELECT i1.i AS j, (SELECT SUM(j+i) FROM integers) AS k FROM integers i1 GROUP BY j ORDER BY j;
-----
-NULL	NULL
-1	9.000000
-2	12.000000
-3	15.000000
-
-query R
-SELECT (SELECT SUM(i1.i*i) FROM integers) FROM integers i1 ORDER BY i;
-----
-NULL
-6.000000
-12.000000
-18.000000
-
-query IRR
-SELECT i, (SELECT SUM(i1.i)) AS k, (SELECT SUM(i1.i)) AS l FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL	NULL
-1	1.000000	1.000000
-2	2.000000	2.000000
-3	3.000000	3.000000
-
-# refer aggregation inside subquery
-query IR
-SELECT i, (SELECT SUM(i1.i)*SUM(i) FROM integers) AS k FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL
-1	6.000000
-2	12.000000
-3	18.000000
-
-# refer to GROUP BY inside subquery
-query IR
-SELECT i AS j, (SELECT j*SUM(i) FROM integers) AS k FROM integers i1 GROUP BY j ORDER BY j;
-----
-NULL	NULL
-1	6.000000
-2	12.000000
-3	18.000000
-
-# refer to GROUP BY without alias but with full name
-query IR
-SELECT i AS j, (SELECT i1.i*SUM(i) FROM integers) AS k FROM integers i1 GROUP BY j ORDER BY j;
-----
-NULL	NULL
-1	6.000000
-2	12.000000
-3	18.000000
-
-# perform SUM on subquery
-query IR
-SELECT i, SUM((SELECT SUM(i)*i1.i FROM integers)) AS k FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL
-1	6.000000
-2	12.000000
-3	18.000000
-
-# aggregate subqueries cannot be nested
-statement error
-SELECT i, SUM((SELECT SUM(i)*SUM(i1.i) FROM integers)) AS k FROM integers i1 GROUP BY i ORDER BY i;
-
-# aggregation but ONLY inside subquery results in implicit aggregation
-query R
-SELECT (SELECT SUM(i1.i)) FROM integers i1;
-----
-6.000000
-
-query IR
-SELECT FIRST(i), (SELECT SUM(i1.i)) FROM integers i1;
-----
-1	6.000000
-
-# aggregate that uses correlated column in aggregation
-query II
-SELECT i AS j, (SELECT MIN(i1.i) FROM integers GROUP BY i HAVING i=j) FROM integers i1 GROUP BY j ORDER BY j;
-----
-NULL	NULL
-1	1
-2	2
-3	3
-
-# ORDER BY correlated subquery
-query IR
-SELECT i, SUM(i1.i) FROM integers i1 GROUP BY i ORDER BY (SELECT SUM(i1.i) FROM integers);
-----
-NULL	NULL
-1	1.000000
-2	2.000000
-3	3.000000
-
-# LIMIT 0 on correlated subquery
-query IR
-SELECT i, SUM((SELECT SUM(i)*i1.i FROM integers LIMIT 0)) AS k FROM integers i1 GROUP BY i ORDER BY i;
-----
-NULL	NULL
-1	NULL
-2	NULL
-3	NULL
-
-# GROUP BY correlated subquery
-query IR
-SELECT (SELECT i+i1.i FROM integers WHERE i=1) AS k, SUM(i) AS j FROM integers i1 GROUP BY k ORDER BY 1;
-----
-NULL	NULL
-2	1.000000
-3	2.000000
-4	3.000000
-
-# correlated subquery in WHERE
-query R
-SELECT SUM(i) FROM integers i1 WHERE i>(SELECT (i+i1.i)/2 FROM integers WHERE i=1);
-----
-5.000000
-
-# correlated aggregate in WHERE
-query R
-SELECT SUM(i) FROM integers i1 WHERE i>(SELECT (SUM(i)+i1.i)/2 FROM integers WHERE i=1);
-----
-5.000000
-
-# use scalar subquery as argument to ALL/ANY
-query IT
-SELECT i, (SELECT MIN(i) FROM integers WHERE i=i1.i) >= ALL(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	0
-2	0
-3	1
-
-query IT
-SELECT i, (SELECT MIN(i) FROM integers WHERE i<>i1.i) > ANY(SELECT i FROM integers WHERE i IS NOT NULL) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	1
-2	0
-3	0
-
-query IT
-SELECT i, NOT((SELECT MIN(i) FROM integers WHERE i<>i1.i) > ANY(SELECT i FROM integers WHERE i IS NOT NULL)) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	0
-2	1
-3	1
-
-# aggregates with multiple parameters
-query R
-SELECT (SELECT COVAR_POP(i1.i, i2.i) FROM integers i2) FROM integers i1 ORDER BY 1
-----
-NULL
-0.000000
-0.000000
-0.000000
-
-query R
-SELECT (SELECT COVAR_POP(i2.i, i1.i) FROM integers i2) FROM integers i1 ORDER BY 1
-----
-NULL
-0.000000
-0.000000
-0.000000
-
-query R
-SELECT (SELECT COVAR_POP(i1.i+i2.i, i1.i+i2.i) FROM integers i2) FROM integers i1 ORDER BY 1
-----
-NULL
-0.666667
-0.666667
-0.666667
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test_ignore
similarity index 96%
copy from modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test_ignore
index 5acec50..6f2fdad 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_aggregate_subquery.test_ignore
@@ -1,6 +1,10 @@
 # name: test/sql/subquery/scalar/test_correlated_aggregate_subquery.test
 # description: Test correlated aggregate subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15588
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15605
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15594
 
 statement ok
 PRAGMA enable_verification
@@ -11,24 +15,6 @@ CREATE TABLE integers(i INTEGER)
 statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
-# aggregate with correlation in final projection
-query II
-SELECT i, (SELECT MIN(i)+i1.i FROM integers) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	2
-2	3
-3	4
-
-# aggregate with correlation inside aggregation
-query II
-SELECT i, (SELECT MIN(i+2*i1.i) FROM integers) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	3
-2	5
-3	7
-
 query IRR
 SELECT i, SUM(i), (SELECT SUM(i)+SUM(i1.i) FROM integers) FROM integers i1 GROUP BY i ORDER BY i;
 ----
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test
index 0e2c6dd..f97b405 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test
@@ -29,33 +29,6 @@ NULL
 2
 1
 
-# subquery returning multiple results
-query II
-SELECT i, (SELECT 42+i1.i FROM integers) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	43
-2	44
-3	45
-
-# subquery with LIMIT
-query II
-SELECT i, (SELECT 42+i1.i FROM integers LIMIT 1) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	43
-2	44
-3	45
-
-# subquery with LIMIT 0
-query II
-SELECT i, (SELECT 42+i1.i FROM integers LIMIT 0) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	NULL
-2	NULL
-3	NULL
-
 # subquery with WHERE clause that is always FALSE
 query II
 SELECT i, (SELECT i FROM integers WHERE 1=0 AND i1.i=i) AS j FROM integers i1 ORDER BY i;
@@ -67,21 +40,21 @@ NULL	NULL
 
 # correlated EXISTS with WHERE clause that is always FALSE
 query IT
-SELECT i, EXISTS(SELECT i FROM integers WHERE 1=0 AND i1.i=i) AS j FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(SELECT i FROM integers WHERE 1=0 AND i1.i=i) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 # correlated ANY with WHERE clause that is always FALSE
 query IT
-SELECT i, i=ANY(SELECT i FROM integers WHERE 1=0 AND i1.i=i) AS j FROM integers i1 ORDER BY i;
+SELECT i, i=ANY(SELECT i FROM integers WHERE 1=0 AND i1.i=i) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 # subquery with OFFSET is not supported
 statement error
@@ -91,27 +64,9 @@ SELECT i, (SELECT i+i1.i FROM integers LIMIT 1 OFFSET 1) AS j FROM integers i1 O
 statement error
 SELECT i, (SELECT i+i1.i FROM integers ORDER BY 1 LIMIT 1 OFFSET 1) AS j FROM integers i1 ORDER BY i;
 
-# correlated filter without FROM clause
-query II
-SELECT i, (SELECT 42 WHERE i1.i>2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	NULL
-2	NULL
-3	42
-
-# correlated filter with matching entry on NULL
-query II
-SELECT i, (SELECT 42 WHERE i1.i IS NULL) AS j FROM integers i1 ORDER BY i;
-----
-NULL	42
-1	NULL
-2	NULL
-3	NULL
-
 # scalar select with correlation in projection
 query II
-SELECT i, (SELECT i+i1.i FROM integers WHERE i=1) AS j FROM integers i1 ORDER BY i;
+SELECT i, (SELECT i+i1.i FROM integers WHERE i=1) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	NULL
 1	2
@@ -120,7 +75,7 @@ NULL	NULL
 
 # scalar select with correlation in filter
 query II
-SELECT i, (SELECT i FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i;
+SELECT i, (SELECT i FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	NULL
 1	1
@@ -129,19 +84,9 @@ NULL	NULL
 
 # scalar select with operation in projection
 query II
-SELECT i, (SELECT i+1 FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i;
+SELECT i, (SELECT i+1 FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	NULL
 1	2
 2	3
 3	4
-
-# correlated scalar select with constant in projection
-query II
-SELECT i, (SELECT 42 FROM integers WHERE i=i1.i) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	42
-2	42
-3	42
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test_ignore
similarity index 95%
copy from modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test_ignore
index 0e2c6dd..a739426 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/subquery/scalar/test_correlated_subquery.test
 # description: Test correlated subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15604
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test
index af4cb03..722c3ad 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test
@@ -13,52 +13,17 @@ INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
 # scalar select with correlation
 query II
-SELECT i, (WITH i2 AS (SELECT 42+i1.i AS j) SELECT j FROM i2) AS j FROM integers i1 ORDER BY i;
+SELECT i, (WITH i2 AS (SELECT 42+i1.i AS j) SELECT j FROM i2) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	NULL
 1	43
 2	44
 3	45
 
-# ORDER BY correlated subquery
-query I
-SELECT i FROM integers i1 ORDER BY (WITH i2 AS (SELECT 100-i1.i as j) SELECT j FROM i2);
-----
-NULL
-3
-2
-1
-
-# subquery returning multiple results
-query II
-SELECT i, (WITH i2 AS (SELECT 42+i1.i AS j FROM integers) SELECT j FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	43
-2	44
-3	45
-
-# subquery with LIMIT
-query II
-SELECT i, (WITH i2 AS (SELECT 42+i1.i AS j FROM integers) SELECT j FROM i2 LIMIT 1) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	43
-2	44
-3	45
-
-# subquery with LIMIT 0
-query II
-SELECT i, (WITH i2 AS (SELECT 42+i1.i AS j FROM integers) SELECT j FROM i2 LIMIT 0) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	NULL
-2	NULL
-3	NULL
 
 # subquery with WHERE clause that is always FALSE
 query II
-SELECT i, (WITH i2 AS (SELECT i FROM integers WHERE 1=0 AND i1.i=i) SELECT i FROM i2) AS j FROM integers i1 ORDER BY i;
+SELECT i, (WITH i2 AS (SELECT i FROM integers WHERE 1=0 AND i1.i=i) SELECT i FROM i2) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	NULL
 1	NULL
@@ -67,81 +32,18 @@ NULL	NULL
 
 # correlated EXISTS with WHERE clause that is always FALSE
 query IT
-SELECT i, EXISTS(WITH i2 AS (SELECT i FROM integers WHERE 1=0 AND i1.i=i) SELECT i FROM i2) AS j FROM integers i1 ORDER BY i;
+SELECT i, EXISTS(WITH i2 AS (SELECT i FROM integers WHERE 1=0 AND i1.i=i) SELECT i FROM i2) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	0
-1	0
-2	0
-3	0
+NULL	false
+1	false
+2	false
+3	false
 
 # correlated ANY with WHERE clause that is always FALSE
 query IT
-SELECT i, i=ANY(WITH i2 AS (SELECT i FROM integers WHERE 1=0 AND i1.i=i) SELECT i FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	0
-1	0
-2	0
-3	0
-
-# subquery with OFFSET is not supported
-statement error
-SELECT i, (WITH i2 AS (SELECT i+i1.i FROM integers LIMIT 1 OFFSET 1) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-
-# subquery with ORDER BY is not supported
-statement error
-SELECT i, (WITH i2 AS (SELECT i+i1.i FROM integers ORDER BY 1 LIMIT 1 OFFSET 1) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-
-# correlated filter without FROM clause
-query II
-SELECT i, (WITH i2 AS (SELECT 42 WHERE i1.i>2) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
+SELECT i, i=ANY(WITH i2 AS (SELECT i FROM integers WHERE 1=0 AND i1.i=i) SELECT i FROM i2) AS j FROM integers i1 ORDER BY i NULLS FIRST;
 ----
-NULL	NULL
-1	NULL
-2	NULL
-3	42
-
-# correlated filter with matching entry on NULL
-query II
-SELECT i, (WITH i2 AS (SELECT 42 WHERE i1.i IS NULL) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	42
-1	NULL
-2	NULL
-3	NULL
-
-# scalar select with correlation in projection
-query II
-SELECT i, (WITH i2 AS (SELECT i+i1.i FROM integers WHERE i=1) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	2
-2	3
-3	4
-
-# scalar select with correlation in filter
-query II
-SELECT i, (WITH i2 AS (SELECT i FROM integers WHERE i=i1.i) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	1
-2	2
-3	3
-
-# scalar select with operation in projection
-query II
-SELECT i, (WITH i2 AS (SELECT i+1 FROM integers WHERE i=i1.i) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	2
-2	3
-3	4
-
-# correlated scalar select with constant in projection
-query II
-SELECT i, (WITH i2 AS (SELECT 42 FROM integers WHERE i=i1.i) SELECT * FROM i2) AS j FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	42
-2	42
-3	42
-
+NULL	false
+1	false
+2	false
+3	false
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test_ignore
similarity index 96%
copy from modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test_ignore
index af4cb03..57971c4 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_cte.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/subquery/scalar/test_correlated_subquery_cte.test
 # description: Test correlated subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15583
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_where.test b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_where.test
index 0f3bd90..88b7e6d 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_where.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_correlated_subquery_where.test
@@ -18,14 +18,14 @@ statement ok
 INSERT INTO test VALUES (2, 22)
 
 # correlated subquery with one correlated expression
-query II
+query II rowsort
 SELECT * FROM test WHERE b=(SELECT MIN(b) FROM test AS a WHERE a.id=test.id)
 ----
 1	21
 2	22
 
 # correlated subquery with two correlated expressions
-query II
+query II rowsort
 SELECT * FROM test WHERE b=(SELECT MIN(b) FROM test AS a WHERE a.id=test.id AND a.id < test.b)
 ----
 1	21
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test
index de69d1e..3e0b7024 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test
@@ -11,90 +11,10 @@ CREATE TABLE integers(i INTEGER)
 statement ok
 INSERT INTO integers VALUES (1), (2), (3), (NULL)
 
-# COUNT(*) and SUM(i) IS NULL aggregates
 query II
-SELECT i, (SELECT i FROM integers i2 WHERE i=(SELECT SUM(i) FROM integers i2 WHERE i2.i>i1.i)) FROM integers i1 ORDER BY 1;
-----
-NULL	NULL
-1	NULL
-2	3
-3	NULL
-
-query IT
-SELECT i, (SELECT SUM(i) IS NULL FROM integers i2 WHERE i2.i>i1.i) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	0
-2	0
-3	1
-
-query II
-SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i) FROM integers i1 ORDER BY i;
-----
-NULL	0
-1	2
-2	1
-3	0
-
-query II
-SELECT i, (SELECT COUNT(i) FROM integers i2 WHERE i2.i>i1.i OR i2.i IS NULL) FROM integers i1 ORDER BY i;
-----
-NULL	0
-1	2
-2	1
-3	0
-
-query II
-SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i OR i2.i IS NULL) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	3
-2	2
-3	1
-
-query II
-SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i OR (i1.i IS NULL AND i2.i IS NULL)) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	2
-2	1
-3	0
-
-query I
-SELECT i FROM integers i1 WHERE (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i)=0 ORDER BY i;
-----
-NULL
-3
-
-query II
-SELECT i, (SELECT i FROM integers i2 WHERE i-2=(SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i)) FROM integers i1 ORDER BY 1;
-----
-NULL	2
-1	NULL
-2	3
-3	2
-
-query II
-SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i GROUP BY i1.i) FROM integers i1 ORDER BY i;
-----
-NULL	NULL
-1	2
-2	1
-3	NULL
-
-query II
-SELECT i, (SELECT CASE WHEN (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i)=0 THEN 1 ELSE 0 END) FROM integers i1 ORDER BY i;
-----
-NULL	1
-1	0
-2	0
-3	1
-
-query II
-SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i) FROM integers i1 ORDER BY i;
+SELECT i, (SELECT COUNT(*) FROM integers i2 WHERE i2.i>i1.i) FROM integers i1 ORDER BY i NULLS FIRST;
 ----
 NULL	0
 1	2
 2	1
 3	0
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test_ignore
similarity index 94%
copy from modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test_ignore
index de69d1e..0a6fa57 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_count_star_subquery.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/subquery/scalar/test_count_star_subquery.test
 # description: Test for COUNT(*) and SUM(i) IS NULL in subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15586
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15605
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test
index e40bc77..f66ab64 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test
@@ -6,7 +6,7 @@ statement ok
 PRAGMA enable_verification
 
 statement ok
-CREATE TABLE tbl_ProductSales (ColID int, Product_Category  varchar(64), Product_Name  varchar(64), TotalSales int); 
+CREATE TABLE tbl_ProductSales (ColID int, Product_Category  varchar(64), Product_Name  varchar(64), TotalSales int);
 
 statement ok
 CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT);
@@ -17,119 +17,10 @@ INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Ga
 statement ok
 INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888);
 
-query T
-SELECT col1 IN (SELECT ColID FROM tbl_ProductSales) FROM another_T;
+query T rowsort
+SELECT col1 IN (SELECT ColID FROM tbl_ProductSales) FROM another_T
 ----
-1
-0
-0
-0
-
-query T
-SELECT col1 IN (SELECT ColID + col1 FROM tbl_ProductSales) FROM another_T;
-----
-0
-0
-0
-0
-
-query T
-SELECT col1 IN (SELECT ColID + col1 FROM tbl_ProductSales) FROM another_T GROUP BY col1;
-----
-0
-0
-0
-0
-
-query T
-SELECT col1 IN (SELECT ColID + another_T.col1 FROM tbl_ProductSales) FROM another_T GROUP BY col1;
-----
-0
-0
-0
-0
-
-query IT
-SELECT (col1 + 1) AS k, k IN (SELECT ColID + k FROM tbl_ProductSales) FROM another_T GROUP BY k ORDER BY 1;
-----
-2	0
-12	0
-112	0
-1112	0
-
-query T
-SELECT (col1 + 1) IN (SELECT ColID + (col1 + 1) FROM tbl_ProductSales) FROM another_T GROUP BY (col1 + 1);
-----
-0
-0
-0
-0
-
-# this should fail, col1 + 42 is not a grouping column
-statement error
-SELECT col1+1, col1+42 FROM another_T GROUP BY col1+1;
-
-# this should also fail, col1 + 42 is not a grouping column
-statement error
-SELECT (col1 + 1) IN (SELECT ColID + (col1 + 42) FROM tbl_ProductSales) FROM another_T GROUP BY (col1 + 1);
-
-# having without GROUP BY in subquery
-query T
-SELECT col5 = ALL (SELECT 1 FROM tbl_ProductSales HAVING MIN(col8) IS NULL) FROM another_T GROUP BY col1, col2, col5, col8;
-----
-1
-1
-1
-1
-
-query I
-SELECT CASE WHEN 1 IN (SELECT MAX(col7) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1;
-----
-NULL
-
-query I
-SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1;
-----
-NULL
-
-# UNION ALL with correlated subquery on either side
-query I
-SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1;
-----
-NULL
-
-query I
-SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1;
-----
-NULL
-
-# correlated column comparison with correlated subquery
-query I
-SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1;
-----
-1
-1
-1
-1
-
-query I
-SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> ANY(SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1;
-----
-1
-1
-1
-1
-
-# LEFT JOIN between correlated columns not supported for now
-statement error
-SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1;
-
-# REQUIRE(CHECK_COLUMN(result, 0, {1, 2, 2, 2}));
-# correlated columns in window functions not supported yet
-statement error
-SELECT EXISTS (SELECT RANK() OVER (PARTITION BY SUM(DISTINCT col5))) FROM another_T t1;
-
-# REQUIRE(CHECK_COLUMN(result, 0, {true}));
-statement error
-SELECT (SELECT SUM(col2) OVER (PARTITION BY SUM(col2) ORDER BY MAX(col1 + ColID) ROWS UNBOUNDED PRECEDING) FROM tbl_ProductSales) FROM another_T t1 GROUP BY col1
-
+true
+false
+false
+false
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test_ignore
similarity index 91%
copy from modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test_ignore
index e40bc77..6021511 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_grouped_correlated_subquery.test_ignore
@@ -1,12 +1,14 @@
 # name: test/sql/subquery/scalar/test_grouped_correlated_subquery.test
 # description: Test correlated subquery with grouping columns
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15607
 
 statement ok
 PRAGMA enable_verification
 
 statement ok
-CREATE TABLE tbl_ProductSales (ColID int, Product_Category  varchar(64), Product_Name  varchar(64), TotalSales int); 
+CREATE TABLE tbl_ProductSales (ColID int, Product_Category  varchar(64), Product_Name  varchar(64), TotalSales int);
 
 statement ok
 CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT);
@@ -17,53 +19,53 @@ INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Ga
 statement ok
 INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888);
 
-query T
-SELECT col1 IN (SELECT ColID FROM tbl_ProductSales) FROM another_T;
+query T rowsort
+SELECT col1 IN (SELECT ColID FROM tbl_ProductSales) FROM another_T
 ----
-1
-0
-0
-0
+true
+false
+false
+false
 
 query T
 SELECT col1 IN (SELECT ColID + col1 FROM tbl_ProductSales) FROM another_T;
 ----
-0
-0
-0
-0
+false
+false
+false
+false
 
 query T
 SELECT col1 IN (SELECT ColID + col1 FROM tbl_ProductSales) FROM another_T GROUP BY col1;
 ----
-0
-0
-0
-0
+false
+false
+false
+false
 
 query T
 SELECT col1 IN (SELECT ColID + another_T.col1 FROM tbl_ProductSales) FROM another_T GROUP BY col1;
 ----
-0
-0
-0
-0
+false
+false
+false
+false
 
 query IT
 SELECT (col1 + 1) AS k, k IN (SELECT ColID + k FROM tbl_ProductSales) FROM another_T GROUP BY k ORDER BY 1;
 ----
-2	0
-12	0
-112	0
-1112	0
+2	false
+12	false
+112	false
+1112	false
 
 query T
 SELECT (col1 + 1) IN (SELECT ColID + (col1 + 1) FROM tbl_ProductSales) FROM another_T GROUP BY (col1 + 1);
 ----
-0
-0
-0
-0
+false
+false
+false
+false
 
 # this should fail, col1 + 42 is not a grouping column
 statement error
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_join_in_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_join_in_subquery.test
index 4414299..9694fe4 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_join_in_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_join_in_subquery.test
@@ -26,7 +26,7 @@ INSERT INTO test2 VALUES (1, 44)
 statement ok
 INSERT INTO test2 VALUES (2, 42)
 
-query IIII
+query IIII rowsort
 SELECT * FROM test, test2 WHERE test.id=test2.id AND test_value*test2_value=(SELECT MIN(test_value*test2_value) FROM test AS a, test2 WHERE a.id=test.id AND a.id=test2.id)
 ----
 1	21	1	44
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_many_correlated_columns.test b/modules/calcite/src/test/sql/subquery/scalar/test_many_correlated_columns.test_ignore
similarity index 100%
rename from modules/calcite/src/test/sql/subquery/scalar/test_many_correlated_columns.test
rename to modules/calcite/src/test/sql/subquery/scalar/test_many_correlated_columns.test_ignore
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_nested_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_nested_correlated_subquery.test_ignore
similarity index 100%
rename from modules/calcite/src/test/sql/subquery/scalar/test_nested_correlated_subquery.test
rename to modules/calcite/src/test/sql/subquery/scalar/test_nested_correlated_subquery.test_ignore
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test
index cea32ed..07d9a0a 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test
@@ -13,12 +13,12 @@ SELECT 1+(SELECT 1)
 query T
 SELECT 1=(SELECT 1)
 ----
-1
+true
 
 query T
 SELECT 1<>(SELECT 1)
 ----
-0
+false
 
 query T
 SELECT 1=(SELECT NULL)
@@ -48,12 +48,6 @@ SELECT * FROM (SELECT 42) v1(a);
 ----
 42
 
-# not enough aliases: defaults to using names for missing columns
-query II
-SELECT * FROM (SELECT 42, 41 AS x) v1(a);
-----
-42	41
-
 # too many aliases: fails
 statement error
 SELECT * FROM (SELECT 42, 41 AS x) v1(a, b, c);
@@ -70,14 +64,8 @@ INSERT INTO test VALUES (12, 21)
 statement ok
 INSERT INTO test VALUES (13, 22)
 
-# select single tuple only in scalar subquery
-query I
-SELECT (SELECT a * 42 FROM test)
-----
-462
-
 # operations on subquery
-query I
+query I rowsort
 SELECT a*(SELECT 42) FROM test
 ----
 462
@@ -93,20 +81,20 @@ INSERT INTO t1(e,c,b,d,a) VALUES(103,102,100,101,104)
 statement ok
 INSERT INTO t1(a,c,d,e,b) VALUES(107,106,108,109,105)
 
-query R
+query R rowsort
 SELECT c-(SELECT sum(c) FROM t1) FROM t1
 ----
 -106.000000
 -102.000000
 
-query I
+query I rowsort
 SELECT CASE WHEN c>(SELECT sum(c)/count(*) FROM t1) THEN a*2 ELSE b*10 END FROM t1
 ----
 1000
 214
 
 # correlated subqueries
-query IR
+query IR rowsort
 SELECT a, (SELECT SUM(b) FROM test tsub WHERE test.a=tsub.a) FROM test
 ----
 11	22.000000
@@ -119,35 +107,3 @@ SELECT a, (SELECT CASE WHEN test.a=11 THEN 22 ELSE NULL END) FROM test ORDER BY
 11	22
 12	NULL
 13	NULL
-
-query II
-SELECT a, (SELECT CASE WHEN test.a=11 THEN b ELSE NULL END FROM test tsub) FROM test ORDER BY a
-----
-11	22
-12	NULL
-13	NULL
-
-query II
-SELECT a, (SELECT CASE WHEN test.a=11 THEN b ELSE NULL END FROM test tsub LIMIT 1) FROM test ORDER BY a
-----
-11	22
-12	NULL
-13	NULL
-
-query II
-SELECT * from test where a=(SELECT MIN(a) FROM test t WHERE t.b=test.b)
-----
-11	22
-12	21
-
-# exists / in / any subqueries
-query II
-SELECT * FROM test WHERE EXISTS (SELECT a FROM test ts WHERE ts.a = test.a AND b>21)
-----
-11	22
-13	22
-
-# duplicate name in subquery
-statement error
-SELECT * FROM (SELECT 42 AS a, 44 AS a) tbl1
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test_ignore
similarity index 89%
copy from modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test_ignore
index cea32ed..044aa54 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/scalar/test_scalar_subquery.test
 # description: Test subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
 
 statement ok
 PRAGMA enable_verification
@@ -13,12 +14,12 @@ SELECT 1+(SELECT 1)
 query T
 SELECT 1=(SELECT 1)
 ----
-1
+true
 
 query T
 SELECT 1<>(SELECT 1)
 ----
-0
+false
 
 query T
 SELECT 1=(SELECT NULL)
@@ -48,12 +49,6 @@ SELECT * FROM (SELECT 42) v1(a);
 ----
 42
 
-# not enough aliases: defaults to using names for missing columns
-query II
-SELECT * FROM (SELECT 42, 41 AS x) v1(a);
-----
-42	41
-
 # too many aliases: fails
 statement error
 SELECT * FROM (SELECT 42, 41 AS x) v1(a, b, c);
@@ -70,14 +65,8 @@ INSERT INTO test VALUES (12, 21)
 statement ok
 INSERT INTO test VALUES (13, 22)
 
-# select single tuple only in scalar subquery
-query I
-SELECT (SELECT a * 42 FROM test)
-----
-462
-
 # operations on subquery
-query I
+query I rowsort
 SELECT a*(SELECT 42) FROM test
 ----
 462
@@ -93,20 +82,20 @@ INSERT INTO t1(e,c,b,d,a) VALUES(103,102,100,101,104)
 statement ok
 INSERT INTO t1(a,c,d,e,b) VALUES(107,106,108,109,105)
 
-query R
+query R rowsort
 SELECT c-(SELECT sum(c) FROM t1) FROM t1
 ----
 -106.000000
 -102.000000
 
-query I
+query I rowsort
 SELECT CASE WHEN c>(SELECT sum(c)/count(*) FROM t1) THEN a*2 ELSE b*10 END FROM t1
 ----
 1000
 214
 
 # correlated subqueries
-query IR
+query IR rowsort
 SELECT a, (SELECT SUM(b) FROM test tsub WHERE test.a=tsub.a) FROM test
 ----
 11	22.000000
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test
index 09d25f8..1598ae3 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test
@@ -13,12 +13,12 @@ SELECT 1+(WITH cte AS (SELECT 1) SELECT * FROM cte)
 query T
 SELECT 1=(WITH cte AS (SELECT 1) SELECT * FROM cte)
 ----
-1
+true
 
 query T
 SELECT 1<>(WITH cte AS (SELECT 1) SELECT * FROM cte)
 ----
-0
+false
 
 query T
 SELECT 1=(WITH cte AS (SELECT NULL) SELECT * FROM cte)
@@ -48,12 +48,6 @@ SELECT * FROM (WITH cte(x) AS (SELECT 42) SELECT x FROM cte) v1(a);
 ----
 42
 
-# not enough aliases: defaults to using names for missing columns
-query II
-SELECT * FROM (WITH cte AS (SELECT 42, 41 AS x) SELECT * FROM cte) v1(a);
-----
-42	41
-
 # too many aliases: fails
 statement error
 SELECT * FROM (WITH cte AS (SELECT 42, 41 AS x) SELECT * FROM cte) v1(a, b, c);
@@ -70,14 +64,8 @@ INSERT INTO test VALUES (12, 21)
 statement ok
 INSERT INTO test VALUES (13, 22)
 
-# select single tuple only in scalar subquery
-query I
-SELECT (WITH cte AS (SELECT a * 42 FROM test) SELECT * FROM cte)
-----
-462
-
 # operations on subquery
-query I
+query I rowsort
 SELECT a*(WITH cte AS (SELECT 42) SELECT * FROM cte) FROM test
 ----
 462
@@ -93,20 +81,20 @@ INSERT INTO t1(e,c,b,d,a) VALUES(103,102,100,101,104)
 statement ok
 INSERT INTO t1(a,c,d,e,b) VALUES(107,106,108,109,105)
 
-query R
+query R rowsort
 SELECT c-(WITH cte AS (SELECT sum(c) FROM t1) SELECT * FROM cte) FROM t1
 ----
 -106.000000
 -102.000000
 
-query I
+query I rowsort
 SELECT CASE WHEN c>(WITH cte AS (SELECT sum(c)/count(*) FROM t1) SELECT * FROM cte) THEN a*2 ELSE b*10 END FROM t1
 ----
 1000
 214
 
 # correlated subqueries
-query IR
+query IR rowsort
 SELECT a, (WITH cte AS (SELECT SUM(b) FROM test tsub WHERE test.a=tsub.a) SELECT * FROM cte) FROM test
 ----
 11	22.000000
@@ -119,35 +107,3 @@ SELECT a, (WITH cte AS (SELECT CASE WHEN test.a=11 THEN 22 ELSE NULL END) SELECT
 11	22
 12	NULL
 13	NULL
-
-query II
-SELECT a, (WITH cte AS (SELECT CASE WHEN test.a=11 THEN b ELSE NULL END FROM test tsub) SELECT * FROM cte) FROM test ORDER BY a
-----
-11	22
-12	NULL
-13	NULL
-
-query II
-SELECT a, (WITH cte AS (SELECT CASE WHEN test.a=11 THEN b ELSE NULL END FROM test tsub LIMIT 1) SELECT * FROM cte) FROM test ORDER BY a
-----
-11	22
-12	NULL
-13	NULL
-
-query II
-SELECT * from test where a=(WITH cte AS (SELECT a FROM test t WHERE t.b=test.b) SELECT min(a) FROM cte)
-----
-11	22
-12	21
-
-# exists / in / any subqueries
-query II
-SELECT * FROM test WHERE EXISTS (WITH cte AS (SELECT * FROM test ts WHERE ts.a = test.a AND b>21) SELECT a FROM cte)
-----
-11	22
-13	22
-
-# duplicate name in subquery
-statement error
-SELECT * FROM (WITH cte AS (SELECT 42 AS a, 44 AS a) SELECT * FROM cte) tbl1
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test_ignore
similarity index 94%
copy from modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test_ignore
index 09d25f8..9c5ed16 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_scalar_subquery_cte.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/scalar/test_scalar_subquery_cte.test
 # description: Test subqueries with CTEs
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15603
 
 statement ok
 PRAGMA enable_verification
@@ -13,12 +14,12 @@ SELECT 1+(WITH cte AS (SELECT 1) SELECT * FROM cte)
 query T
 SELECT 1=(WITH cte AS (SELECT 1) SELECT * FROM cte)
 ----
-1
+true
 
 query T
 SELECT 1<>(WITH cte AS (SELECT 1) SELECT * FROM cte)
 ----
-0
+false
 
 query T
 SELECT 1=(WITH cte AS (SELECT NULL) SELECT * FROM cte)
@@ -70,14 +71,8 @@ INSERT INTO test VALUES (12, 21)
 statement ok
 INSERT INTO test VALUES (13, 22)
 
-# select single tuple only in scalar subquery
-query I
-SELECT (WITH cte AS (SELECT a * 42 FROM test) SELECT * FROM cte)
-----
-462
-
 # operations on subquery
-query I
+query I rowsort
 SELECT a*(WITH cte AS (SELECT 42) SELECT * FROM cte) FROM test
 ----
 462
@@ -93,20 +88,20 @@ INSERT INTO t1(e,c,b,d,a) VALUES(103,102,100,101,104)
 statement ok
 INSERT INTO t1(a,c,d,e,b) VALUES(107,106,108,109,105)
 
-query R
+query R rowsort
 SELECT c-(WITH cte AS (SELECT sum(c) FROM t1) SELECT * FROM cte) FROM t1
 ----
 -106.000000
 -102.000000
 
-query I
+query I rowsort
 SELECT CASE WHEN c>(WITH cte AS (SELECT sum(c)/count(*) FROM t1) SELECT * FROM cte) THEN a*2 ELSE b*10 END FROM t1
 ----
 1000
 214
 
 # correlated subqueries
-query IR
+query IR rowsort
 SELECT a, (WITH cte AS (SELECT SUM(b) FROM test tsub WHERE test.a=tsub.a) SELECT * FROM cte) FROM test
 ----
 11	22.000000
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test
index a3b2a9c..f4fa943 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test
@@ -3,16 +3,9 @@
 # group: [scalar]
 
 statement ok
-PRAGMA enable_verification
-
-statement ok
 CREATE TABLE item(i_manufact INTEGER)
 
 query I
 SELECT * FROM item i1 WHERE (SELECT count(*) AS item_cnt FROM item WHERE (i_manufact = i1.i_manufact AND i_manufact=3) OR (i_manufact = i1.i_manufact AND i_manufact=3)) > 0 ORDER BY 1 LIMIT 100;
 ----
 
-query I
-SELECT * FROM item i1 WHERE (SELECT count(*) AS item_cnt FROM item WHERE (i_manufact = i1.i_manufact AND i_manufact=3) OR (i_manufact = i1.i_manufact AND i_manufact=3)) ORDER BY 1 LIMIT 100;
-----
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test_ignore
similarity index 91%
copy from modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test_ignore
index a3b2a9c..0e96ef3 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_tpcds_correlated_subquery.test_ignore
@@ -1,9 +1,7 @@
 # name: test/sql/subquery/scalar/test_tpcds_correlated_subquery.test
 # description: Test correlated subqueries based on TPC-DS
 # group: [scalar]
-
-statement ok
-PRAGMA enable_verification
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15609
 
 statement ok
 CREATE TABLE item(i_manufact INTEGER)
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test
index 7371582..7b7e58e 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test
@@ -39,89 +39,3 @@ NULL	3
 1	3
 2	3
 3	3
-
-# group by on subquery
-query II
-SELECT (SELECT 42) AS k, MAX(i) FROM integers GROUP BY k
-----
-42	3
-
-# subquery as parameter to aggregate
-query II
-SELECT i, MAX((SELECT 42)) FROM integers GROUP BY i ORDER BY i
-----
-NULL	42
-1	42
-2	42
-3	42
-
-# scalar subquery returning zero results should result in NULL
-query I
-SELECT (SELECT * FROM integers WHERE i>10) FROM integers
-----
-NULL
-NULL
-NULL
-NULL
-
-# return more than one row in a scalar subquery
-# controversial: in postgres this gives an error
-# but SQLite accepts it and just uses the first value
-# we choose to agree with SQLite here
-query I
-SELECT * FROM integers WHERE i=(SELECT i FROM integers WHERE i IS NOT NULL ORDER BY i)
-----
-1
-
-# i.e. the above query is equivalent to this query
-query I
-SELECT * FROM integers WHERE i=(SELECT i FROM integers WHERE i IS NOT NULL ORDER BY i LIMIT 1)
-----
-1
-
-# returning multiple columns should fail though
-statement error
-SELECT * FROM integers WHERE i=(SELECT 1, 2)
-
-statement error
-SELECT * FROM integers WHERE i=(SELECT i, i + 2 FROM integers)
-
-# but not for EXISTS queries!
-query I
-SELECT * FROM integers WHERE EXISTS (SELECT 1, 2)
-----
-1
-2
-3
-NULL
-
-query I
-SELECT * FROM integers WHERE EXISTS (SELECT i, i + 2 FROM integers)
-----
-1
-2
-3
-NULL
-
-# SELECT * should be fine if the star only expands to a single column
-query I
-SELECT (SELECT * FROM integers WHERE i=1)
-----
-1
-
-# but not if the star expands to more than one column!
-statement error
-SELECT (SELECT * FROM integers i1, integers i2)
-
-#  uncorrelated subquery in SELECT
-query I
-SELECT (SELECT i FROM integers WHERE i=1)
-----
-1
-
-query I
-SELECT * FROM integers WHERE i > (SELECT i FROM integers WHERE i=1)
-----
-2
-3
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test_ignore
similarity index 97%
copy from modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test_ignore
index 7371582..8e4c6f9 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/scalar/test_uncorrelated_scalar_subquery.test
 # description: Test uncorrelated subqueries
 # group: [scalar]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15583
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_varchar_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_varchar_subquery.test
index 8287f29..5245db8 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_varchar_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_uncorrelated_varchar_subquery.test
@@ -27,7 +27,7 @@ NULL
 query T
 SELECT 'hello' IN (SELECT * FROM strings)
 ----
-1
+true
 
 query T
 SELECT 'bla' IN (SELECT * FROM strings)
@@ -37,10 +37,10 @@ NULL
 query T
 SELECT 'bla' IN (SELECT * FROM strings WHERE v IS NOT NULL)
 ----
-0
+false
 
 # EXISTS
-query T
+query T rowsort
 SELECT * FROM strings WHERE EXISTS(SELECT NULL)
 ----
 hello
@@ -65,4 +65,3 @@ SELECT (SELECT v FROM strings WHERE v='bla') FROM strings
 NULL
 NULL
 NULL
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test
index 5b5597e..4cdf18f 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test
@@ -59,15 +59,3 @@ SELECT id, i FROM integers ORDER BY id
 3	2
 4	3
 
-# use DEFAULT with correlated subquery in WHERE
-statement ok
-UPDATE integers i1 SET i=DEFAULT WHERE i=(SELECT MIN(i) FROM integers WHERE i1.id<id);
-
-query II
-SELECT id, i FROM integers ORDER BY id
-----
-1	NULL
-2	NULL
-3	2
-4	3
-
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test_ignore
similarity index 100%
copy from modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test
copy to modules/calcite/src/test/sql/subquery/scalar/test_update_subquery.test_ignore
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_varchar_correlated_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_varchar_correlated_subquery.test_ignore
similarity index 100%
rename from modules/calcite/src/test/sql/subquery/scalar/test_varchar_correlated_subquery.test
rename to modules/calcite/src/test/sql/subquery/scalar/test_varchar_correlated_subquery.test_ignore
diff --git a/modules/calcite/src/test/sql/subquery/scalar/test_window_function_subquery.test b/modules/calcite/src/test/sql/subquery/scalar/test_window_function_subquery.test_ignore
similarity index 95%
rename from modules/calcite/src/test/sql/subquery/scalar/test_window_function_subquery.test
rename to modules/calcite/src/test/sql/subquery/scalar/test_window_function_subquery.test_ignore
index 38a6185..a9b99f2 100644
--- a/modules/calcite/src/test/sql/subquery/scalar/test_window_function_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/scalar/test_window_function_subquery.test_ignore
@@ -1,8 +1,7 @@
 # name: test/sql/subquery/scalar/test_window_function_subquery.test
 # description: Test window functions in correlated subqueries
 # group: [scalar]
-
-require vector_size 512
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14777
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/table/test_aliasing.test b/modules/calcite/src/test/sql/subquery/table/test_aliasing.test_ignore
similarity index 87%
rename from modules/calcite/src/test/sql/subquery/table/test_aliasing.test
rename to modules/calcite/src/test/sql/subquery/table/test_aliasing.test_ignore
index 6be7087..4fd7f19 100644
--- a/modules/calcite/src/test/sql/subquery/table/test_aliasing.test
+++ b/modules/calcite/src/test/sql/subquery/table/test_aliasing.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/table/test_aliasing.test
 # description: Aliasing and aggregation in subqueries
 # group: [table]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15583
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/subquery/table/test_nested_table_subquery.test b/modules/calcite/src/test/sql/subquery/table/test_nested_table_subquery.test
index dee7cd4..11c18a1 100644
--- a/modules/calcite/src/test/sql/subquery/table/test_nested_table_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/table/test_nested_table_subquery.test
@@ -18,8 +18,8 @@ SELECT * FROM (SELECT i, j FROM (SELECT j AS i, i AS j FROM (SELECT j AS i, i AS
 5	6	5	5	4	5
 
 # 100 nesting levels
-query I
-SELECT i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FRO [...]
+query I rowsort
+SELECT i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FROM (SELECT i + 1 AS i FRO [...]
 ----
 103
 104
diff --git a/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test b/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test
index f5a2c70..e8b7c9e 100644
--- a/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test
@@ -3,24 +3,21 @@
 # group: [table]
 
 statement ok
-PRAGMA enable_verification
-
-statement ok
 CREATE TABLE test (i INTEGER, j INTEGER)
 
 statement ok
 INSERT INTO test VALUES (3, 4), (4, 5), (5, 6);
 
-query II
-SELECT * FROM (SELECT i, j AS d FROM test ORDER BY i) AS b;
+query II rowsort
+SELECT * FROM (SELECT i, j AS d FROM test ORDER BY i) AS b
 ----
 3	4
 4	5
 5	6
 
 # check column names for simple projections and aliases
-query I
-SELECT b.d FROM (SELECT i * 2 + j AS d FROM test) AS b;
+query I rowsort
+SELECT b.d FROM (SELECT i * 2 + j AS d FROM test) AS b
 ----
 10
 13
@@ -45,10 +42,3 @@ query R
 select sum(x) from (select i as x from test group by i) sq;
 ----
 12.000000
-
-# subquery group aliases are visible
-query R
-select sum(x) from (select i+1 as x from test group by x) sq;
-----
-15.000000
-
diff --git a/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test b/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test_ignore
similarity index 82%
copy from modules/calcite/src/test/sql/subquery/table/test_table_subquery.test
copy to modules/calcite/src/test/sql/subquery/table/test_table_subquery.test_ignore
index f5a2c70..ca608c6 100644
--- a/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test
+++ b/modules/calcite/src/test/sql/subquery/table/test_table_subquery.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/subquery/table/test_table_subquery.test
 # description: Table subquery
 # group: [table]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15583
 
 statement ok
 PRAGMA enable_verification
@@ -11,16 +12,16 @@ CREATE TABLE test (i INTEGER, j INTEGER)
 statement ok
 INSERT INTO test VALUES (3, 4), (4, 5), (5, 6);
 
-query II
-SELECT * FROM (SELECT i, j AS d FROM test ORDER BY i) AS b;
+query II rowsort
+SELECT * FROM (SELECT i, j AS d FROM test ORDER BY i) AS b
 ----
 3	4
 4	5
 5	6
 
 # check column names for simple projections and aliases
-query I
-SELECT b.d FROM (SELECT i * 2 + j AS d FROM test) AS b;
+query I rowsort
+SELECT b.d FROM (SELECT i * 2 + j AS d FROM test) AS b
 ----
 10
 13
diff --git a/modules/calcite/src/test/sql/types/blob/test_blob.test b/modules/calcite/src/test/sql/types/blob/test_blob.test_ignore
similarity index 92%
rename from modules/calcite/src/test/sql/types/blob/test_blob.test
rename to modules/calcite/src/test/sql/types/blob/test_blob.test_ignore
index 369944a..d0769d8 100644
--- a/modules/calcite/src/test/sql/types/blob/test_blob.test
+++ b/modules/calcite/src/test/sql/types/blob/test_blob.test_ignore
@@ -1,12 +1,13 @@
 # name: test/sql/types/blob/test_blob.test
 # description: BLOB tests
 # group: [blob]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15618
 
 statement ok
 PRAGMA enable_verification
 
 statement ok
-CREATE TABLE blobs (b BYTEA);
+CREATE TABLE blobs (b binary);
 
 # Insert valid hex strings
 statement ok
@@ -77,7 +78,7 @@ SELECT NULL::BLOB
 NULL
 
 statement ok
-CREATE TABLE blob_empty (b BYTEA);
+CREATE TABLE blob_empty (b binary);
 
 statement ok
 INSERT INTO blob_empty VALUES(''), (''::BLOB)
diff --git a/modules/calcite/src/test/sql/types/blob/test_blob_cast.test b/modules/calcite/src/test/sql/types/blob/test_blob_cast.test_ignore
similarity index 88%
rename from modules/calcite/src/test/sql/types/blob/test_blob_cast.test
rename to modules/calcite/src/test/sql/types/blob/test_blob_cast.test_ignore
index 74e753f..03eaf09 100644
--- a/modules/calcite/src/test/sql/types/blob/test_blob_cast.test
+++ b/modules/calcite/src/test/sql/types/blob/test_blob_cast.test_ignore
@@ -1,25 +1,26 @@
 # name: test/sql/types/blob/test_blob_cast.test
 # description: Cast BLOB values
 # group: [blob]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15618
 
 statement ok
 PRAGMA enable_verification
 
 # BLOB to VARCHAR -> CastFromBlob, it always results in a hex representation
 query T
-SELECT 'a'::BYTEA::VARCHAR
+SELECT 'a'::binary::VARCHAR
 ----
 a
 
 # VARCHAR to BLOB -> CastToBlob
 query T
-SELECT 'a'::VARCHAR::BYTEA
+SELECT 'a'::VARCHAR::binary
 ----
 a
 
 # Hex string with BLOB
 query T
-SELECT '\x20\x00\xFF'::BYTEA
+SELECT '\x20\x00\xFF'::binary
 ----
  \x00\xFF
 
diff --git a/modules/calcite/src/test/sql/types/blob/test_blob_function.test b/modules/calcite/src/test/sql/types/blob/test_blob_function.test_ignore
similarity index 59%
rename from modules/calcite/src/test/sql/types/blob/test_blob_function.test
rename to modules/calcite/src/test/sql/types/blob/test_blob_function.test_ignore
index 2bf8c8e..5040b6d 100644
--- a/modules/calcite/src/test/sql/types/blob/test_blob_function.test
+++ b/modules/calcite/src/test/sql/types/blob/test_blob_function.test_ignore
@@ -1,29 +1,30 @@
 # name: test/sql/types/blob/test_blob_function.test
 # description: BLOB with Functions
 # group: [blob]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15618
 
 statement ok
 PRAGMA enable_verification
 
 statement ok
-CREATE TABLE blobs (b BYTEA);
+CREATE TABLE blobs (b binary);
 
 statement ok
-INSERT INTO blobs VALUES ('a'::BYTEA)
+INSERT INTO blobs VALUES ('a'::binary)
 
 # conventional concat
 query T
-SELECT b || 'ZZ'::BYTEA FROM blobs
+SELECT b || 'ZZ'::binary FROM blobs
 ----
 aZZ
 
 query T
-SELECT 'abc '::BYTEA || 'klm *\xEF\xBF\xBDT'::BYTEA
+SELECT 'abc '::BYTEA || 'klm *\xEF\xBF\xBDT'::binary
 ----
 abc klm *\xEF\xBF\xBDT
 
 statement ok
-INSERT INTO blobs VALUES ('abc klm *\xEF\xBF\xBDT'::BYTEA)
+INSERT INTO blobs VALUES ('abc klm *\xEF\xBF\xBDT'::binary)
 
 query I
 SELECT COUNT(*) FROM blobs
@@ -42,15 +43,15 @@ statement ok
 DELETE FROM blobs
 
 statement ok
-INSERT INTO blobs VALUES ('\xFF'::BYTEA)
+INSERT INTO blobs VALUES ('\xFF'::binary)
 
 query T
-SELECT b || 'ZZ'::BYTEA FROM blobs
+SELECT b || 'ZZ'::binary FROM blobs
 ----
 \xFFZZ
 
 query T
-SELECT b || '\x5A\x5A'::BYTEA FROM blobs
+SELECT b || '\x5A\x5A'::binary FROM blobs
 ----
 \xFFZZ
 
@@ -63,16 +64,16 @@ statement ok
 DELETE FROM blobs
 
 statement ok
-INSERT INTO blobs VALUES ('\xFF'::BYTEA)
+INSERT INTO blobs VALUES ('\xFF'::binary)
 
 statement ok
-INSERT INTO blobs VALUES ('FF'::BYTEA)
+INSERT INTO blobs VALUES ('FF'::binary)
 
 statement ok
-INSERT INTO blobs VALUES ('\x55\xAA\xFF\x55\xAA\xFF\x55\xAA\xFF\x01'::BYTEA)
+INSERT INTO blobs VALUES ('\x55\xAA\xFF\x55\xAA\xFF\x55\xAA\xFF\x01'::binary)
 
 statement ok
-INSERT INTO blobs VALUES ('55AAFF55AAFF55AAFF01'::BYTEA)
+INSERT INTO blobs VALUES ('55AAFF55AAFF55AAFF01'::binary)
 
 query I
 SELECT OCTET_LENGTH(b) FROM blobs
diff --git a/modules/calcite/src/test/sql/types/blob/test_blob_operator.test b/modules/calcite/src/test/sql/types/blob/test_blob_operator.test_ignore
similarity index 93%
rename from modules/calcite/src/test/sql/types/blob/test_blob_operator.test
rename to modules/calcite/src/test/sql/types/blob/test_blob_operator.test_ignore
index 34b6a12..0473de7 100644
--- a/modules/calcite/src/test/sql/types/blob/test_blob_operator.test
+++ b/modules/calcite/src/test/sql/types/blob/test_blob_operator.test_ignore
@@ -1,12 +1,13 @@
 # name: test/sql/types/blob/test_blob_operator.test
 # description: Test BLOBs with various SQL operators
 # group: [blob]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15618
 
 statement ok
 PRAGMA enable_verification
 
 statement ok
-CREATE TABLE blobs (b BYTEA, g INTEGER);
+CREATE TABLE blobs (b binary, g INTEGER);
 
 # strings: hello -> \x68656C6C6F, r -> \x72
 statement ok
diff --git a/modules/calcite/src/test/sql/types/blob/test_blob_string.test b/modules/calcite/src/test/sql/types/blob/test_blob_string.test_ignore
similarity index 92%
rename from modules/calcite/src/test/sql/types/blob/test_blob_string.test
rename to modules/calcite/src/test/sql/types/blob/test_blob_string.test_ignore
index 5b82083..0214477 100644
--- a/modules/calcite/src/test/sql/types/blob/test_blob_string.test
+++ b/modules/calcite/src/test/sql/types/blob/test_blob_string.test_ignore
@@ -1,12 +1,13 @@
 # name: test/sql/types/blob/test_blob_string.test
 # description: Insert BLOB values from normal strings
 # group: [blob]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15618
 
 statement ok
 PRAGMA enable_verification
 
 statement ok
-CREATE TABLE blobs (b BYTEA);
+CREATE TABLE blobs (b binary);
 
 # insert BLOB from string
 statement ok
diff --git a/modules/calcite/src/test/sql/types/date/date_parsing.test b/modules/calcite/src/test/sql/types/date/date_parsing.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/types/date/date_parsing.test
rename to modules/calcite/src/test/sql/types/date/date_parsing.test_ignore
index d8f8fda..6f45c20 100644
--- a/modules/calcite/src/test/sql/types/date/date_parsing.test
+++ b/modules/calcite/src/test/sql/types/date/date_parsing.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/date/date_parsing.test
 # description: Test date parsing
 # group: [date]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15619
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/date/test_bc_dates.test b/modules/calcite/src/test/sql/types/date/test_bc_dates.test_ignore
similarity index 85%
rename from modules/calcite/src/test/sql/types/date/test_bc_dates.test
rename to modules/calcite/src/test/sql/types/date/test_bc_dates.test_ignore
index 85331fc..9df007a 100644
--- a/modules/calcite/src/test/sql/types/date/test_bc_dates.test
+++ b/modules/calcite/src/test/sql/types/date/test_bc_dates.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/date/test_bc_dates.test
 # description: Test BC dates
 # group: [date]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15619
 
 # BC dates are off-by-one because the year 0 does not exist in the gregorian calender
 # however, ISO 8601 DOES have the year 0, thus "0000-01-01" is actually equal to "0001-01-01 (BC)"
@@ -13,30 +14,6 @@ PRAGMA enable_verification
 statement ok
 CREATE TABLE dates(i DATE)
 
-# insert an ISO-8601 formatted negative date
-statement ok
-INSERT INTO dates VALUES ('-1993-08-14'), (NULL)
-
-# the BC formatted variant is one off
-query T
-SELECT * FROM dates
-----
-1994-08-14 (BC)
-NULL
-
-# YEAR function returns the ISO 8601 year
-query I
-SELECT year(i) FROM dates
-----
--1993
-NULL
-
-query T
-SELECT cast(i AS VARCHAR) FROM dates
-----
-1994-08-14 (BC)
-NULL
-
 # the year 0 exists in the ISO-8601 format
 query T
 SELECT DATE '0000-01-01'
diff --git a/modules/calcite/src/test/sql/types/date/test_date.test b/modules/calcite/src/test/sql/types/date/test_date.test_ignore
similarity index 94%
rename from modules/calcite/src/test/sql/types/date/test_date.test
rename to modules/calcite/src/test/sql/types/date/test_date.test_ignore
index 9de860d..bf207cf 100644
--- a/modules/calcite/src/test/sql/types/date/test_date.test
+++ b/modules/calcite/src/test/sql/types/date/test_date.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/date/test_date.test
 # description: Test basic DATE functionality
 # group: [date]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15619
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/interval/interval_constants.test b/modules/calcite/src/test/sql/types/interval/interval_constants.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/types/interval/interval_constants.test
rename to modules/calcite/src/test/sql/types/interval/interval_constants.test_ignore
index 69b3836..bc00456 100644
--- a/modules/calcite/src/test/sql/types/interval/interval_constants.test
+++ b/modules/calcite/src/test/sql/types/interval/interval_constants.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/interval/interval_constants.test
 # description: Test constant intervals
 # group: [interval]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14975
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/interval/test_interval.test b/modules/calcite/src/test/sql/types/interval/test_interval.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/types/interval/test_interval.test
rename to modules/calcite/src/test/sql/types/interval/test_interval.test_ignore
index 5887488..2abfe62 100644
--- a/modules/calcite/src/test/sql/types/interval/test_interval.test
+++ b/modules/calcite/src/test/sql/types/interval/test_interval.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/interval/test_interval.test
 # description: Test basic interval usage
 # group: [interval]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14975
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/interval/test_interval_addition.test b/modules/calcite/src/test/sql/types/interval/test_interval_addition.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/types/interval/test_interval_addition.test
rename to modules/calcite/src/test/sql/types/interval/test_interval_addition.test_ignore
index 488ba1f..0abf59f 100644
--- a/modules/calcite/src/test/sql/types/interval/test_interval_addition.test
+++ b/modules/calcite/src/test/sql/types/interval/test_interval_addition.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/interval/test_interval_addition.test
 # description: Test interval addition/subtraction
 # group: [interval]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14975
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/interval/test_interval_comparison.test b/modules/calcite/src/test/sql/types/interval/test_interval_comparison.test_ignore
similarity index 92%
rename from modules/calcite/src/test/sql/types/interval/test_interval_comparison.test
rename to modules/calcite/src/test/sql/types/interval/test_interval_comparison.test_ignore
index d022f20..10459e3 100644
--- a/modules/calcite/src/test/sql/types/interval/test_interval_comparison.test
+++ b/modules/calcite/src/test/sql/types/interval/test_interval_comparison.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/interval/test_interval_comparison.test
 # description: Test interval comparisons
 # group: [interval]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14975
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/interval/test_interval_ops.test b/modules/calcite/src/test/sql/types/interval/test_interval_ops.test_ignore
similarity index 94%
rename from modules/calcite/src/test/sql/types/interval/test_interval_ops.test
rename to modules/calcite/src/test/sql/types/interval/test_interval_ops.test_ignore
index b07ba75..8f86e68 100644
--- a/modules/calcite/src/test/sql/types/interval/test_interval_ops.test
+++ b/modules/calcite/src/test/sql/types/interval/test_interval_ops.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/interval/test_interval_ops.test
 # description: Test various ops involving intervals
 # group: [interval]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-14975
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/list/array_agg.test b/modules/calcite/src/test/sql/types/list/array_agg.test_ignore
similarity index 95%
rename from modules/calcite/src/test/sql/types/list/array_agg.test
rename to modules/calcite/src/test/sql/types/list/array_agg.test_ignore
index 3d847b1..c98ac28 100644
--- a/modules/calcite/src/test/sql/types/list/array_agg.test
+++ b/modules/calcite/src/test/sql/types/list/array_agg.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/list/array_agg.test
 # description: Test array_agg function
 # group: [list]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15620
 
 # scalar array agg
 query II
diff --git a/modules/calcite/src/test/sql/types/list/lineitem_list.test_slow b/modules/calcite/src/test/sql/types/list/lineitem_list.test_slow
deleted file mode 100644
index 9aaa5e0..0000000
--- a/modules/calcite/src/test/sql/types/list/lineitem_list.test_slow
+++ /dev/null
@@ -1,120 +0,0 @@
-# name: test/sql/types/list/lineitem_list.test_slow
-# description: Test packing and unpacking lineitem into lists
-# group: [list]
-
-require tpch
-
-statement ok
-PRAGMA enable_verification
-
-statement ok
-CALL dbgen(sf=0.01, suffix='_org')
-
-# TODO this has a small limit in it right now because of performance issues. Fix this.
-statement ok
-CREATE TABLE lineitem_small AS SELECT * FROM lineitem_org LIMIT 1050;
-
-statement ok
-CREATE OR REPLACE VIEW lineitem AS SELECT * FROM lineitem_small
-
-# run the regular Q1 on the small lineitem set
-query I nosort q1small
-PRAGMA tpch(1)
-----
-
-# list of structs, grouped by order key
-statement ok
-CREATE OR REPLACE VIEW lineitem AS
-SELECT l_orderkey,
-       STRUCT_EXTRACT(struct, 'l_partkey') l_partkey,
-       STRUCT_EXTRACT(struct, 'l_suppkey') l_suppkey,
-       STRUCT_EXTRACT(struct, 'l_linenumber') l_linenumber,
-       STRUCT_EXTRACT(struct, 'l_quantity') l_quantity,
-       STRUCT_EXTRACT(struct, 'l_extendedprice') l_extendedprice,
-       STRUCT_EXTRACT(struct, 'l_discount') l_discount,
-       STRUCT_EXTRACT(struct, 'l_tax') l_tax,
-       STRUCT_EXTRACT(struct, 'l_returnflag') l_returnflag,
-       STRUCT_EXTRACT(struct, 'l_linestatus') l_linestatus,
-       STRUCT_EXTRACT(struct, 'l_shipdate') l_shipdate,
-       STRUCT_EXTRACT(struct, 'l_commitdate') l_commitdate,
-       STRUCT_EXTRACT(struct, 'l_receiptdate') l_receiptdate,
-       STRUCT_EXTRACT(struct, 'l_shipinstruct') l_shipinstruct,
-       STRUCT_EXTRACT(struct, 'l_shipmode') l_shipmode,
-       STRUCT_EXTRACT(struct, 'l_comment') l_comment
-FROM
-  (SELECT l_orderkey,
-          UNLIST(rest) as struct
-   FROM
-     (SELECT l_orderkey,
-             LIST(STRUCT_PACK(l_partkey , l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus , l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment)) rest
-      FROM
-        (SELECT *
-         FROM lineitem_small) lss
-      GROUP BY l_orderkey) s1) s2
-
-# run Q1 again: it should produce the same result
-query I nosort q1small
-PRAGMA tpch(1)
-----
-
-# list of structs holding the entire table
-# i.e. database as a value
-statement ok
-CREATE OR REPLACE VIEW lineitem AS
-SELECT STRUCT_EXTRACT(ls, 'l_orderkey') l_orderkey,
-       STRUCT_EXTRACT(ls, 'l_partkey') l_partkey,
-       STRUCT_EXTRACT(ls, 'l_suppkey') l_suppkey,
-       STRUCT_EXTRACT(ls, 'l_linenumber') l_linenumber,
-       STRUCT_EXTRACT(ls, 'l_quantity') l_quantity,
-       STRUCT_EXTRACT(ls, 'l_extendedprice') l_extendedprice,
-       STRUCT_EXTRACT(ls, 'l_discount') l_discount,
-       STRUCT_EXTRACT(ls, 'l_tax') l_tax,
-       STRUCT_EXTRACT(ls, 'l_returnflag') l_returnflag,
-       STRUCT_EXTRACT(ls, 'l_linestatus') l_linestatus,
-       STRUCT_EXTRACT(ls, 'l_shipdate') l_shipdate,
-       STRUCT_EXTRACT(ls, 'l_commitdate') l_commitdate,
-       STRUCT_EXTRACT(ls, 'l_receiptdate') l_receiptdate,
-       STRUCT_EXTRACT(ls, 'l_shipinstruct') l_shipinstruct,
-       STRUCT_EXTRACT(ls, 'l_shipmode') l_shipmode,
-       STRUCT_EXTRACT(ls, 'l_comment') l_comment
-FROM
-  (SELECT UNNEST(lineitem) ls
-   FROM
-     (SELECT LIST(STRUCT_PACK(l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment)) lineitem
-      FROM
-        (SELECT *
-         FROM lineitem_small) s1) s2) s3
-
-# run Q1 again: it should produce the same result
-query I nosort q1small
-PRAGMA tpch(1)
-----
-
-# struct of lists
-statement ok
-CREATE OR REPLACE VIEW lineitem AS
-SELECT UNNEST(STRUCT_EXTRACT(lineitem, 'll_orderkey')) l_orderkey,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_partkey')) l_partkey,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_suppkey')) l_suppkey,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_linenumber')) l_linenumber,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_quantity')) l_quantity,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_extendedprice')) l_extendedprice,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_discount')) l_discount,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_tax')) l_tax,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_returnflag')) l_returnflag,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_linestatus')) l_linestatus,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_shipdate')) l_shipdate,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_commitdate')) l_commitdate,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_receiptdate')) l_receiptdate,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_shipinstruct')) l_shipinstruct,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_shipmode')) l_shipmode,
-       UNNEST(STRUCT_EXTRACT(lineitem, 'll_comment')) l_comment
-FROM
-  (SELECT STRUCT_PACK(ll_orderkey:= LIST(l_orderkey), ll_partkey := LIST(l_partkey), ll_suppkey := LIST(l_suppkey), ll_linenumber := LIST(l_linenumber), ll_quantity := LIST(l_quantity), ll_extendedprice := LIST(l_extendedprice), ll_discount := LIST(l_discount), ll_tax := LIST(l_tax), ll_returnflag := LIST(l_returnflag), ll_linestatus := LIST(l_linestatus), ll_shipdate := LIST(l_shipdate), ll_commitdate := LIST(l_commitdate), ll_receiptdate := LIST(l_receiptdate), ll_shipinstruct := LIST( [...]
-   FROM
-     (SELECT *
-      FROM lineitem_small) s1) s2;
-
-query I nosort q1small
-PRAGMA tpch(1)
-----
diff --git a/modules/calcite/src/test/sql/types/list/list.test_slow b/modules/calcite/src/test/sql/types/list/list.test_slow_ignore
similarity index 77%
rename from modules/calcite/src/test/sql/types/list/list.test_slow
rename to modules/calcite/src/test/sql/types/list/list.test_slow_ignore
index 83703d0..47b6e82 100644
--- a/modules/calcite/src/test/sql/types/list/list.test_slow
+++ b/modules/calcite/src/test/sql/types/list/list.test_slow_ignore
@@ -1,9 +1,7 @@
 # name: test/sql/types/list/list.test_slow
 # description: Test big list
 # group: [list]
-
-statement ok
-PRAGMA enable_verification
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 CREATE TABLE test AS (SELECT range i, 0 as j FROM range(70000));
@@ -11,4 +9,4 @@ CREATE TABLE test AS (SELECT range i, 0 as j FROM range(70000));
 query I
 SELECT list(i)[69999] FROM test GROUP BY j;
 ----
-69999
\ No newline at end of file
+69999
diff --git a/modules/calcite/src/test/sql/types/list/list_aggregates.test b/modules/calcite/src/test/sql/types/list/list_aggregates.test_ignore
similarity index 95%
rename from modules/calcite/src/test/sql/types/list/list_aggregates.test
rename to modules/calcite/src/test/sql/types/list/list_aggregates.test_ignore
index 27ef780..a5c72ff 100644
--- a/modules/calcite/src/test/sql/types/list/list_aggregates.test
+++ b/modules/calcite/src/test/sql/types/list/list_aggregates.test_ignore
@@ -1,8 +1,7 @@
 # name: test/sql/types/list/list_aggregates.test
 # description: Test lists with aggregations
 # group: [list]
-
-require vector_size 512
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15563
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/list/test_list_extract.test b/modules/calcite/src/test/sql/types/list/test_list_extract.test
deleted file mode 100644
index 809899f..0000000
--- a/modules/calcite/src/test/sql/types/list/test_list_extract.test
+++ /dev/null
@@ -1,182 +0,0 @@
-# name: test/sql/types/list/test_list_extract.test
-# description: Test scalar lists
-# group: [list]
-
-statement ok
-PRAGMA enable_verification
-
-statement error
-SELECT LIST_EXTRACT(NULL, 0)
-
-statement error
-SELECT LIST_EXTRACT(42, 0)
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(), 0)
-----
-NULL
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(NULL), 0)
-----
-NULL
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(NULL), -1)
-----
-NULL
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42), NULL)
-----
-NULL
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42), 0)
-----
-42
-
-# alias
-query I
-SELECT LIST_ELEMENT(LIST_VALUE(42), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42, 43), 1)
-----
-43
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42, 43, 44, 45), -1)
-----
-45
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42, 43, 44, 45), -2)
-----
-44
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42, 43, 44, 45), -4)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42, 43, 44, 45), -5)
-----
-NULL
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42), 1)
-----
-NULL
-
-statement ok
-CREATE TABLE list_extract_test(i INTEGER, g INTEGER);
-
-statement ok
-INSERT INTO list_extract_test VALUES (1, 1), (2, 1), (3, 2), (NULL, 3), (42, 3);
-
-query I
-SELECT LIST_EXTRACT(LIST(i), 0) FROM list_extract_test GROUP BY g;
-----
-1
-3
-NULL
-
-query I
-SELECT LIST_EXTRACT(LIST(i), 1) FROM list_extract_test GROUP BY g;
-----
-2
-NULL
-42
-
-query I
-SELECT LIST_EXTRACT(LIST(i), 2) FROM list_extract_test GROUP BY g;
-----
-NULL
-NULL
-NULL
-
-# all them types woo
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE('Hello', 'World'), 0)
-----
-Hello
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE('Hello', 'World'), 1)
-----
-World
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::UTINYINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::TINYINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::USMALLINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::SMALLINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::UINTEGER), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::INTEGER), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::UBIGINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::BIGINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::HUGEINT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::FLOAT), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE(42::DOUBLE), 0)
-----
-42
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE('2019-11-26'::DATE), 0)
-----
-2019-11-26
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE('21:42:00'::TIME), 0)
-----
-21:42:00
-
-query I
-SELECT LIST_EXTRACT(LIST_VALUE('2019-11-26 09:42:00'::TIMESTAMP), 0)
-----
-2019-11-26 09:42:00
\ No newline at end of file
diff --git a/modules/calcite/src/test/sql/types/list/test_list_index.test b/modules/calcite/src/test/sql/types/list/test_list_index.test
deleted file mode 100644
index e1a0aeb..0000000
--- a/modules/calcite/src/test/sql/types/list/test_list_index.test
+++ /dev/null
@@ -1,62 +0,0 @@
-# name: test/sql/types/list/test_list_index.test
-# description: Test accessing list elements using a list
-# group: [list]
-
-statement ok
-PRAGMA enable_verification
-
-query I
-SELECT a[0] FROM (VALUES (LIST_VALUE())) tbl(a)
-----
-NULL
-
-query I
-SELECT a[0] FROM (VALUES (LIST_VALUE(NULL))) tbl(a)
-----
-NULL
-
-query I
-SELECT a[-1] FROM (VALUES (LIST_VALUE(NULL))) tbl(a)
-----
-NULL
-
-query I
-SELECT a[NULL] FROM (VALUES (LIST_VALUE(42))) tbl(a)
-----
-NULL
-
-query I
-SELECT a[0] FROM (VALUES (LIST_VALUE(42))) tbl(a)
-----
-42
-
-query I
-SELECT a[0+1-1] FROM (VALUES (LIST_VALUE(42))) tbl(a)
-----
-42
-
-query I
-SELECT a[b] FROM (VALUES (LIST_VALUE(42), 0)) tbl(a, b)
-----
-42
-
-# doing it directly on the function does not require brackets
-query I
-SELECT (LIST_VALUE(42))[0]
-----
-42
-
-query I
-SELECT LIST_VALUE(42)[0]
-----
-42
-
-# slicing: supported in parser but not yet in engine
-statement error
-SELECT a[1:] FROM (VALUES (LIST_VALUE(42, 43, 44))) tbl(a)
-
-statement error
-SELECT a[:1] FROM (VALUES (LIST_VALUE(42, 43, 44))) tbl(a)
-
-statement error
-SELECT a[0:1] FROM (VALUES (LIST_VALUE(42, 43, 44))) tbl(a)
diff --git a/modules/calcite/src/test/sql/types/list/test_nested_list.test b/modules/calcite/src/test/sql/types/list/test_nested_list.test
deleted file mode 100644
index 081809d..0000000
--- a/modules/calcite/src/test/sql/types/list/test_nested_list.test
+++ /dev/null
@@ -1,265 +0,0 @@
-# name: test/sql/types/list/test_nested_list.test
-# description: Test filter and projection of nested lists
-# group: [list]
-
-statement ok
-PRAGMA enable_verification
-
-statement ok
-CREATE TABLE list_data (g INTEGER, e INTEGER)
-
-statement ok
-INSERT INTO list_data VALUES (1, 1), (1, 2), (2, 3), (2, 4), (2, 5), (3, 6), (5, NULL)
-
-query T
-SELECT LIST(a) l1 FROM (VALUES (1), (2), (3)) AS t1 (a)
-----
-[1, 2, 3]
-
-query I
-SELECT UNNEST(l1) FROM (SELECT LIST(a) l1 FROM (VALUES (1), (2), (3)) AS t1 (a)) t1
-----
-1
-2
-3
-
-query TT
-SELECT * FROM (SELECT LIST(a) l1 FROM (VALUES (1), (2), (3)) AS t1 (a)) t1, (SELECT LIST(b) l2 FROM (VALUES (4), (5), (6), (7)) AS t2 (b)) t2
-----
-[1, 2, 3]	[4, 5, 6, 7]
-
-query II
-SELECT UNNEST(l1) u1, UNNEST(l2) u2 FROM (SELECT LIST(a) l1 FROM (VALUES (1), (2), (3)) AS t1 (a)) t1, (SELECT LIST(b) l2 FROM (VALUES (4), (5), (6), (7)) AS t2 (b)) t2
-----
-1	4
-2	5
-3	6
-NULL	7
-
-query IT
-SELECT UNNEST(l1), l2 FROM (SELECT LIST(a) l1 FROM (VALUES (1), (2), (3)) AS t1 (a)) t1, 	(SELECT LIST(b) l2 FROM (VALUES (4), (5), (6), (7)) AS t2 (b)) t2
-----
-1	[4, 5, 6, 7]
-2	[4, 5, 6, 7]
-3	[4, 5, 6, 7]
-
-query TI
-SELECT l1, UNNEST(l2) FROM (SELECT LIST(a) l1 FROM (VALUES (1), (2), (3)) AS t1 (a)) t1, (SELECT LIST(b) l2 FROM (VALUES (4), (5), (6), (7)) AS t2 (b)) t2
-----
-[1, 2, 3]	4
-[1, 2, 3]	5
-[1, 2, 3]	6
-[1, 2, 3]	7
-
-query IT
-SELECT UNNEST(LIST(e)) ue, LIST(g) from list_data ORDER BY 1 NULLS LAST
-----
-1	[1, 1, 2, 2, 2, 3, 5]
-2	[1, 1, 2, 2, 2, 3, 5]
-3	[1, 1, 2, 2, 2, 3, 5]
-4	[1, 1, 2, 2, 2, 3, 5]
-5	[1, 1, 2, 2, 2, 3, 5]
-6	[1, 1, 2, 2, 2, 3, 5]
-NULL	[1, 1, 2, 2, 2, 3, 5]
-
-query IT
-SELECT g, LIST(e) from list_data GROUP BY g ORDER BY g
-----
-1	[1, 2]
-2	[3, 4, 5]
-3	[6]
-5	[NULL]
-
-query ITT
-SELECT g, LIST(e) l1, LIST(e) l2 from list_data GROUP BY g ORDER BY g
-----
-1	[1, 2]	[1, 2]
-2	[3, 4, 5]	[3, 4, 5]
-3	[6]	[6]
-5	[NULL]	[NULL]
-
-query IT
-SELECT g, LIST(e/2.0) from list_data GROUP BY g order by g
-----
-1	[0.500000, 1.000000]
-2	[1.500000, 2.000000, 2.500000]
-3	[3.000000]
-5	[NULL]
-
-query IT
-SELECT g, LIST(CAST(e AS VARCHAR)) from list_data GROUP BY g order by g
-----
-1	[1, 2]
-2	[3, 4, 5]
-3	[6]
-5	[NULL]
-
-query T
-SELECT LIST(e) from list_data
-----
-[1, 2, 3, 4, 5, 6, NULL]
-
-query I
-SELECT UNNEST(LIST(e)) ue from list_data ORDER BY ue
-----
-NULL
-1
-2
-3
-4
-5
-6
-
-query TT
-SELECT LIST(e), LIST(g) from list_data
-----
-[1, 2, 3, 4, 5, 6, NULL]	[1, 1, 2, 2, 2, 3, 5]
-
-query T
-SELECT LIST(42)
-----
-[42]
-
-query T
-SELECT LIST(42) FROM list_data
-----
-[42, 42, 42, 42, 42, 42, 42]
-
-query I
-SELECT UNNEST(LIST(42))
-----
-42
-
-# unlist is alias of unnest for symmetry reasons
-query I
-SELECT UNLIST(LIST(42))
-----
-42
-
-query II
-SELECT UNNEST(LIST(e)) ue, UNNEST(LIST(g)) ug from list_data ORDER BY ue
-----
-NULL	5
-1	1
-2	1
-3	2
-4	2
-5	2
-6	3
-
-query III
-SELECT g, UNNEST(LIST(e)) ue, UNNEST(LIST(e+1)) ue2 from list_data GROUP BY g ORDER BY ue
-----
-5	NULL	NULL
-1	1	2
-1	2	3
-2	3	4
-2	4	5
-2	5	6
-3	6	7
-
-query II
-SELECT g, UNNEST(l) u FROM (SELECT g, LIST(e) l FROM list_data GROUP BY g) u1 ORDER BY u
-----
-5	NULL
-1	1
-1	2
-2	3
-2	4
-2	5
-3	6
-
-query II
-SELECT g, UNNEST(l)+1 u FROM (SELECT g, LIST(e) l FROM list_data GROUP BY g) u1 ORDER BY u
-----
-5	NULL
-1	2
-1	3
-2	4
-2	5
-2	6
-3	7
-
-# omg omg, list of structs, structs of lists
-query IT
-SELECT g, STRUCT_PACK(a := g, b := le) sl FROM (SELECT g, LIST(e) le from list_data GROUP BY g) xx WHERE g < 3 ORDER BY g
-----
-1	<a: 1, b: [1, 2]>
-2	<a: 2, b: [3, 4, 5]>
-
-query T
-SELECT LIST(STRUCT_PACK(a := g, b := le)) mind_blown FROM (SELECT g, LIST(e) le from list_data  GROUP BY g ORDER BY g) xx
-----
-[<a: 1, b: [1, 2]>, <a: 2, b: [3, 4, 5]>, <a: 3, b: [6]>, <a: 5, b: [NULL]>]
-
-query IT
-SELECT g, LIST(STRUCT_PACK(a := e, b := e+1)) ls from list_data GROUP BY g ORDER BY g
-----
-1	[<a: 1, b: 2>, <a: 2, b: 3>]
-2	[<a: 3, b: 4>, <a: 4, b: 5>, <a: 5, b: 6>]
-3	[<a: 6, b: 7>]
-5	[<a: NULL, b: NULL>]
-
-# TODO check second col
-query IT
-SELECT g, LIST(STRUCT_PACK(a := e, b := e+1)) ls from list_data WHERE g > 2GROUP BY g ORDER BY g
-----
-3	[<a: 6, b: 7>]
-5	[<a: NULL, b: NULL>]
-
-# list of list of int
-query IT
-SELECT g2, LIST(le) FROM (SELECT g % 2 g2, LIST(e) le from list_data GROUP BY g ORDER BY g) sq 	GROUP BY g2 ORDER BY g2
-----
-0	[[3, 4, 5]]
-1	[[1, 2], [6], [NULL]]
-
-query R
-SELECT SUM(ue) FROM (SELECT UNNEST(le) ue FROM (SELECT g, LIST(e) le from list_data  GROUP BY g ORDER BY g) xx) xy
-----
-21.000000
-
-# this is technically equivalent but is not supported
-statement error
-SELECT SUM(UNNEST(le)) FROM ( SELECT g, LIST(e) le from list_data  GROUP BY g ORDER BY g) xx
-
-# you're holding it wrong
-statement error
-SELECT LIST(LIST(42))
-
-statement error
-SELECT UNNEST(UNNEST(LIST(42))
-
-statement error
-SELECT LIST()
-
-statement error
-SELECT LIST() FROM list_data
-
-statement error
-SELECT LIST(e, g) FROM list_data
-
-statement error
-SELECT g, UNNEST(l+1) u FROM (SELECT g, LIST(e) l FROM list_data GROUP BY g) u1
-
-statement error
-SELECT g, UNNEST(g) u FROM (SELECT g, LIST(e) l FROM list_data GROUP BY g) u1
-
-statement error
-SELECT g, UNNEST() u FROM (SELECT g, LIST(e) l FROM list_data GROUP BY g) u1
-
-statement error
-SELECT UNNEST(42)
-
-statement error
-SELECT UNNEST()
-
-statement error
-SELECT UNNEST(42) from list_data
-
-statement error
-SELECT UNNEST() from list_data
-
-statement error
-SELECT g FROM (SELECT g, LIST(e) l FROM list_data GROUP BY g) u1 where UNNEST(l) > 42
-
diff --git a/modules/calcite/src/test/sql/types/list/test_scalar_list.test b/modules/calcite/src/test/sql/types/list/test_scalar_list.test
deleted file mode 100644
index 775e0e2..0000000
--- a/modules/calcite/src/test/sql/types/list/test_scalar_list.test
+++ /dev/null
@@ -1,140 +0,0 @@
-# name: test/sql/types/list/test_scalar_list.test
-# description: Test scalar lists
-# group: [list]
-
-statement ok
-PRAGMA enable_verification
-
-query T
-SELECT LIST_VALUE('hello')
-----
-[hello]
-
-query T
-SELECT LIST_VALUE('hello')::VARCHAR
-----
-[hello]
-
-query T
-SELECT l::VARChAR FROM (VALUES (LIST_VALUE('hello', 'world')), (LIST_VALUE('a', 'b', 'c'))) t(l)
-----
-[hello, world]
-[a, b, c]
-
-statement ok
-CREATE TABLE list_data (g INTEGER, e INTEGER)
-
-statement ok
-INSERT INTO list_data VALUES (1, 1), (1, 2), (2, 3), (2, 4), (2, 5), (3, 6), (5, NULL)
-
-query TTTTT
-SELECT LIST_VALUE(1, 2, 3, '4') a, LIST_VALUE('a','b','c') b, LIST_VALUE(42, NULL) c, LIST_VALUE(NULL, NULL, NULL) d, LIST_VALUE() e
-----
-[1, 2, 3, 4]	[a, b, c]	[42, NULL]	[NULL, NULL, NULL]	[]
-
-query T
-SELECT a FROM (VALUES (LIST_VALUE(1, 2, 3, 4)), (LIST_VALUE()), (LIST_VALUE(NULL::INTEGER)), (LIST_VALUE(42))) lv(a)
-----
-[1, 2, 3, 4]
-[]
-[NULL]
-[42]
-
-query T
-SELECT a FROM (VALUES (LIST_VALUE('hello', 'world')), (LIST_VALUE()), (LIST_VALUE(NULL::VARCHAR)), (LIST_VALUE('42'))) lv(a)
-----
-[hello, world]
-[]
-[NULL]
-[42]
-
-query TTT
-SELECT * FROM (VALUES ((LIST_VALUE()), (LIST_VALUE(NULL)), LIST_VALUE(1, 2))) lv(a)
-----
-[]	[NULL]	[1, 2]
-
-query T
-SELECT * FROM (VALUES (LIST_VALUE(1, 2)), (LIST_VALUE()), (LIST_VALUE(NULL::INTEGER))) lv(a)
-----
-[1, 2]
-[]
-[NULL]
-
-# casting null to list or empty list to something else should work
-query T
-SELECT LIST_VALUE(1, 2, 3) UNION ALL SELECT LIST_VALUE(NULL::INTEGER) UNION ALL SELECT LIST_VALUE() UNION ALL SELECT NULL
-----
-[1, 2, 3]
-[NULL]
-[]
-NULL
-
-query T
-SELECT NULL UNION ALL SELECT LIST_VALUE() UNION ALL SELECT LIST_VALUE(NULL::INTEGER) UNION ALL SELECT LIST_VALUE(1, 2, 3)
-----
-NULL
-[]
-[NULL]
-[1, 2, 3]
-
-# empty list should not abort UNNEST
-query I
-SELECT UNNEST(a) ua FROM (VALUES (LIST_VALUE(1, 2, 3, 4)), (LIST_VALUE()), (LIST_VALUE(NULL::INTEGER)), (LIST_VALUE(42))) lv(a)
-----
-1
-2
-3
-4
-NULL
-42
-
-# TODO this should work but does not. its also kind of obscure
-# SELECT UNNEST(a) ua FROM (VALUES (LIST_VALUE()), (LIST_VALUE(1, 2, 3, 4)), LIST_VALUE(NULL)), (LIST_VALUE(42))) lv(a));
-# 1, 2, 3, 4, Value(), 42
-# list child type mismatch
-statement error
-SELECT * FROM (VALUES (LIST_VALUE(1, 2)), (LIST_VALUE()), (LIST_VALUE('a'))) lv(a)
-
-# can't cast lists to stuff
-statement error
-SELECT CAST(LIST_VALUE(42) AS INTEGER)
-
-# can't add a number to a list
-statement error
-SELECT LIST_VALUE(42) + 4
-
-# can't add a number to a list
-statement error
-SELECT LIST_VALUE(42, 'a')
-
-# can have unnest anywhere
-query I
-SELECT CAST(UNNEST(LIST_VALUE(42))+2 AS INTEGER)
-----
-44
-
-query T
-SELECT LIST_VALUE(g, e, 42, NULL) FROM list_data WHERE g > 2
-----
-[3, 6, 42, NULL]
-[5, NULL, 42, NULL]
-
-query T
-SELECT CASE WHEN g = 2 THEN LIST_VALUE(g, e, 42) ELSE LIST_VALUE(84, NULL) END FROM list_data WHERE g > 1 UNION ALL SELECT LIST_VALUE(NULL::INTEGER)
-----
-[2, 3, 42]
-[2, 4, 42]
-[2, 5, 42]
-[84, NULL]
-[84, NULL]
-[NULL]
-
-query I
-SELECT CASE WHEN g = 2 THEN LIST_VALUE(g, e, 42) ELSE LIST_VALUE('eeek') END FROM list_data	WHERE g > 1
-----
-[2, 3, 42]
-[2, 4, 42]
-[2, 5, 42]
-[eeek]
-[eeek]
-
diff --git a/modules/calcite/src/test/sql/types/null/test_boolean_null.test b/modules/calcite/src/test/sql/types/null/test_boolean_null.test
index ef034a8..842e1c7 100644
--- a/modules/calcite/src/test/sql/types/null/test_boolean_null.test
+++ b/modules/calcite/src/test/sql/types/null/test_boolean_null.test
@@ -42,7 +42,7 @@ statement ok
 INSERT INTO test VALUES (11, 22), (NULL, 21), (13, 22), (12, NULL), (16, NULL)
 
 query IIRII
-SELECT b, COUNT(a), SUM(a), MIN(a), MAX(a) FROM test GROUP BY b ORDER BY b
+SELECT b, COUNT(a), SUM(a), MIN(a), MAX(a) FROM test GROUP BY b ORDER BY b NULLS LAST;
 ----
 21	0	NULL	NULL	NULL
 22	2	24.000000	11	13
diff --git a/modules/calcite/src/test/sql/types/null/test_null.test b/modules/calcite/src/test/sql/types/null/test_null.test
index 9f8ce2c..2774a4e 100644
--- a/modules/calcite/src/test/sql/types/null/test_null.test
+++ b/modules/calcite/src/test/sql/types/null/test_null.test
@@ -42,7 +42,7 @@ INSERT INTO test VALUES (11, 22), (NULL, 21), (13, 22)
 
 # NULL selection
 query I
-SELECT a FROM test ORDER BY a;
+SELECT a FROM test ORDER BY a NULLS LAST
 ----
 11
 13
@@ -50,7 +50,7 @@ NULL
 
 # cast NULL
 query I
-SELECT cast(a AS BIGINT) FROM test ORDER BY a;
+SELECT cast(a AS BIGINT) FROM test ORDER BY a NULLS LAST;
 ----
 11
 13
@@ -58,7 +58,7 @@ NULL
 
 # NULL addition results in NULL
 query I
-SELECT a + b FROM test ORDER BY a;
+SELECT a + b FROM test ORDER BY a NULLS LAST
 ----
 33
 35
diff --git a/modules/calcite/src/test/sql/types/null/test_null_aggr.test b/modules/calcite/src/test/sql/types/null/test_null_aggr.test_ignore
similarity index 95%
rename from modules/calcite/src/test/sql/types/null/test_null_aggr.test
rename to modules/calcite/src/test/sql/types/null/test_null_aggr.test_ignore
index b29a0e5..8fa87d8 100644
--- a/modules/calcite/src/test/sql/types/null/test_null_aggr.test
+++ b/modules/calcite/src/test/sql/types/null/test_null_aggr.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/null/test_null_aggr.test
 # description: Test NULL handling in aggregations
 # group: [null]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15539
 
 statement ok
 CREATE TABLE test (a INTEGER, b INTEGER);
diff --git a/modules/calcite/src/test/sql/types/time/test_time.test b/modules/calcite/src/test/sql/types/time/test_time.test_ignore
similarity index 89%
rename from modules/calcite/src/test/sql/types/time/test_time.test
rename to modules/calcite/src/test/sql/types/time/test_time.test_ignore
index 4380780..a1feccf 100644
--- a/modules/calcite/src/test/sql/types/time/test_time.test
+++ b/modules/calcite/src/test/sql/types/time/test_time.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/time/test_time.test
 # description: Test basic TIME functionality
 # group: [time]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15619
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/time/time_parsing.test b/modules/calcite/src/test/sql/types/time/time_parsing.test_ignore
similarity index 86%
rename from modules/calcite/src/test/sql/types/time/time_parsing.test
rename to modules/calcite/src/test/sql/types/time/time_parsing.test_ignore
index 104a4a5..cf07496 100644
--- a/modules/calcite/src/test/sql/types/time/time_parsing.test
+++ b/modules/calcite/src/test/sql/types/time/time_parsing.test_ignore
@@ -1,6 +1,8 @@
 # name: test/sql/types/time/time_parsing.test
 # description: Test time parsing
 # group: [time]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15619
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15622
 
 statement ok
 PRAGMA enable_verification
@@ -57,4 +59,4 @@ SELECT '14:42:70.500'::TIME::VARCHAR
 
 # invalid separator
 statement error
-SELECT '14-42-04'::TIME::VARCHAR
\ No newline at end of file
+SELECT '14-42-04'::TIME::VARCHAR
diff --git a/modules/calcite/src/test/sql/types/timestamp/bc_timestamp.test b/modules/calcite/src/test/sql/types/timestamp/bc_timestamp.test_ignore
similarity index 87%
rename from modules/calcite/src/test/sql/types/timestamp/bc_timestamp.test
rename to modules/calcite/src/test/sql/types/timestamp/bc_timestamp.test_ignore
index bb9a8cb..0da6543 100644
--- a/modules/calcite/src/test/sql/types/timestamp/bc_timestamp.test
+++ b/modules/calcite/src/test/sql/types/timestamp/bc_timestamp.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/timestamp/bc_timestamp.test
 # description: Test BC timestamps
 # group: [timestamp]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15622
 
 query I
 SELECT '1969-01-01 01:03:20.45432'::TIMESTAMP::VARCHAR
diff --git a/modules/calcite/src/test/sql/types/timestamp/test_incorrect_timestamp.test b/modules/calcite/src/test/sql/types/timestamp/test_incorrect_timestamp.test_ignore
similarity index 95%
rename from modules/calcite/src/test/sql/types/timestamp/test_incorrect_timestamp.test
rename to modules/calcite/src/test/sql/types/timestamp/test_incorrect_timestamp.test_ignore
index cc64d41..8901fc6 100644
--- a/modules/calcite/src/test/sql/types/timestamp/test_incorrect_timestamp.test
+++ b/modules/calcite/src/test/sql/types/timestamp/test_incorrect_timestamp.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/timestamp/test_incorrect_timestamp.test
 # description: Test out of range/incorrect timestamp formats
 # group: [timestamp]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15623
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/timestamp/test_timestamp.test b/modules/calcite/src/test/sql/types/timestamp/test_timestamp.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/types/timestamp/test_timestamp.test
rename to modules/calcite/src/test/sql/types/timestamp/test_timestamp.test_ignore
index 9212a5f..8c8e609 100644
--- a/modules/calcite/src/test/sql/types/timestamp/test_timestamp.test
+++ b/modules/calcite/src/test/sql/types/timestamp/test_timestamp.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/timestamp/test_timestamp.test
 # description: Test TIMESTAMP type
 # group: [timestamp]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15623
 
 statement ok
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/types/timestamp/test_timestamp_ms.test b/modules/calcite/src/test/sql/types/timestamp/test_timestamp_ms.test_ignore
similarity index 86%
rename from modules/calcite/src/test/sql/types/timestamp/test_timestamp_ms.test
rename to modules/calcite/src/test/sql/types/timestamp/test_timestamp_ms.test_ignore
index e91b493..17711eb 100644
--- a/modules/calcite/src/test/sql/types/timestamp/test_timestamp_ms.test
+++ b/modules/calcite/src/test/sql/types/timestamp/test_timestamp_ms.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/types/timestamp/test_timestamp_ms.test
 # description: Test milliseconds with timestamp
 # group: [timestamp]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15623
 
 query TT
 SELECT CAST('2001-04-20 14:42:11.123' AS TIMESTAMP) a, CAST('2001-04-20 14:42:11.0' AS TIMESTAMP) b;
diff --git a/modules/calcite/src/test/sql/update/null_update_merge_transaction.test b/modules/calcite/src/test/sql/update/null_update_merge_transaction.test
deleted file mode 100644
index fb9613b..0000000
--- a/modules/calcite/src/test/sql/update/null_update_merge_transaction.test
+++ /dev/null
@@ -1,120 +0,0 @@
-# name: test/sql/update/null_update_merge_transaction.test
-# description: Test repeated updates that change the nullness of values
-# group: [update]
-
-# create a table
-statement ok
-CREATE TABLE test (id INTEGER, a INTEGER);
-
-statement ok
-INSERT INTO test VALUES (1, 1), (2, 2), (3, 3), (4, NULL);
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	1
-2	2
-3	3
-4	NULL
-
-statement ok
-BEGIN TRANSACTION
-
-statement ok
-UPDATE test SET a=CASE WHEN a IS NULL THEN 1 ELSE NULL END
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	NULL
-2	NULL
-3	NULL
-4	1
-
-statement ok
-UPDATE test SET a=2 WHERE id >= 2 AND id <= 3
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	NULL
-2	2
-3	2
-4	1
-
-statement ok
-UPDATE test SET a=NULL WHERE id >= 3
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	NULL
-2	2
-3	NULL
-4	NULL
-
-statement ok
-UPDATE test SET a=id WHERE id != 3
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	1
-2	2
-3	NULL
-4	4
-
-statement ok
-UPDATE test SET a=NULL WHERE id != 3
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	NULL
-2	NULL
-3	NULL
-4	NULL
-
-statement ok
-UPDATE test SET a=3 WHERE id != 2
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	3
-2	NULL
-3	3
-4	3
-
-statement ok
-UPDATE test SET a=7 WHERE id != 3
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	7
-2	7
-3	3
-4	7
-
-statement ok
-UPDATE test SET a=NULL
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	NULL
-2	NULL
-3	NULL
-4	NULL
-
-statement ok
-COMMIT
-
-query II
-SELECT * FROM test ORDER BY id;
-----
-1	NULL
-2	NULL
-3	NULL
-4	NULL
diff --git a/modules/calcite/src/test/sql/update/test_big_string_update.test b/modules/calcite/src/test/sql/update/test_big_string_update.test_ignore
similarity index 100%
rename from modules/calcite/src/test/sql/update/test_big_string_update.test
rename to modules/calcite/src/test/sql/update/test_big_string_update.test_ignore
diff --git a/modules/calcite/src/test/sql/update/test_big_table_update.test_slow b/modules/calcite/src/test/sql/update/test_big_table_update.test_slow
deleted file mode 100644
index e54fedd..0000000
--- a/modules/calcite/src/test/sql/update/test_big_table_update.test_slow
+++ /dev/null
@@ -1,111 +0,0 @@
-# name: test/sql/update/test_big_table_update.test_slow
-# description: Update big table of even and odd values
-# group: [update]
-
-# create a table with the values [0, ..., 3K]
-statement ok con1
-BEGIN TRANSACTION
-
-statement ok con1
-CREATE TABLE test AS SELECT i AS a, 'hello' AS b FROM range (0, 3000, 1) t1(i);
-
-# insert a bunch more values
-# 3000 -> 6000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-# 6000 -> 12000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-# 12000 -> 24000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-# 24000 -> 48000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-# 48000 -> 96000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-# 96000 -> 192000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-# 192000 -> 384000
-statement ok con1
-INSERT INTO test SELECT * FROM test
-
-statement ok con1
-COMMIT
-
-query RR con1
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-575808000.000000	1920000.000000
-
-query RR con2
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-575808000.000000	1920000.000000
-
-# increment all even values by two
-statement ok con1
-BEGIN TRANSACTION
-
-statement ok con1
-UPDATE test SET a=a+2, b='hellohello' WHERE a%2=0
-
-query RR con1
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-576192000.000000	2880000.000000
-
-query RR con2
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-575808000.000000	1920000.000000
-
-# now increment all odd values by two
-statement ok con1
-UPDATE test SET a=a+2, b='hellohello' WHERE a%2=1
-
-query RR con1
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-576576000.000000	3840000.000000
-
-query RR con2
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-575808000.000000	1920000.000000
-
-# increment all tuples by two now
-statement ok con1
-UPDATE test SET a=a+2, b='hellohellohellohello'
-
-query RR con1
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-577344000.000000	7680000.000000
-
-query RR con2
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-575808000.000000	1920000.000000
-
-statement ok con1
-COMMIT
-
-query RR con1
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-577344000.000000	7680000.000000
-
-query RR con2
-SELECT SUM(a), SUM(LENGTH(b)) FROM test
-----
-577344000.000000	7680000.000000
-
diff --git a/modules/calcite/src/test/sql/update/test_null_update.test b/modules/calcite/src/test/sql/update/test_null_update.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/update/test_null_update.test
rename to modules/calcite/src/test/sql/update/test_null_update.test_ignore
index 46dab87..94b63dd 100644
--- a/modules/calcite/src/test/sql/update/test_null_update.test
+++ b/modules/calcite/src/test/sql/update/test_null_update.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_null_update.test
 # description: Test standard update behavior with NULLs
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_repeated_string_update.test b/modules/calcite/src/test/sql/update/test_repeated_string_update.test_ignore
similarity index 93%
rename from modules/calcite/src/test/sql/update/test_repeated_string_update.test
rename to modules/calcite/src/test/sql/update/test_repeated_string_update.test_ignore
index 3084acc..d65b04b 100644
--- a/modules/calcite/src/test/sql/update/test_repeated_string_update.test
+++ b/modules/calcite/src/test/sql/update/test_repeated_string_update.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_repeated_string_update.test
 # description: Test repeated update of string in same segment
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_string_update.test b/modules/calcite/src/test/sql/update/test_string_update.test_ignore
similarity index 93%
rename from modules/calcite/src/test/sql/update/test_string_update.test
rename to modules/calcite/src/test/sql/update/test_string_update.test_ignore
index 73067d5..bbbaffd 100644
--- a/modules/calcite/src/test/sql/update/test_string_update.test
+++ b/modules/calcite/src/test/sql/update/test_string_update.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_string_update.test
 # description: Test update of string columns
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_string_update_many_strings.test b/modules/calcite/src/test/sql/update/test_string_update_many_strings.test_ignore
similarity index 96%
rename from modules/calcite/src/test/sql/update/test_string_update_many_strings.test
rename to modules/calcite/src/test/sql/update/test_string_update_many_strings.test_ignore
index c82997f..01f06d4 100644
--- a/modules/calcite/src/test/sql/update/test_string_update_many_strings.test
+++ b/modules/calcite/src/test/sql/update/test_string_update_many_strings.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_string_update_many_strings.test
 # description: Test string updates with many strings
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_string_update_null.test b/modules/calcite/src/test/sql/update/test_string_update_null.test_ignore
similarity index 90%
rename from modules/calcite/src/test/sql/update/test_string_update_null.test
rename to modules/calcite/src/test/sql/update/test_string_update_null.test_ignore
index e37dff9..a7dbd96 100644
--- a/modules/calcite/src/test/sql/update/test_string_update_null.test
+++ b/modules/calcite/src/test/sql/update/test_string_update_null.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_string_update_null.test
 # description: Test update of string columns with NULLs
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_string_update_rollback.test b/modules/calcite/src/test/sql/update/test_string_update_rollback.test_ignore
similarity index 96%
rename from modules/calcite/src/test/sql/update/test_string_update_rollback.test
rename to modules/calcite/src/test/sql/update/test_string_update_rollback.test_ignore
index 5b40ae8..3b125b7 100644
--- a/modules/calcite/src/test/sql/update/test_string_update_rollback.test
+++ b/modules/calcite/src/test/sql/update/test_string_update_rollback.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_string_update_rollback.test
 # description: Test rollback of string update
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_string_update_rollback_null.test b/modules/calcite/src/test/sql/update/test_string_update_rollback_null.test_ignore
similarity index 95%
rename from modules/calcite/src/test/sql/update/test_string_update_rollback_null.test
rename to modules/calcite/src/test/sql/update/test_string_update_rollback_null.test_ignore
index 94b5983..f960e8d 100644
--- a/modules/calcite/src/test/sql/update/test_string_update_rollback_null.test
+++ b/modules/calcite/src/test/sql/update/test_string_update_rollback_null.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_string_update_rollback_null.test
 # description: Test rollback of string update with NULL
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_update.test b/modules/calcite/src/test/sql/update/test_update.test_ignore
similarity index 93%
rename from modules/calcite/src/test/sql/update/test_update.test
rename to modules/calcite/src/test/sql/update/test_update.test_ignore
index 84f8d9d..761b8ad 100644
--- a/modules/calcite/src/test/sql/update/test_update.test
+++ b/modules/calcite/src/test/sql/update/test_update.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update.test
 # description: Test standard update behavior
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_update_delete_same_tuple.test b/modules/calcite/src/test/sql/update/test_update_delete_same_tuple.test_ignore
similarity index 91%
rename from modules/calcite/src/test/sql/update/test_update_delete_same_tuple.test
rename to modules/calcite/src/test/sql/update/test_update_delete_same_tuple.test_ignore
index 5094e7c..feaeddd 100644
--- a/modules/calcite/src/test/sql/update/test_update_delete_same_tuple.test
+++ b/modules/calcite/src/test/sql/update/test_update_delete_same_tuple.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update_delete_same_tuple.test
 # description: Test update and delete of the same tuple
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # on a normal table, we can update and delete the same tuple concurrently without a conflict
 statement ok con1
diff --git a/modules/calcite/src/test/sql/update/test_update_from.test b/modules/calcite/src/test/sql/update/test_update_from.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/update/test_update_from.test
rename to modules/calcite/src/test/sql/update/test_update_from.test_ignore
index 2fdf5dc..87b6f1e 100644
--- a/modules/calcite/src/test/sql/update/test_update_from.test
+++ b/modules/calcite/src/test/sql/update/test_update_from.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update_from.test
 # description: Test updates that use a from clause
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # create a table
 statement ok
diff --git a/modules/calcite/src/test/sql/update/test_update_many_updaters.test b/modules/calcite/src/test/sql/update/test_update_many_updaters.test_ignore
similarity index 98%
rename from modules/calcite/src/test/sql/update/test_update_many_updaters.test
rename to modules/calcite/src/test/sql/update/test_update_many_updaters.test_ignore
index ead512e..801ed5b 100644
--- a/modules/calcite/src/test/sql/update/test_update_many_updaters.test
+++ b/modules/calcite/src/test/sql/update/test_update_many_updaters.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update_many_updaters.test
 # description: Test update behavior with multiple updaters
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 # this test contains four query connections (con1, con2, con3, con4)
 # and one updating connection (updater)
diff --git a/modules/calcite/src/test/sql/update/test_update_many_updaters_nulls.test b/modules/calcite/src/test/sql/update/test_update_many_updaters_nulls.test_ignore
similarity index 96%
rename from modules/calcite/src/test/sql/update/test_update_many_updaters_nulls.test
rename to modules/calcite/src/test/sql/update/test_update_many_updaters_nulls.test_ignore
index 5e2ef17..2e347b8 100644
--- a/modules/calcite/src/test/sql/update/test_update_many_updaters_nulls.test
+++ b/modules/calcite/src/test/sql/update/test_update_many_updaters_nulls.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update_many_updaters_nulls.test
 # description: Test update behavior with multiple updaters and NULL values
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 statement ok con1
 PRAGMA enable_verification
diff --git a/modules/calcite/src/test/sql/update/test_update_mix.test b/modules/calcite/src/test/sql/update/test_update_mix.test_ignore
similarity index 94%
rename from modules/calcite/src/test/sql/update/test_update_mix.test
rename to modules/calcite/src/test/sql/update/test_update_mix.test_ignore
index 584392d..872e5a7 100644
--- a/modules/calcite/src/test/sql/update/test_update_mix.test
+++ b/modules/calcite/src/test/sql/update/test_update_mix.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update_mix.test
 # description: Test mix of updates inserts and deletes
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 statement ok con1
 CREATE TABLE test (a INTEGER);
diff --git a/modules/calcite/src/test/sql/update/test_update_same_string_value.test b/modules/calcite/src/test/sql/update/test_update_same_string_value.test
deleted file mode 100644
index e8653c4..0000000
--- a/modules/calcite/src/test/sql/update/test_update_same_string_value.test
+++ /dev/null
@@ -1,41 +0,0 @@
-# name: test/sql/update/test_update_same_string_value.test
-# description: Test repeated update of string in same transaction
-# group: [update]
-
-# create a table
-statement ok
-CREATE TABLE test (a VARCHAR);
-
-statement ok
-INSERT INTO test VALUES ('hello'), ('world')
-
-statement ok
-BEGIN TRANSACTION;
-
-statement ok
-UPDATE test SET a='test' WHERE a='hello';
-
-query T
-SELECT * FROM test ORDER BY a
-----
-test
-world
-
-statement ok
-UPDATE test SET a='test2' WHERE a='world';
-
-query T
-SELECT * FROM test ORDER BY a
-----
-test
-test2
-
-statement ok
-COMMIT;
-
-query T
-SELECT * FROM test ORDER BY a
-----
-test
-test2
-
diff --git a/modules/calcite/src/test/sql/update/test_update_same_value.test b/modules/calcite/src/test/sql/update/test_update_same_value.test_ignore
similarity index 97%
rename from modules/calcite/src/test/sql/update/test_update_same_value.test
rename to modules/calcite/src/test/sql/update/test_update_same_value.test_ignore
index 6fd626b..0c22e47 100644
--- a/modules/calcite/src/test/sql/update/test_update_same_value.test
+++ b/modules/calcite/src/test/sql/update/test_update_same_value.test_ignore
@@ -1,6 +1,7 @@
 # name: test/sql/update/test_update_same_value.test
 # description: Update the same value multiple times in one transaction
 # group: [update]
+# Ignore https://issues.apache.org/jira/browse/IGNITE-15624
 
 statement ok
 PRAGMA enable_verification